From 7a3604823cdbeb39c6cf7c1ff09efea9054b3e4b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 02:29:26 +0200 Subject: [PATCH 0001/1722] Stricter tryParse --- src/IO/ReadHelpers.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 2636898c1b3..98fafe37cff 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1398,6 +1398,7 @@ inline T parse(const char * data, size_t size) T res; ReadBufferFromMemory buf(data, size); readText(res, buf); + assertEOF(buf); return res; } @@ -1405,7 +1406,9 @@ template inline bool tryParse(T & res, const char * data, size_t size) { ReadBufferFromMemory buf(data, size); - return tryReadText(res, buf); + if (!tryReadText(res, buf)) + return false; + return buf.eof(); } template From 9dfed89dc85f017c1c8ebd0e077a07f793fab78b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Aug 2023 20:28:34 +0200 Subject: [PATCH 0002/1722] Fix Keeper --- src/Coordination/KeeperSnapshotManager.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a216272a9e1..f30a716206d 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -56,7 +56,7 @@ namespace std::filesystem::path path(snapshot_path); std::string filename = path.stem(); Strings name_parts; - splitInto<'_'>(name_parts, filename); + splitInto<'_', '.'>(name_parts, filename); return parse(name_parts[1]); } @@ -440,7 +440,6 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial } } - size_t active_sessions_size; readBinary(active_sessions_size, in); From aee9612e5127f9cbe0a05c12a08480fc5072ff44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Aug 2023 20:53:56 +0200 Subject: [PATCH 0003/1722] Fix strange trash --- src/Core/SettingsFields.cpp | 2 +- src/Core/SettingsFields.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 9b8677a9888..207a7464228 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -191,7 +191,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - return "'auto(" + ::DB::toString(value) + ")'"; + return "auto(" + ::DB::toString(value) + ")"; else return ::DB::toString(value); } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 32fffd3af06..e25d688bcf4 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -155,7 +155,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto==true`. + /// Writes "auto()" instead of simple "" if `is_auto == true`. String toString() const; void parseFromString(const String & str); From 347a2ed46cfa9b759d159735d43e5d303471f6d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 01:43:54 +0200 Subject: [PATCH 0004/1722] Fix a comment --- base/poco/Data/include/Poco/Data/TypeHandler.h | 2 +- programs/format/Format.cpp | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/base/poco/Data/include/Poco/Data/TypeHandler.h b/base/poco/Data/include/Poco/Data/TypeHandler.h index 34f88e986f7..e7633de7018 100644 --- a/base/poco/Data/include/Poco/Data/TypeHandler.h +++ b/base/poco/Data/include/Poco/Data/TypeHandler.h @@ -97,7 +97,7 @@ namespace Data /// /// static void extract(std::size_t pos, Person& obj, const Person& defVal, AbstractExtractor::Ptr pExt) /// { - /// // defVal is the default person we should use if we encunter NULL entries, so we take the individual fields + /// // defVal is the default person we should use if we encounter NULL entries, so we take the individual fields /// // as defaults. You can do more complex checking, ie return defVal if only one single entry of the fields is null etc... /// poco_assert_dbg (!pExt.isNull()); /// std::string lastName; diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 43c66a32302..d7d61bbcd3b 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -163,13 +163,15 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { ASTPtr res = parseQueryAndMovePosition( parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth); - /// For insert query with data(INSERT INTO ... VALUES ...), will lead to format fail, - /// should throw exception early and make exception message more readable. + + /// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure, + /// we should throw an exception early, and make exception message more readable. if (const auto * insert_query = res->as(); insert_query && insert_query->data) { throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA, "Can't format ASTInsertQuery with data, since data will be lost"); } + if (!quiet) { if (!backslash) From 576a8406946b085f9438328f3e8bc1b9e1f66fb2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Oct 2023 07:20:28 +0200 Subject: [PATCH 0005/1722] Fix error --- src/Interpreters/Cluster.cpp | 36 +++++++++++++++++++++++++----------- src/Interpreters/Cluster.h | 2 +- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 82c3d48bc05..1819e00ac4f 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -225,21 +225,31 @@ String Cluster::Address::toFullString(bool use_compact_format) const } } -Cluster::Address Cluster::Address::fromFullString(const String & full_string) +Cluster::Address Cluster::Address::fromFullString(std::string_view full_string) { - const char * address_begin = full_string.data(); - const char * address_end = address_begin + full_string.size(); - - const char * user_pw_end = strchr(full_string.data(), '@'); + std::string_view user_password; + if (auto pos = full_string.find('@'); pos != std::string_view::npos) + user_password = full_string.substr(pos + 1); /// parsing with the new shard{shard_index}[_replica{replica_index}] format - if (!user_pw_end && startsWith(full_string, "shard")) + if (user_password.empty() && full_string.starts_with("shard")) { - const char * underscore = strchr(full_string.data(), '_'); - Address address; - address.shard_index = parse(address_begin + strlen("shard")); - address.replica_index = underscore ? parse(underscore + strlen("_replica")) : 0; + + if (auto underscore_pos = full_string.find('_'); underscore_pos != std::string_view::npos) + { + address.shard_index = parse(full_string.substr(0, underscore_pos).substr(strlen("shard"))); + + if (!full_string.substr(underscore_pos + 1).starts_with("replica")) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Incorrect address '{}', should be in a form of `shardN` or `shardN_replicaM`", full_string); + + address.replica_index = parse(full_string.substr(underscore_pos + 1 + strlen("replica"))); + } + else + { + address.shard_index = parse(full_string.substr(strlen("shard"))); + address.replica_index = 0; + } return address; } @@ -250,9 +260,13 @@ Cluster::Address Cluster::Address::fromFullString(const String & full_string) /// - credentials are exposed in file name; /// - the file name can be too long. + const char * address_begin = full_string.data(); + const char * address_end = address_begin + full_string.size(); + const char * user_pw_end = strchr(address_begin, '@'); + Protocol::Secure secure = Protocol::Secure::Disable; const char * secure_tag = "+secure"; - if (endsWith(full_string, secure_tag)) + if (full_string.ends_with(secure_tag)) { address_end -= strlen(secure_tag); secure = Protocol::Secure::Enable; diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index b2bc03dd74d..042a43040c2 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -166,7 +166,7 @@ public: String toFullString(bool use_compact_format) const; /// Returns address with only shard index and replica index or full address without shard index and replica index - static Address fromFullString(const String & address_full_string); + static Address fromFullString(std::string_view full_string); /// Returns resolved address if it does resolve. std::optional getResolvedAddress() const; From f2757a5c645d4747d328744c0433a3acd389845c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Oct 2023 07:23:25 +0200 Subject: [PATCH 0006/1722] Fix error --- src/Interpreters/Cluster.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 1819e00ac4f..027b3cdf76b 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -240,10 +240,16 @@ Cluster::Address Cluster::Address::fromFullString(std::string_view full_string) { address.shard_index = parse(full_string.substr(0, underscore_pos).substr(strlen("shard"))); - if (!full_string.substr(underscore_pos + 1).starts_with("replica")) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Incorrect address '{}', should be in a form of `shardN` or `shardN_replicaM`", full_string); - - address.replica_index = parse(full_string.substr(underscore_pos + 1 + strlen("replica"))); + if (full_string.substr(underscore_pos + 1).starts_with("replica")) + { + address.replica_index = parse(full_string.substr(underscore_pos + 1 + strlen("replica"))); + } + else if (full_string.substr(underscore_pos + 1).starts_with("all_replicas")) + { + address.replica_index = 0; + } + else + throw Exception(ErrorCodes::SYNTAX_ERROR, "Incorrect address '{}', should be in a form of `shardN_all_replicas` or `shardN_replicaM`", full_string); } else { From 753ea3ab889684e2f994c424821b8f7923f55534 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Oct 2023 20:04:10 +0200 Subject: [PATCH 0007/1722] Fix unit test --- src/Common/ZooKeeper/ZooKeeper.h | 1 + src/Coordination/RaftServerConfig.cpp | 8 ++++++-- src/Databases/TablesLoader.h | 2 ++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c41d1d8dbab..15cbd8b2f89 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -548,6 +548,7 @@ public: void setZooKeeperLog(std::shared_ptr zk_log_); UInt32 getSessionUptime() const { return static_cast(session_uptime.elapsedSeconds()); } + bool hasReachedDeadline() const { return impl->hasReachedDeadline(); } void setServerCompletelyStarted(); diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 929eeeb640e..bafc177b736 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -26,12 +26,16 @@ std::optional RaftServerConfig::parse(std::string_view server) if (!with_id_endpoint && !with_server_type && !with_priority) return std::nullopt; - const std::string_view id_str = parts[0]; + std::string_view id_str = parts[0]; if (!id_str.starts_with("server.")) return std::nullopt; + id_str = id_str.substr(7); + if (auto eq_pos = id_str.find('='); std::string_view::npos != eq_pos) + id_str = id_str.substr(0, eq_pos); + Int32 id; - if (!tryParse(id, std::next(id_str.begin(), 7))) + if (!tryParse(id, id_str)) return std::nullopt; if (id <= 0) return std::nullopt; diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index eb07351bd7f..31702db0345 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -12,6 +13,7 @@ #include #include + namespace Poco { class Logger; // NOLINT(cppcoreguidelines-virtual-class-destructor) From 0f3506bf7af20231a290ed97a59b0e233fbc7c6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Oct 2023 20:05:45 +0200 Subject: [PATCH 0008/1722] Fix integration test --- tests/integration/test_cgroup_limit/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index e77b0f70960..5d56135d9ff 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -46,7 +46,7 @@ def test_cgroup_cpu_limit(): "clickhouse local -q \"select value from system.settings where name='max_threads'\"", num_cpus, ) - expect_output = (r"\'auto({})\'".format(math.ceil(num_cpus))).encode() + expect_output = (r"auto({})".format(math.ceil(num_cpus))).encode() assert ( result.strip() == expect_output ), f"fail for cpu limit={num_cpus}, result={result.strip()}, expect={expect_output}" From cd94eda70457bac2e13534d8bf58d42c855af649 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Nov 2023 13:25:47 +0000 Subject: [PATCH 0009/1722] Simplify --- src/Functions/toStartOfInterval.cpp | 130 +++++++++++++++++----------- 1 file changed, 78 insertions(+), 52 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 48bf88cb14c..0a3a409ea9f 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -324,38 +324,61 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - bool first_argument_is_date = false; + bool value_is_date = false; auto check_first_argument = [&] { - if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + const DataTypePtr & type_arg1 = arguments[0].type; + if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Should be a date or a date with time", arguments[0].type->getName(), getName()); - first_argument_is_date = isDate(arguments[0].type); + "Should be a date or a date with time", type_arg1->getName(), getName()); + value_is_date = isDate(type_arg1); }; const DataTypeInterval * interval_type = nullptr; - bool result_type_is_date = false; - bool result_type_is_datetime = false; - auto check_interval_argument = [&] + enum class ResultType { - interval_type = checkAndGetDataType(arguments[1].type.get()); + Date, + DateTime, + DateTime64 + }; + ResultType result_type; + auto check_second_argument = [&] + { + const DataTypePtr & type_arg2 = arguments[1].type; + interval_type = checkAndGetDataType(type_arg2.get()); if (!interval_type) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Should be an interval of time", arguments[1].type->getName(), getName()); - result_type_is_date = (interval_type->getKind() == IntervalKind::Year) - || (interval_type->getKind() == IntervalKind::Quarter) || (interval_type->getKind() == IntervalKind::Month) - || (interval_type->getKind() == IntervalKind::Week); - result_type_is_datetime = (interval_type->getKind() == IntervalKind::Day) || (interval_type->getKind() == IntervalKind::Hour) - || (interval_type->getKind() == IntervalKind::Minute) || (interval_type->getKind() == IntervalKind::Second); + "Should be an interval of time", type_arg2->getName(), getName()); + switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) + { + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: + result_type = ResultType::DateTime64; + break; + case IntervalKind::Second: + case IntervalKind::Minute: + case IntervalKind::Hour: + case IntervalKind::Day: + result_type = ResultType::DateTime; + break; + case IntervalKind::Week: + case IntervalKind::Month: + case IntervalKind::Quarter: + case IntervalKind::Year: + result_type = ResultType::Date; + break; + } }; - auto check_timezone_argument = [&] + auto check_third_argument = [&] { - if (!WhichDataType(arguments[2].type).isString()) + const DataTypePtr & type_arg3 = arguments[2].type; + if (!isString(type_arg3)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " "This argument is optional and must be a constant string with timezone name", - arguments[2].type->getName(), getName()); - if (first_argument_is_date && result_type_is_date) + type_arg3->getName(), getName()); + if (value_is_date && result_type == ResultType::Date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The timezone argument of function {} with interval type {} is allowed only when the 1st argument " "has the type DateTime or DateTime64", @@ -365,13 +388,13 @@ public: if (arguments.size() == 2) { check_first_argument(); - check_interval_argument(); + check_second_argument(); } else if (arguments.size() == 3) { check_first_argument(); - check_interval_argument(); - check_timezone_argument(); + check_second_argument(); + check_third_argument(); } else { @@ -380,24 +403,27 @@ public: getName(), arguments.size()); } - if (result_type_is_date) - return std::make_shared(); - else if (result_type_is_datetime) - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); - else + switch (result_type) { - auto scale = 0; + case ResultType::Date: + return std::make_shared(); + case ResultType::DateTime: + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + case ResultType::DateTime64: + { + UInt32 scale = 0; + if (interval_type->getKind() == IntervalKind::Nanosecond) + scale = 9; + else if (interval_type->getKind() == IntervalKind::Microsecond) + scale = 6; + else if (interval_type->getKind() == IntervalKind::Millisecond) + scale = 3; - if (interval_type->getKind() == IntervalKind::Nanosecond) - scale = 9; - else if (interval_type->getKind() == IntervalKind::Microsecond) - scale = 6; - else if (interval_type->getKind() == IntervalKind::Millisecond) - scale = 3; - - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + } } + std::unreachable(); } bool useDefaultImplementationForConstants() const override { return true; } @@ -426,34 +452,34 @@ private: ColumnPtr dispatchForColumns( const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const { - const auto & from_datatype = *time_column.type.get(); - const auto which_type = WhichDataType(from_datatype); + const auto & time_column_type = *time_column.type.get(); + const auto & time_column_col = *time_column.column.get(); - if (which_type.isDateTime64()) + if (isDateTime64(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - auto scale = assert_cast(from_datatype).getScale(); + const auto * time_column_vec = checkAndGetColumn(time_column_col); + auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, scale); } - if (which_type.isDateTime()) + else if (isDateTime(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } - if (which_type.isDate()) + else if (isDate(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } - if (which_type.isDate32()) + else if (isDate32(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " "Must contain dates or dates with time", getName()); @@ -502,7 +528,7 @@ private: return execute(from, time_column, num_units, result_type, time_zone, scale); } - UNREACHABLE(); + std::unreachable(); } template @@ -515,7 +541,7 @@ private: size_t size = time_data.size(); auto result_col = result_type->createColumn(); - auto *col_to = assert_cast(result_col.get()); + auto * col_to = assert_cast(result_col.get()); auto & result_data = col_to->getData(); result_data.resize(size); From aef1ea3df71d7d19c6267f6bb03ce994b7e14909 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 14 Nov 2023 17:49:08 +0000 Subject: [PATCH 0010/1722] Extend with origin argument, pt. I --- src/Functions/toStartOfInterval.cpp | 128 +++++++++++++++++++++------- 1 file changed, 95 insertions(+), 33 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 0a3a409ea9f..4d716637932 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -321,6 +321,11 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } + + bool hasInformationAboutMonotonicity() const override { return true; } + Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { return { .is_monotonic = true, .is_always_monotonic = true }; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -329,7 +334,7 @@ public: { const DataTypePtr & type_arg1 = arguments[0].type; if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 1st argument of function {}. " "Should be a date or a date with time", type_arg1->getName(), getName()); value_is_date = isDate(type_arg1); }; @@ -347,7 +352,7 @@ public: const DataTypePtr & type_arg2 = arguments[1].type; interval_type = checkAndGetDataType(type_arg2.get()); if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " "Should be an interval of time", type_arg2->getName(), getName()); switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { @@ -371,18 +376,48 @@ public: } }; + enum class ThirdArgument + { + IsTimezone, + IsOrigin + }; + ThirdArgument third_argument; /// valid only if 3rd argument is given auto check_third_argument = [&] { const DataTypePtr & type_arg3 = arguments[2].type; - if (!isString(type_arg3)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "This argument is optional and must be a constant string with timezone name", + if (isString(type_arg3)) + { + third_argument = ThirdArgument::IsTimezone; + if (value_is_date && result_type == ResultType::Date) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", + getName(), interval_type->getKind().toString()); + } + else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3)) + third_argument = ThirdArgument::IsOrigin; + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " + "This argument is optional and must be a constant String with timezone name or a Date/Date32/DateTime/DateTime64 with a constant origin", type_arg3->getName(), getName()); + + }; + + auto check_fourth_argument = [&] + { + if (third_argument != ThirdArgument::IsOrigin) /// sanity check + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " + "The third argument must a Date/Date32/DateTime/DateTime64 with a constant origin", + arguments[2].type->getName(), getName()); + + const DataTypePtr & type_arg4 = arguments[3].type; + if (!isString(type_arg4)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 4th argument of function {}. " + "This argument is optional and must be a constant String with timezone name", + type_arg4->getName(), getName()); if (value_is_date && result_type == ResultType::Date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The timezone argument of function {} with interval type {} is allowed only when the 1st argument " - "has the type DateTime or DateTime64", - getName(), interval_type->getKind().toString()); + "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", + getName(), interval_type->getKind().toString()); }; if (arguments.size() == 2) @@ -396,10 +431,17 @@ public: check_second_argument(); check_third_argument(); } + else if (arguments.size() == 4) + { + check_first_argument(); + check_second_argument(); + check_third_argument(); + check_fourth_argument(); + } else { throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + "Number of arguments for function {} doesn't match: passed {}, should be 2, 3 or 4", getName(), arguments.size()); } @@ -408,7 +450,10 @@ public: case ResultType::Date: return std::make_shared(); case ResultType::DateTime: - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + { + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); + } case ResultType::DateTime64: { UInt32 scale = 0; @@ -419,7 +464,8 @@ public: else if (interval_type->getKind() == IntervalKind::Millisecond) scale = 3; - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); } } @@ -433,8 +479,16 @@ public: { const auto & time_column = arguments[0]; const auto & interval_column = arguments[1]; - const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - auto result_column = dispatchForColumns(time_column, interval_column, result_type, time_zone); + + ColumnWithTypeAndName origin_column; + const bool has_origin_arg = (arguments.size() == 3 && isDateOrDate32OrDateTimeOrDateTime64(arguments[2].type)) || arguments.size() == 4; + if (has_origin_arg) + origin_column = arguments[2]; + + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3; + const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); + + auto result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); return result_column; } @@ -449,8 +503,8 @@ public: } private: - ColumnPtr dispatchForColumns( - const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const + ColumnPtr dispatchForTimeColumn( + const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const { const auto & time_column_type = *time_column.type.get(); const auto & time_column_col = *time_column.column.get(); @@ -461,25 +515,25 @@ private: auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, scale); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale); } else if (isDateTime(time_column_type)) { const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate(time_column_type)) { const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate32(time_column_type)) { const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " "Must contain dates or dates with time", getName()); @@ -487,7 +541,7 @@ private: template ColumnPtr dispatchForIntervalColumn( - const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column, + const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); @@ -505,34 +559,34 @@ private: switch (interval_type->getKind()) { case IntervalKind::Nanosecond: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Microsecond: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Millisecond: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Second: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Minute: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Hour: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Day: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Week: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Month: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Quarter: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Year: - return execute(from, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); } std::unreachable(); } - template - ColumnPtr execute(const FromDataType &, const ColumnType & time_column_type, Int64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const + template + ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, [[maybe_unused]] const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const { using ToColumnType = typename ToDataType::ColumnType; using ToFieldType = typename ToDataType::FieldType; @@ -547,6 +601,14 @@ private: Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); + /// TODO: This part is missing. origin_column is either {} (<-- to check, you could do `origin_column.column == nullptr`) or not {} + /// In the former case, we can execute below existing code. + /// In the latter case, we need to read the actual origin value. As per `getArgumentsThatAreAlwaysConstant()` (see above), we + /// can be sure that origin_column is a `ColumnConst`. The second assumption we can reasonable make is that it has the same + /// type (Date/Date32/DateTime/DateTime64) as the time column (1st argument). Since the method we are in is already + /// templatized on the data type of the time column, we can use `checkAndGetColumnConst(...)` to cast + /// `origin_column.column` to a const column and then read the (const) value from it, and proceed with the calculations. + for (size_t i = 0; i != size; ++i) result_data[i] = static_cast( Transform::execute(time_data[i], num_units, time_zone, scale_multiplier)); From 188c469318e8a7a7e339bcf4048952181e4e324c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 Nov 2023 14:22:39 +0000 Subject: [PATCH 0011/1722] done suggestings about logic --- src/Functions/toStartOfInterval.cpp | 341 +++------------------------- 1 file changed, 27 insertions(+), 314 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 4d716637932..29f8faa810c 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include #include #include @@ -21,300 +23,18 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; - extern const int DECIMAL_OVERFLOW; } namespace { - constexpr auto function_name = "toStartOfInterval"; - - template - struct Transform; - - template <> - struct Transform - { - static UInt16 execute(UInt16 d, Int64 years, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfYearInterval(DayNum(d), years); - } - - static UInt16 execute(Int32 d, Int64 years, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years); - } - - static UInt16 execute(UInt32 t, Int64 years, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); - } - - static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); - } - }; - - template <> - struct Transform - { - static UInt16 execute(UInt16 d, Int64 quarters, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfQuarterInterval(DayNum(d), quarters); - } - - static UInt16 execute(Int32 d, Int64 quarters, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters); - } - - static UInt16 execute(UInt32 t, Int64 quarters, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); - } - - static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); - } - }; - - template <> - struct Transform - { - static UInt16 execute(UInt16 d, Int64 months, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMonthInterval(DayNum(d), months); - } - - static UInt16 execute(Int32 d, Int64 months, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months); - } - - static UInt16 execute(UInt32 t, Int64 months, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); - } - - static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); - } - }; - - template <> - struct Transform - { - static UInt16 execute(UInt16 d, Int64 weeks, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfWeekInterval(DayNum(d), weeks); - } - - static UInt16 execute(Int32 d, Int64 weeks, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks); - } - - static UInt16 execute(UInt32 t, Int64 weeks, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); - } - - static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64) - { - return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); - } - - static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64) - { - return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); - } - - static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64) - { - return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days)); - } - - static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days); - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfHourInterval(t, hours); - } - - static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfHourInterval(t / scale_multiplier, hours); - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMinuteInterval(t, minutes); - } - - static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes); - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfSecondInterval(t, seconds); - } - - static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds); - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } - - static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier) - { - if (scale_multiplier < 1000) - { - Int64 t_milliseconds = 0; - if (common::mulOverflow(t, static_cast(1000) / scale_multiplier, t_milliseconds)) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); - if (likely(t >= 0)) - return t_milliseconds / milliseconds * milliseconds; - else - return ((t_milliseconds + 1) / milliseconds - 1) * milliseconds; - } - else if (scale_multiplier > 1000) - { - Int64 scale_diff = scale_multiplier / static_cast(1000); - if (likely(t >= 0)) - return t / milliseconds / scale_diff * milliseconds; - else - return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds; - } - else - if (likely(t >= 0)) - return t / milliseconds * milliseconds; - else - return ((t + 1) / milliseconds - 1) * milliseconds; - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } - - static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier) - { - if (scale_multiplier < 1000000) - { - Int64 t_microseconds = 0; - if (common::mulOverflow(t, static_cast(1000000) / scale_multiplier, t_microseconds)) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); - if (likely(t >= 0)) - return t_microseconds / microseconds * microseconds; - else - return ((t_microseconds + 1) / microseconds - 1) * microseconds; - } - else if (scale_multiplier > 1000000) - { - Int64 scale_diff = scale_multiplier / static_cast(1000000); - if (likely(t >= 0)) - return t / microseconds / scale_diff * microseconds; - else - return ((t + 1) / microseconds / scale_diff - 1) * microseconds; - } - else - if (likely(t >= 0)) - return t / microseconds * microseconds; - else - return ((t + 1) / microseconds - 1) * microseconds; - } - }; - - template <> - struct Transform - { - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } - - static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier) - { - if (scale_multiplier < 1000000000) - { - Int64 t_nanoseconds = 0; - if (common::mulOverflow(t, (static_cast(1000000000) / scale_multiplier), t_nanoseconds)) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); - if (likely(t >= 0)) - return t_nanoseconds / nanoseconds * nanoseconds; - else - return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds; - } - else - if (likely(t >= 0)) - return t / nanoseconds * nanoseconds; - else - return ((t + 1) / nanoseconds - 1) * nanoseconds; - } - }; class FunctionToStartOfInterval : public IFunction { public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } - static constexpr auto name = function_name; + static constexpr auto name = "toStartOfInterval"; String getName() const override { return name; } bool isVariadic() const override { return true; } @@ -472,9 +192,6 @@ public: std::unreachable(); } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override { const auto & time_column = arguments[0]; @@ -485,23 +202,13 @@ public: if (has_origin_arg) origin_column = arguments[2]; - const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3; + const size_t time_zone_arg_num = (arguments.size() == 3 && isString(arguments[2].type)) ? 2 : 3; const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); auto result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); return result_column; } - bool hasInformationAboutMonotonicity() const override - { - return true; - } - - Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override - { - return { .is_monotonic = true, .is_always_monotonic = true }; - } - private: ColumnPtr dispatchForTimeColumn( const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const @@ -535,11 +242,10 @@ private: if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. " - "Must contain dates or dates with time", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. Must contain dates or dates with time", getName()); } - template + template ColumnPtr dispatchForIntervalColumn( const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const @@ -547,16 +253,16 @@ private: const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be an interval of time.", getName()); + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for second argument of function {}, must be a const interval of time.", - getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be a const interval of time.", getName()); + Int64 num_units = interval_column_const_int64->getValue(); if (num_units <= 0) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for second argument of function {} must be positive.", getName()); - switch (interval_type->getKind()) + switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Nanosecond: return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); @@ -601,17 +307,24 @@ private: Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); - /// TODO: This part is missing. origin_column is either {} (<-- to check, you could do `origin_column.column == nullptr`) or not {} - /// In the former case, we can execute below existing code. - /// In the latter case, we need to read the actual origin value. As per `getArgumentsThatAreAlwaysConstant()` (see above), we - /// can be sure that origin_column is a `ColumnConst`. The second assumption we can reasonable make is that it has the same - /// type (Date/Date32/DateTime/DateTime64) as the time column (1st argument). Since the method we are in is already - /// templatized on the data type of the time column, we can use `checkAndGetColumnConst(...)` to cast - /// `origin_column.column` to a const column and then read the (const) value from it, and proceed with the calculations. + if (origin_column.column == nullptr) + for (size_t i = 0; i != size; ++i) + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + else + { + UInt64 od = origin_column.column->get64(0); + + for (size_t i = 0; i != size; ++i) + { + auto td = time_data[i]; + if (od > size_t(td)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); + td -= od; + result_data[i] = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier)); - for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast( - Transform::execute(time_data[i], num_units, time_zone, scale_multiplier)); + result_data[i] += scale_multiplier == 10 ? od : od / scale_multiplier; + } + } return result_col; } From 6605a375b642d0bc8b6ffccbd0400eb4ccee223e Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 Nov 2023 17:35:04 +0000 Subject: [PATCH 0012/1722] style check, docs, tests and modified logic --- .../functions/date-time-functions.md | 7 +- src/Functions/toStartOfInterval.cpp | 306 +++++++++++++++++- ...tart_of_interval_origin_overload.reference | 15 + ...6_to_start_of_interval_origin_overload.sql | 12 + 4 files changed, 335 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference create mode 100644 tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 0364a610404..3f522eeb164 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -590,7 +590,7 @@ Rounds down a date with time to the start of the ten-minute interval. Rounds down the date with time to the start of the fifteen-minute interval. -## toStartOfInterval(time_or_data, INTERVAL x unit \[, time_zone\]) +## toStartOfInterval(time_or_data, INTERVAL x unit \[, origin_time_or_data\] \[, time_zone\]) This is a generalization of other functions named `toStartOf*`. For example, `toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`, @@ -598,6 +598,11 @@ This is a generalization of other functions named `toStartOf*`. For example, `toStartOfInterval(t, INTERVAL 1 day)` returns the same as `toStartOfDay(t)`, `toStartOfInterval(t, INTERVAL 15 minute)` returns the same as `toStartOfFifteenMinutes(t)` etc. +Also it has an overload including `origin_time_or_data` column which emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function: +``` SQL +SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), toIntervalMinute(1), toDateTime('2023-01-01 14:35:30')); +``` + ## toTime Converts a date with time to a certain fixed date, while preserving the time. diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 29f8faa810c..8529643ee70 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -23,11 +24,292 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int BAD_ARGUMENTS; } namespace { +constexpr auto function_name = "toStartOfInterval"; + +template +struct Transform; + +template <> +struct Transform +{ + static UInt16 execute(UInt16 d, Int64 years, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfYearInterval(DayNum(d), years); + } + + static UInt16 execute(Int32 d, Int64 years, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years); + } + + static UInt16 execute(UInt32 t, Int64 years, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); + } + + static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); + } +}; + +template <> +struct Transform +{ + static UInt16 execute(UInt16 d, Int64 quarters, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfQuarterInterval(DayNum(d), quarters); + } + + static UInt16 execute(Int32 d, Int64 quarters, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters); + } + + static UInt16 execute(UInt32 t, Int64 quarters, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); + } + + static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); + } +}; + +template <> +struct Transform +{ + static UInt16 execute(UInt16 d, Int64 months, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMonthInterval(DayNum(d), months); + } + + static UInt16 execute(Int32 d, Int64 months, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months); + } + + static UInt16 execute(UInt32 t, Int64 months, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); + } + + static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); + } +}; + +template <> +struct Transform +{ + static UInt16 execute(UInt16 d, Int64 weeks, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfWeekInterval(DayNum(d), weeks); + } +static UInt16 execute(Int32 d, Int64 weeks, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks); + } + + static UInt16 execute(UInt32 t, Int64 weeks, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); + } + + static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64) + { + return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); + } + + static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64) + { + return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); + } + + static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64) + { + return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days)); + } + + static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days); + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfHourInterval(t, hours); + } + + static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfHourInterval(t / scale_multiplier, hours); + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfMinuteInterval(t, minutes); + } + + static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes); + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64) + { + return time_zone.toStartOfSecondInterval(t, seconds); + } + + static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier) + { + return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds); + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } + + static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier) + { + if (scale_multiplier < 1000) + { + Int64 t_milliseconds = 0; + if (common::mulOverflow(t, static_cast(1000) / scale_multiplier, t_milliseconds)) + throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); + if (likely(t >= 0)) + return t_milliseconds / milliseconds * milliseconds; + else + return ((t_milliseconds + 1) / milliseconds - 1) * milliseconds; + } + else if (scale_multiplier > 1000) + { + Int64 scale_diff = scale_multiplier / static_cast(1000); + if (likely(t >= 0)) + return t / milliseconds / scale_diff * milliseconds; + else + return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds; + } + else + if (likely(t >= 0)) + return t / milliseconds * milliseconds; + else + return ((t + 1) / milliseconds - 1) * milliseconds; + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } + + static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier) + { + if (scale_multiplier < 1000000) + { + Int64 t_microseconds = 0; + if (common::mulOverflow(t, static_cast(1000000) / scale_multiplier, t_microseconds)) + throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); + if (likely(t >= 0)) + return t_microseconds / microseconds * microseconds; + else + return ((t_microseconds + 1) / microseconds - 1) * microseconds; + } + else if (scale_multiplier > 1000000) + { + Int64 scale_diff = scale_multiplier / static_cast(1000000); + if (likely(t >= 0)) + return t / microseconds / scale_diff * microseconds; + else + return ((t + 1) / microseconds / scale_diff - 1) * microseconds; + } + else + if (likely(t >= 0)) + return t / microseconds * microseconds; + else + return ((t + 1) / microseconds - 1) * microseconds; + } +}; + +template <> +struct Transform +{ + static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } + + static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } + + static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier) + { + if (scale_multiplier < 1000000000) + { + Int64 t_nanoseconds = 0; + if (common::mulOverflow(t, (static_cast(1000000000) / scale_multiplier), t_nanoseconds)) + throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); + if (likely(t >= 0)) + return t_nanoseconds / nanoseconds * nanoseconds; + else + return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds; + } + else + if (likely(t >= 0)) + return t / nanoseconds * nanoseconds; + else + return ((t + 1) / nanoseconds - 1) * nanoseconds; + } +}; class FunctionToStartOfInterval : public IFunction { @@ -218,6 +500,10 @@ private: if (isDateTime64(time_column_type)) { + if (origin_column.column != nullptr) + if (!isDateTime64(origin_column.type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); auto scale = assert_cast(time_column_type).getScale(); @@ -226,18 +512,30 @@ private: } else if (isDateTime(time_column_type)) { + if (origin_column.column != nullptr) + if (!isDateTime(origin_column.type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate(time_column_type)) { + if (origin_column.column != nullptr) + if (!isDate(origin_column.type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate32(time_column_type)) { + if (origin_column.column != nullptr) + if (!isDate32(origin_column.type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); @@ -292,7 +590,7 @@ private: } template - ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, [[maybe_unused]] const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const + ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const { using ToColumnType = typename ToDataType::ColumnType; using ToFieldType = typename ToDataType::FieldType; @@ -309,18 +607,18 @@ private: if (origin_column.column == nullptr) for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + result_data[i] = static_cast(Transform::execute(time_data[i], num_units, time_zone, scale_multiplier)); else { UInt64 od = origin_column.column->get64(0); - + for (size_t i = 0; i != size; ++i) { auto td = time_data[i]; if (od > size_t(td)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); td -= od; - result_data[i] = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier)); + result_data[i] = static_cast(Transform::execute(td, num_units, time_zone, scale_multiplier)); result_data[i] += scale_multiplier == 10 ? od : od / scale_multiplier; } diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference new file mode 100644 index 00000000000..7213925fb64 --- /dev/null +++ b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference @@ -0,0 +1,15 @@ +2023-03-01 16:55:00 +2023-02-01 16:55:00 +2023-03-01 16:55:00 +2023-02-01 16:55:00 +2023-03-01 16:55:00 +2023-03-01 16:55:00 +2023-02-01 16:55:00 +2023-03-01 16:55:00 +2023-02-01 16:55:00 +2023-03-01 16:55:00 +2023-01-02 15:44:30 +2023-01-02 15:44:30 +2023-01-02 14:45:30 +2023-01-02 14:45:30 +2023-01-02 diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql new file mode 100644 index 00000000000..ce4c8f87811 --- /dev/null +++ b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql @@ -0,0 +1,12 @@ +set session_timezone = 'UTC'; +SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalMinute(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalWeek(1), toDate('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError 43 } +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError 43 } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError 36 } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00'), 'Europe/Amsterdam') from numbers(1); -- { serverError 44 } From f596ae7a2c02e7915917daca82dad717a7df2640 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 18 Nov 2023 16:24:56 +0000 Subject: [PATCH 0013/1722] Fix style --- src/Functions/toStartOfInterval.cpp | 7 ++++--- utils/check-style/aspell-ignore/en/aspell-dict.txt | 9 +++++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 8529643ee70..10792922afb 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -20,11 +20,12 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; extern const int BAD_ARGUMENTS; + extern const int DECIMAL_OVERFLOW; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 5d11185ff76..7eaafe8a777 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -261,6 +261,7 @@ FOSDEM FQDN Failover FarmHash +FileLog FilesystemCacheBytes FilesystemCacheElements FilesystemCacheFiles @@ -278,7 +279,6 @@ FilesystemMainPathTotalBytes FilesystemMainPathTotalINodes FilesystemMainPathUsedBytes FilesystemMainPathUsedINodes -FileLog FixedString Flink ForEach @@ -571,13 +571,13 @@ NetworkSendPackets NodeJs NuRaft NumHexagons +NumPy NumToString NumToStringClassC NumberOfDatabases NumberOfDetachedByUserParts NumberOfDetachedParts NumberOfTables -NumPy OFNS OLAP OLTP @@ -588,10 +588,10 @@ OSGuestNiceTimeNormalized OSGuestTime OSGuestTimeCPU OSGuestTimeNormalized +OSIOWaitMicroseconds OSIOWaitTime OSIOWaitTimeCPU OSIOWaitTimeNormalized -OSIOWaitMicroseconds OSIdleTime OSIdleTimeCPU OSIdleTimeNormalized @@ -900,6 +900,7 @@ ThreadPoolRemoteFSReaderThreads ThreadPoolRemoteFSReaderThreadsActive ThreadsActive ThreadsInOvercommitTracker +TimescaleDB's Timeunit TinyLog Tkachenko @@ -1470,12 +1471,12 @@ fastops fcoverage fibonacci fifo +filelog filesystem filesystemAvailable filesystemCapacity filesystemFree filesystems -filelog finalizeAggregation fips firstLine From ea2ba82c1001febabdd8753246b64143e4bf1f6f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 18 Nov 2023 16:37:50 +0000 Subject: [PATCH 0014/1722] Update docs --- .../functions/date-time-functions.md | 36 ++++++++++++------- 1 file changed, 23 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 1291b570da4..989b39e46c1 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1345,7 +1345,7 @@ toStartOfTenMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:20:00 Rounds down the date with time to the start of the fifteen-minute interval. -## toStartOfInterval(date_or_date_with_time, INTERVAL x unit \[, origin_time_or_data\] \[, time_zone\]) +## toStartOfInterval This function generalizes other `toStartOf*()` functions. For example, - `toStartOfInterval(t, INTERVAL 1 year)` returns the same as `toStartOfYear(t)`, @@ -1372,21 +1372,21 @@ The calculation is performed relative to specific points in time: (*) hour intervals are special: the calculation is always performed relative to 00:00:00 (midnight) of the current day. As a result, only hour values between 1 and 23 are useful. -Also it has an overload including `origin_time_or_data` column which emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function: -``` SQL -SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), toIntervalMinute(1), toDateTime('2023-01-01 14:35:30')); +**Syntax** + +```sql +toStartOfInterval(value, INTERVAL x unit[, time_zone]) +toStartOfInterval(value, INTERVAL x unit[, origin[, time_zone]]) ``` +The second overload emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function, e.g. + +``` SQL +SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), INTERVAL 1 MINUTE, toDateTime('2023-01-01 14:35:30')); +``` **See Also** - - - [date_trunc](#date_trunc) -Also it has an overload including `origin_time_or_data` column which emulates TimescaleDB's `time_bucket()` function, respectively PostgreSQL's `date_bin()` function: -``` SQL -SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), toIntervalMinute(1), toDateTime('2023-01-01 14:35:30')); -``` - ## toTime Converts a date with time to a certain fixed date, while preserving the time. @@ -2462,19 +2462,29 @@ SELECT └──────────────────────────┴───────────────────────────────┴──────────────────────────────────────┘ ``` -## timeSlots(StartTime, Duration,\[, Size\]) +## timeSlots For a time interval starting at ‘StartTime’ and continuing for ‘Duration’ seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the ‘Size’ in seconds. ‘Size’ is an optional parameter set to 1800 (30 minutes) by default. This is necessary, for example, when searching for pageviews in the corresponding session. Accepts DateTime and DateTime64 as ’StartTime’ argument. For DateTime, ’Duration’ and ’Size’ arguments must be `UInt32`. For ’DateTime64’ they must be `Decimal64`. Returns an array of DateTime/DateTime64 (return type matches the type of ’StartTime’). For DateTime64, the return value's scale can differ from the scale of ’StartTime’ --- the highest scale among all given arguments is taken. -Example: +**Syntax** + +```sql +timeSlots(StartTime, Duration,\[, Size\]) +``` + +**Example** + ```sql SELECT timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)); SELECT timeSlots(toDateTime('1980-12-12 21:01:02', 'UTC'), toUInt32(600), 299); SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64(600.1, 1), toDecimal64(299, 0)); ``` + +Result: + ``` text ┌─timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600))─┠│ ['2012-01-01 12:00:00','2012-01-01 12:30:00'] │ From 74449872180542ebdcc26e7410100b6d897efc65 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 20 Nov 2023 18:23:18 +0000 Subject: [PATCH 0015/1722] need to deal with week+ time intervals --- src/Functions/toStartOfInterval.cpp | 311 +----------------- ...to_start_of_interva_with_origin.reference} | 0 ...6_to_start_of_interval_origin_overload.sql | 12 - ...02916_to_start_of_interval_with_origin.sql | 42 +++ 4 files changed, 58 insertions(+), 307 deletions(-) rename tests/queries/0_stateless/{02916_to_start_of_interval_origin_overload.reference => 02916_to_start_of_interva_with_origin.reference} (100%) delete mode 100644 tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql create mode 100644 tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 10792922afb..fd472d453b5 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -20,297 +20,16 @@ namespace DB { namespace ErrorCodes { - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int BAD_ARGUMENTS; - extern const int DECIMAL_OVERFLOW; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int BAD_ARGUMENTS; } namespace { -constexpr auto function_name = "toStartOfInterval"; - -template -struct Transform; - -template <> -struct Transform -{ - static UInt16 execute(UInt16 d, Int64 years, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfYearInterval(DayNum(d), years); - } - - static UInt16 execute(Int32 d, Int64 years, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years); - } - - static UInt16 execute(UInt32 t, Int64 years, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); - } - - static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); - } -}; - -template <> -struct Transform -{ - static UInt16 execute(UInt16 d, Int64 quarters, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfQuarterInterval(DayNum(d), quarters); - } - - static UInt16 execute(Int32 d, Int64 quarters, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters); - } - - static UInt16 execute(UInt32 t, Int64 quarters, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); - } - - static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); - } -}; - -template <> -struct Transform -{ - static UInt16 execute(UInt16 d, Int64 months, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMonthInterval(DayNum(d), months); - } - - static UInt16 execute(Int32 d, Int64 months, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months); - } - - static UInt16 execute(UInt32 t, Int64 months, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); - } - - static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); - } -}; - -template <> -struct Transform -{ - static UInt16 execute(UInt16 d, Int64 weeks, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfWeekInterval(DayNum(d), weeks); - } -static UInt16 execute(Int32 d, Int64 weeks, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks); - } - - static UInt16 execute(UInt32 t, Int64 weeks, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); - } - - static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16 d, Int64 days, const DateLUTImpl & time_zone, Int64) - { - return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); - } - - static UInt32 execute(Int32 d, Int64 days, const DateLUTImpl & time_zone, Int64) - { - return static_cast(time_zone.toStartOfDayInterval(ExtendedDayNum(d), days)); - } - - static UInt32 execute(UInt32 t, Int64 days, const DateLUTImpl & time_zone, Int64) - { - return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days)); - } - - static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days); - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32 t, Int64 hours, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfHourInterval(t, hours); - } - - static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfHourInterval(t / scale_multiplier, hours); - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32 t, Int64 minutes, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfMinuteInterval(t, minutes); - } - - static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes); - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32 t, Int64 seconds, const DateLUTImpl & time_zone, Int64) - { - return time_zone.toStartOfSecondInterval(t, seconds); - } - - static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier) - { - return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds); - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } - - static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier) - { - if (scale_multiplier < 1000) - { - Int64 t_milliseconds = 0; - if (common::mulOverflow(t, static_cast(1000) / scale_multiplier, t_milliseconds)) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); - if (likely(t >= 0)) - return t_milliseconds / milliseconds * milliseconds; - else - return ((t_milliseconds + 1) / milliseconds - 1) * milliseconds; - } - else if (scale_multiplier > 1000) - { - Int64 scale_diff = scale_multiplier / static_cast(1000); - if (likely(t >= 0)) - return t / milliseconds / scale_diff * milliseconds; - else - return ((t + 1) / milliseconds / scale_diff - 1) * milliseconds; - } - else - if (likely(t >= 0)) - return t / milliseconds * milliseconds; - else - return ((t + 1) / milliseconds - 1) * milliseconds; - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } - - static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier) - { - if (scale_multiplier < 1000000) - { - Int64 t_microseconds = 0; - if (common::mulOverflow(t, static_cast(1000000) / scale_multiplier, t_microseconds)) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); - if (likely(t >= 0)) - return t_microseconds / microseconds * microseconds; - else - return ((t_microseconds + 1) / microseconds - 1) * microseconds; - } - else if (scale_multiplier > 1000000) - { - Int64 scale_diff = scale_multiplier / static_cast(1000000); - if (likely(t >= 0)) - return t / microseconds / scale_diff * microseconds; - else - return ((t + 1) / microseconds / scale_diff - 1) * microseconds; - } - else - if (likely(t >= 0)) - return t / microseconds * microseconds; - else - return ((t + 1) / microseconds - 1) * microseconds; - } -}; - -template <> -struct Transform -{ - static UInt32 execute(UInt16, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(Int32, Int64, const DateLUTImpl &, Int64) { throwDateIsNotSupported(function_name); } - - static UInt32 execute(UInt32, Int64, const DateLUTImpl &, Int64) { throwDateTimeIsNotSupported(function_name); } - - static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier) - { - if (scale_multiplier < 1000000000) - { - Int64 t_nanoseconds = 0; - if (common::mulOverflow(t, (static_cast(1000000000) / scale_multiplier), t_nanoseconds)) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); - if (likely(t >= 0)) - return t_nanoseconds / nanoseconds * nanoseconds; - else - return ((t_nanoseconds + 1) / nanoseconds - 1) * nanoseconds; - } - else - if (likely(t >= 0)) - return t / nanoseconds * nanoseconds; - else - return ((t + 1) / nanoseconds - 1) * nanoseconds; - } -}; class FunctionToStartOfInterval : public IFunction { @@ -485,7 +204,7 @@ public: if (has_origin_arg) origin_column = arguments[2]; - const size_t time_zone_arg_num = (arguments.size() == 3 && isString(arguments[2].type)) ? 2 : 3; + const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3; const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); auto result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); @@ -501,9 +220,8 @@ private: if (isDateTime64(time_column_type)) { - if (origin_column.column != nullptr) - if (!isDateTime64(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + if (!isDateTime64(origin_column.type.get()) && origin_column.column != nullptr) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); const auto * time_column_vec = checkAndGetColumn(time_column_col); auto scale = assert_cast(time_column_type).getScale(); @@ -608,20 +326,23 @@ private: if (origin_column.column == nullptr) for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast(Transform::execute(time_data[i], num_units, time_zone, scale_multiplier)); + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); else { - UInt64 od = origin_column.column->get64(0); + UInt64 origin = origin_column.column->get64(0); + std::cerr << "origin: " << origin << std::endl; + std::cerr << "scale_multiplier: " << scale_multiplier << std::endl; for (size_t i = 0; i != size; ++i) { auto td = time_data[i]; - if (od > size_t(td)) + if (origin > size_t(td)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); - td -= od; - result_data[i] = static_cast(Transform::execute(td, num_units, time_zone, scale_multiplier)); - - result_data[i] += scale_multiplier == 10 ? od : od / scale_multiplier; + td -= origin; + result_data[i] = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier)); + if (!(unit == IntervalKind::Millisecond || unit == IntervalKind::Microsecond || unit == IntervalKind::Nanosecond) && scale_multiplier != 10) + origin = origin / scale_multiplier; + result_data[i] += origin; } } diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference b/tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference similarity index 100% rename from tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.reference rename to tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql b/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql deleted file mode 100644 index ce4c8f87811..00000000000 --- a/tests/queries/0_stateless/02916_to_start_of_interval_origin_overload.sql +++ /dev/null @@ -1,12 +0,0 @@ -set session_timezone = 'UTC'; -SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); -SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalMinute(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalWeek(1), toDate('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError 43 } -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError 43 } -SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError 36 } -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00'), 'Europe/Amsterdam') from numbers(1); -- { serverError 44 } diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql new file mode 100644 index 00000000000..5f5f941047a --- /dev/null +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -0,0 +1,42 @@ +set session_timezone = 'UTC'; + +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122341', 9), toIntervalNanosecond(10000), toDateTime64('2023-01-02 14:44:30.500600700', 9)); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122', 6), toIntervalMicrosecond(10000), toDateTime64('2023-01-02 14:44:30.500600', 6)); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.91', 3), toIntervalMillisecond(100), toDateTime64('2023-01-02 14:44:30.501', 3)); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalSecond(2), toDateTime('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2)); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalDay(1), toDateTime('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30')); + +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122341', 9), toIntervalNanosecond(10000), toDateTime64('2023-01-02 14:44:30.500600700', 9), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122', 6), toIntervalMicrosecond(10000), toDateTime64('2023-01-02 14:44:30.500600', 6), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.91', 3), toIntervalMillisecond(100), toDateTime64('2023-01-02 14:44:30.501', 3), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalSecond(2), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalDay(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam'); + +SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); + +SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalSecond(5), toDate32('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalMillisecond(12), toDateTime('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate32('2023-01-02 14:45:50'), toIntervalHour(5), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError BAD_ARGUMENTS } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00'), 'Europe/Amsterdam') from numbers(1); -- { serverError ILLEGAL_COLUMN } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 'Europe/Amsterdam', 'a'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5, 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), materialize(toDateTime('2023-01-02 14:44:30')), 'Europe/Amsterdam'); -- { serverError ILLEGAL_COLUMN } From 8cd4d35ead1dc7ea5fff3a794522ffb5dbc75726 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 29 Nov 2023 17:35:33 +0000 Subject: [PATCH 0016/1722] fixed review --- src/Functions/toStartOfInterval.cpp | 26 ++++++++------ ..._to_start_of_interva_with_origin.reference | 35 ++++++++++++++----- ...02916_to_start_of_interval_with_origin.sql | 18 +++++----- 3 files changed, 51 insertions(+), 28 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index fd472d453b5..c4f60d6d6db 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,6 +1,8 @@ #include +#include "Common/IntervalKind.h" #include #include +#include "base/types.h" #include #include #include @@ -220,7 +222,7 @@ private: if (isDateTime64(time_column_type)) { - if (!isDateTime64(origin_column.type.get()) && origin_column.column != nullptr) + if (origin_column.column != nullptr && !isDateTime64(origin_column.type.get())) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); const auto * time_column_vec = checkAndGetColumn(time_column_col); @@ -231,9 +233,8 @@ private: } else if (isDateTime(time_column_type)) { - if (origin_column.column != nullptr) - if (!isDateTime(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + if (origin_column.column != nullptr && !isDateTime(origin_column.type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) @@ -241,9 +242,8 @@ private: } else if (isDate(time_column_type)) { - if (origin_column.column != nullptr) - if (!isDate(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + if (origin_column.column != nullptr && !isDate(origin_column.type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) @@ -330,19 +330,23 @@ private: else { UInt64 origin = origin_column.column->get64(0); - std::cerr << "origin: " << origin << std::endl; - std::cerr << "scale_multiplier: " << scale_multiplier << std::endl; for (size_t i = 0; i != size; ++i) { auto td = time_data[i]; + result_data[i] = 0; if (origin > size_t(td)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); + td -= origin; - result_data[i] = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier)); + auto res = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier)); + if (!(unit == IntervalKind::Millisecond || unit == IntervalKind::Microsecond || unit == IntervalKind::Nanosecond) && scale_multiplier != 10) origin = origin / scale_multiplier; - result_data[i] += origin; + if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) + result_data[i] = UInt16(origin/86400 + res); + else + result_data[i] += origin + res; } } diff --git a/tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference index 7213925fb64..78e3b117cb8 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference +++ b/tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference @@ -1,15 +1,34 @@ +2023-01-02 14:45:50.917120700 +2023-01-02 14:45:50.910600 +2023-01-02 14:45:50.901 +2023-01-02 14:45:50 +2023-01-02 14:45:30 +2023-01-02 14:44:30 +2023-01-02 14:44:30 +2023-01-06 +2023-03-02 +2023-07-02 +2022-01-02 +2023-01-02 15:45:50.917120700 +2023-01-02 15:45:50.910600 +2023-01-02 15:45:50.901 +2023-01-02 15:45:50 +2023-01-02 15:45:30 +2023-01-02 15:44:30 +2023-01-03 14:44:30 +2023-01-06 +2023-03-02 +2023-07-02 +2022-01-02 2023-03-01 16:55:00 -2023-02-01 16:55:00 -2023-03-01 16:55:00 -2023-02-01 16:55:00 -2023-03-01 16:55:00 +2023-02-01 16:54:33 +2023-03-01 16:54:38 +2023-02-01 16:54:52 +2023-03-01 16:54:16 2023-03-01 16:55:00 2023-02-01 16:55:00 2023-03-01 16:55:00 2023-02-01 16:55:00 2023-03-01 16:55:00 2023-01-02 15:44:30 -2023-01-02 15:44:30 -2023-01-02 14:45:30 -2023-01-02 14:45:30 -2023-01-02 +2023-02-01 16:44:30 diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql index 5f5f941047a..390b36a4cf1 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -7,10 +7,10 @@ SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalSecond(2), SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2)); SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalDay(1), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-08 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-03-03 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-08-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30')); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-03 14:44:30')); SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122341', 9), toIntervalNanosecond(10000), toDateTime64('2023-01-02 14:44:30.500600700', 9), 'Europe/Amsterdam'); SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122', 6), toIntervalMicrosecond(10000), toDateTime64('2023-01-02 14:44:30.500600', 6), 'Europe/Amsterdam'); @@ -19,15 +19,15 @@ SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalSecond(2), SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalDay(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-08 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-03-03 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-08-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-03 14:44:30'), 'Europe/Amsterdam'); SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalSecond(5), toDate32('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalMillisecond(12), toDateTime('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From b298bed1cfdb67e21da89e2b8a4bb4affc44f14b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 29 Nov 2023 23:08:19 +0100 Subject: [PATCH 0017/1722] Rename 02916_to_start_of_interva_with_origin.reference to 02916_to_start_of_interval_with_origin.reference --- ...reference => 02916_to_start_of_interval_with_origin.reference} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02916_to_start_of_interva_with_origin.reference => 02916_to_start_of_interval_with_origin.reference} (100%) diff --git a/tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference similarity index 100% rename from tests/queries/0_stateless/02916_to_start_of_interva_with_origin.reference rename to tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference From 450f609227fe0cedefbf2473a1a2c4b4e7b65d5f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 2 Dec 2023 22:58:52 +0000 Subject: [PATCH 0018/1722] Various updates --- src/Functions/toStartOfInterval.cpp | 29 +++-- ...to_start_of_interval_with_origin.reference | 74 +++++++------ ...02916_to_start_of_interval_with_origin.sql | 103 ++++++++++++------ 3 files changed, 128 insertions(+), 78 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index c4f60d6d6db..ec0deee8abd 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,21 +1,21 @@ -#include #include "Common/IntervalKind.h" -#include -#include -#include "base/types.h" -#include -#include #include #include +#include +#include #include #include #include #include #include +#include #include #include +#include #include #include +#include +#include namespace DB @@ -325,26 +325,31 @@ private: Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); if (origin_column.column == nullptr) + { for (size_t i = 0; i != size; ++i) result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + } else { UInt64 origin = origin_column.column->get64(0); for (size_t i = 0; i != size; ++i) { - auto td = time_data[i]; - result_data[i] = 0; - if (origin > size_t(td)) + auto t = time_data[i]; + if (origin > static_cast(t)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); - td -= origin; - auto res = static_cast(ToStartOfInterval::execute(td, num_units, time_zone, scale_multiplier)); + t -= origin; + auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_multiplier)); if (!(unit == IntervalKind::Millisecond || unit == IntervalKind::Microsecond || unit == IntervalKind::Nanosecond) && scale_multiplier != 10) origin = origin / scale_multiplier; + + static constexpr size_t SECONDS_PER_DAY = 86400; + + result_data[i] = 0; if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) - result_data[i] = UInt16(origin/86400 + res); + result_data[i] = static_cast(origin/SECONDS_PER_DAY + res); else result_data[i] += origin + res; } diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference index 78e3b117cb8..3574da8f685 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference @@ -1,34 +1,40 @@ -2023-01-02 14:45:50.917120700 -2023-01-02 14:45:50.910600 -2023-01-02 14:45:50.901 -2023-01-02 14:45:50 -2023-01-02 14:45:30 -2023-01-02 14:44:30 -2023-01-02 14:44:30 -2023-01-06 -2023-03-02 -2023-07-02 -2022-01-02 -2023-01-02 15:45:50.917120700 -2023-01-02 15:45:50.910600 -2023-01-02 15:45:50.901 -2023-01-02 15:45:50 -2023-01-02 15:45:30 -2023-01-02 15:44:30 -2023-01-03 14:44:30 -2023-01-06 -2023-03-02 -2023-07-02 -2022-01-02 -2023-03-01 16:55:00 -2023-02-01 16:54:33 -2023-03-01 16:54:38 -2023-02-01 16:54:52 -2023-03-01 16:54:16 -2023-03-01 16:55:00 -2023-02-01 16:55:00 -2023-03-01 16:55:00 -2023-02-01 16:55:00 -2023-03-01 16:55:00 -2023-01-02 15:44:30 -2023-02-01 16:44:30 +-- Negative tests +Time and origin as Time +1971-01-01 +1971-07-01 +1970-02-01 +1970-01-05 +1970-01-02 05:27:18 +Time and origin as DateTime +2023-02-01 +2023-08-01 +2023-10-09 +2023-10-05 +2023-10-09 09:08:07 +2023-10-09 10:10:07 +2023-10-09 10:11:11 +2023-10-09 10:11:12 +Time and origin as DateTime64(9) +2023-02-01 +2023-08-01 +2023-10-09 +2023-10-05 +2023-10-09 09:08:07 +2023-10-09 10:10:07 +2023-10-09 10:11:11 +2023-10-09 10:11:12 +2299-12-31 23:57:37.653 +2299-12-31 23:42:04.320986 +2023-10-09 10:11:12.987654321 +Time and origin as DateTime64(3) +2023-02-01 +2023-08-01 +2023-10-09 +2023-10-05 +2023-10-09 09:08:07 +2023-10-09 10:10:07 +2023-10-09 10:11:11 +2023-10-09 10:11:12 +2023-10-09 10:11:12.987 +1970-01-20 15:20:47.136123 +1970-01-01 00:28:17.710272123 diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql index 390b36a4cf1..864ef56e7ea 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -1,42 +1,81 @@ set session_timezone = 'UTC'; -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122341', 9), toIntervalNanosecond(10000), toDateTime64('2023-01-02 14:44:30.500600700', 9)); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122', 6), toIntervalMicrosecond(10000), toDateTime64('2023-01-02 14:44:30.500600', 6)); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.91', 3), toIntervalMillisecond(100), toDateTime64('2023-01-02 14:44:30.501', 3)); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalSecond(2), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2)); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalDay(1), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-08 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-03-03 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-08-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30')); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-03 14:44:30')); - -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122341', 9), toIntervalNanosecond(10000), toDateTime64('2023-01-02 14:44:30.500600700', 9), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.917122', 6), toIntervalMicrosecond(10000), toDateTime64('2023-01-02 14:44:30.500600', 6), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50.91', 3), toIntervalMillisecond(100), toDateTime64('2023-01-02 14:44:30.501', 3), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalSecond(2), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalDay(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-08 14:45:50'), toIntervalWeek(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-03-03 14:45:50'), toIntervalMonth(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-08-02 14:45:50'), toIntervalQuarter(1), toDateTime('2022-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-03 14:44:30'), 'Europe/Amsterdam'); - -SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); -SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); -SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); -SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); +SELECT '-- Negative tests'; +-- time and origin arguments must have the same type SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalSecond(5), toDate32('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDate('2023-01-02 14:45:50'), toIntervalMillisecond(12), toDateTime('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDate32('2023-01-02 14:45:50'), toIntervalHour(5), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), toDateTime64('2023-01-02 14:44:30', 2)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDateTime64('2023-01-02 14:45:50', 2), toIntervalMinute(1), toDate('2023-01-02 14:44:30')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +-- the origin must be before the time SELECT toStartOfInterval(toDateTime('2023-01-02 14:42:50'), toIntervalMinute(1), toDateTime('2023-01-02 14:44:30')); -- { serverError BAD_ARGUMENTS } -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00'), 'Europe/Amsterdam') from numbers(1); -- { serverError ILLEGAL_COLUMN } -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 'Europe/Amsterdam', 'a'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5, 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +-- the origin must be constant +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalMinute(1), number % 2 == 0 ? toDateTime('2023-02-01 15:55:00') : toDateTime('2023-01-01 15:55:00')) from numbers(1); -- { serverError ILLEGAL_COLUMN } SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalHour(1), materialize(toDateTime('2023-01-02 14:44:30')), 'Europe/Amsterdam'); -- { serverError ILLEGAL_COLUMN } + +-- with 4 arguments, the 3rd one must not be a string or an integer +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 'Europe/Amsterdam', 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5, 'Europe/Amsterdam'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +-- too many arguments +SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT 'Time and origin as Time'; +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalYear(1), toDate('2022-02-01')); -- broken +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalQuarter(1), toDate('2022-02-01')); -- broken +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMonth(1), toDate('2023-09-08')); -- broken +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalWeek(1), toDate('2023-10-01')); -- broken +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalDay(1), toDate('2023-10-08')); -- broken +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalHour(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMinute(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalSecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMillisecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMicrosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalNanosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Time and origin as DateTime'; +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalYear(1), toDateTime('2022-02-01 09:08:07')); -- broken, should that not return 2023-02-01 09:08:07? +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalQuarter(1), toDateTime('2022-02-01 09:08:07')); -- broken, should that not return 2023-08-01 09:08:07? +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1), toDateTime('2023-09-08 09:08:07')); -- broken, should that not return 2023-10-09 09:08:07? +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalWeek(1), toDateTime('2023-10-01 09:08:07')); -- broken, should that not return 2023-10-05 09:08:07? +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalDay(1), toDateTime('2023-10-08 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalHour(1), toDateTime('2023-10-09 09:10:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 10:10:11')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalSecond(1), toDateTime('2023-10-09 10:11:10')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMillisecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMicrosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalNanosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Time and origin as DateTime64(9)'; +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123456789', 9)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123456789', 9)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); -- broken (2299) +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); -- broken (2299) +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); -- broken (2299) + +SELECT 'Time and origin as DateTime64(3)'; +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123', 3)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123', 3)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123', 3)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123', 3)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); -- broken (1970) +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); -- broken (1970) + +-- SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); +-- SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); +-- SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +-- SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); From 01036a21f59925b78f45415cacb1953481292e31 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 17:44:40 +0100 Subject: [PATCH 0019/1722] Fixed bugs regarding precision --- src/Functions/toStartOfInterval.cpp | 124 +++++++++++++++++++++------- 1 file changed, 95 insertions(+), 29 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index ec0deee8abd..da4eba9a594 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,8 +1,10 @@ -#include "Common/IntervalKind.h" +#include +#include #include #include #include #include +#include "DataTypes/IDataType.h" #include #include #include @@ -14,7 +16,6 @@ #include #include #include -#include #include @@ -116,9 +117,22 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", getName(), interval_type->getKind().toString()); + + if (arguments[0].type.get() != arguments[2].type.get()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); } else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3)) + { third_argument = ThirdArgument::IsOrigin; + if (isDateTime64(arguments[0].type) && isDateTime64(arguments[2].type)) + result_type = ResultType::DateTime64; + else if (isDateTime(arguments[0].type) && isDateTime(arguments[2].type)) + result_type = ResultType::DateTime; + else if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && (isDate(arguments[2].type) || isDate32(arguments[2].type))) + result_type = ResultType::Date; + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); + } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " "This argument is optional and must be a constant String with timezone name or a Date/Date32/DateTime/DateTime64 with a constant origin", @@ -180,13 +194,14 @@ public: } case ResultType::DateTime64: { - UInt32 scale = 0; + auto scale_date_time = assert_cast(*arguments[0].type.get()).getScale(); + UInt32 scale = scale_date_time; if (interval_type->getKind() == IntervalKind::Nanosecond) - scale = 9; + scale = 9 > scale_date_time ? 9 : scale_date_time; else if (interval_type->getKind() == IntervalKind::Microsecond) - scale = 6; + scale = 6 > scale_date_time ? 6 : scale_date_time; else if (interval_type->getKind() == IntervalKind::Millisecond) - scale = 3; + scale = 3 > scale_date_time ? 3 : scale_date_time; const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); @@ -209,11 +224,19 @@ public: const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3; const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); - auto result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + ColumnPtr result_column = nullptr; + if (isDateTime64(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + else if (isDateTime(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + else + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + return result_column; } private: + template ColumnPtr dispatchForTimeColumn( const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const { @@ -229,7 +252,7 @@ private: auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale); } else if (isDateTime(time_column_type)) { @@ -238,7 +261,7 @@ private: const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate(time_column_type)) { @@ -247,7 +270,7 @@ private: const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDate32(time_column_type)) { @@ -257,12 +280,12 @@ private: const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. Must contain dates or dates with time", getName()); } - template + template ColumnPtr dispatchForIntervalColumn( const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const @@ -282,32 +305,52 @@ private: switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Nanosecond: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Microsecond: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Millisecond: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Second: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Minute: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Hour: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Day: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Week: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Month: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Quarter: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); case IntervalKind::Year: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); } std::unreachable(); } + template + Int64 decideScaleOnPrecision(const UInt16 scale) const + { + static constexpr Int64 MILLISECOND_SCALE = 1000; + static constexpr Int64 MICROSECOND_SCALE = 1000000; + static constexpr Int64 NANOSECOND_SCALE = 1000000000; + Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); + switch (unit) + { + case IntervalKind::Millisecond: + return MILLISECOND_SCALE; + case IntervalKind::Microsecond: + return MICROSECOND_SCALE; + case IntervalKind::Nanosecond: + return NANOSECOND_SCALE; + default: + return scale_multiplier; + } + } + template ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const { @@ -323,6 +366,8 @@ private: result_data.resize(size); Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); + Int64 scale_on_precision = decideScaleOnPrecision(scale); + Int64 scale_diff = scale_on_precision > scale_multiplier ? scale_on_precision / scale_multiplier : scale_multiplier / scale_on_precision; if (origin_column.column == nullptr) { @@ -342,19 +387,40 @@ private: t -= origin; auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_multiplier)); - if (!(unit == IntervalKind::Millisecond || unit == IntervalKind::Microsecond || unit == IntervalKind::Nanosecond) && scale_multiplier != 10) - origin = origin / scale_multiplier; - static constexpr size_t SECONDS_PER_DAY = 86400; result_data[i] = 0; if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) - result_data[i] = static_cast(origin/SECONDS_PER_DAY + res); + { + if (isDate(result_type) || isDate32(result_type)) + { + result_data[i] += origin + res; + } + else if (isDateTime64(result_type)) + { + result_data[i] += origin + (res * SECONDS_PER_DAY * scale_multiplier); + } + else + { + result_data[i] += origin + res * SECONDS_PER_DAY; + } + } else - result_data[i] += origin + res; + { + if (isDate(result_type) || isDate32(result_type)) + res = res / SECONDS_PER_DAY; + + if (scale_on_precision > scale_multiplier) + { + result_data[i] += (origin + res / scale_diff) * scale_diff; + } + else + { + result_data[i] += origin + res * scale_diff; + } + } } } - return result_col; } }; From 65f34394cf84fa12f6852e9988c8a3d62c155701 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 17:45:29 +0100 Subject: [PATCH 0020/1722] Update 02916_to_start_of_interval_with_origin.reference --- ...to_start_of_interval_with_origin.reference | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference index 3574da8f685..806330743d7 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference @@ -1,40 +1,40 @@ -- Negative tests Time and origin as Time -1971-01-01 -1971-07-01 -1970-02-01 -1970-01-05 -1970-01-02 05:27:18 -Time and origin as DateTime 2023-02-01 2023-08-01 2023-10-09 2023-10-05 +2023-10-08 +Time and origin as DateTime +2023-02-01 09:08:07 +2023-08-01 09:08:07 +2023-10-09 09:08:07 +2023-10-05 09:08:07 2023-10-09 09:08:07 2023-10-09 10:10:07 2023-10-09 10:11:11 2023-10-09 10:11:12 Time and origin as DateTime64(9) -2023-02-01 -2023-08-01 -2023-10-09 -2023-10-05 -2023-10-09 09:08:07 -2023-10-09 10:10:07 -2023-10-09 10:11:11 -2023-10-09 10:11:12 -2299-12-31 23:57:37.653 -2299-12-31 23:42:04.320986 +2023-02-01 09:08:07.123456789 +2023-08-01 09:08:07.123456789 +2023-09-10 09:08:07.123456789 +2023-10-05 09:08:07.123456789 +2023-10-08 09:08:07.123543189 +2023-10-09 09:10:07.123460389 +2023-10-09 10:10:11.123456849 +2023-10-09 10:11:10.123456791 +2023-10-09 10:11:12.987456789 +2023-10-09 10:11:12.987653789 2023-10-09 10:11:12.987654321 Time and origin as DateTime64(3) -2023-02-01 -2023-08-01 -2023-10-09 -2023-10-05 -2023-10-09 09:08:07 -2023-10-09 10:10:07 -2023-10-09 10:11:11 -2023-10-09 10:11:12 +2023-02-01 09:08:07.123 +2023-08-01 09:08:07.123 +2023-10-09 09:08:07.123 +2023-10-05 09:08:07.123 +2023-10-08 09:09:33.523 +2023-10-09 09:10:10.723 +2023-10-09 10:10:11.183 +2023-10-09 10:11:10.125 2023-10-09 10:11:12.987 -1970-01-20 15:20:47.136123 -1970-01-01 00:28:17.710272123 +2023-10-09 10:11:12.987000 +2023-10-09 10:11:12.987000000 From bd105d51ce469d9ed0c741a7a45012d506e34ddd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 17:46:00 +0100 Subject: [PATCH 0021/1722] fixed tests --- ...02916_to_start_of_interval_with_origin.sql | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql index 864ef56e7ea..853103ecc77 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -24,11 +24,11 @@ SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5 SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT 'Time and origin as Time'; -SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalYear(1), toDate('2022-02-01')); -- broken -SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalQuarter(1), toDate('2022-02-01')); -- broken -SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMonth(1), toDate('2023-09-08')); -- broken -SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalWeek(1), toDate('2023-10-01')); -- broken -SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalDay(1), toDate('2023-10-08')); -- broken +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalYear(1), toDate('2022-02-01')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalQuarter(1), toDate('2022-02-01')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMonth(1), toDate('2023-09-08')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalWeek(1), toDate('2023-10-01')); +SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalDay(1), toDate('2023-10-08')); SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalHour(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMinute(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalSecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } @@ -37,10 +37,10 @@ SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMicrosecond(1), toDate( SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalNanosecond(1), toDate('2023-10-09')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT 'Time and origin as DateTime'; -SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalYear(1), toDateTime('2022-02-01 09:08:07')); -- broken, should that not return 2023-02-01 09:08:07? -SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalQuarter(1), toDateTime('2022-02-01 09:08:07')); -- broken, should that not return 2023-08-01 09:08:07? -SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1), toDateTime('2023-09-08 09:08:07')); -- broken, should that not return 2023-10-09 09:08:07? -SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalWeek(1), toDateTime('2023-10-01 09:08:07')); -- broken, should that not return 2023-10-05 09:08:07? +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalYear(1), toDateTime('2022-02-01 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalQuarter(1), toDateTime('2022-02-01 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1), toDateTime('2023-09-08 09:08:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalWeek(1), toDateTime('2023-10-01 09:08:07')); SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalDay(1), toDateTime('2023-10-08 09:08:07')); SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalHour(1), toDateTime('2023-10-09 09:10:07')); SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 10:10:11')); @@ -50,30 +50,30 @@ SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMicrosecon SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalNanosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT 'Time and origin as DateTime64(9)'; -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); -- broken, see above -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); -- broken, see above -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123456789', 9)); -- broken, see above -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123456789', 9)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMonth(1), toDateTime64('2023-09-10 09:08:07.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123456789', 9)); -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); -- broken (2299) -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); -- broken (2299) -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); -- broken (2299) +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); SELECT 'Time and origin as DateTime64(3)'; -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123', 3)); -- broken, see above -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123', 3)); -- broken, see above -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123', 3)); -- broken, see above -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123', 3)); -- broken, see above +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalYear(1), toDateTime64('2022-02-01 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalQuarter(1), toDateTime64('2022-02-01 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMonth(1), toDateTime64('2023-09-08 09:08:07.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123', 3)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123', 3)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123', 3)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123', 3)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123', 3)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); -- broken (1970) -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); -- broken (1970) +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); -- SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); -- SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); From 885e44c50e1968d660866e53f25110cb662e27a2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 18:17:41 +0100 Subject: [PATCH 0022/1722] style fix --- src/Functions/toStartOfInterval.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index da4eba9a594..4fae8cb6bb6 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -231,7 +231,6 @@ public: result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); else result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); - return result_column; } From e5fdad21609704baa3b90eb07161461120364fb2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 19:09:55 +0100 Subject: [PATCH 0023/1722] fix non-const arguments --- src/Functions/toStartOfInterval.cpp | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 4fae8cb6bb6..94514d1e1a0 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -392,17 +392,11 @@ private: if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) { if (isDate(result_type) || isDate32(result_type)) - { result_data[i] += origin + res; - } else if (isDateTime64(result_type)) - { result_data[i] += origin + (res * SECONDS_PER_DAY * scale_multiplier); - } else - { result_data[i] += origin + res * SECONDS_PER_DAY; - } } else { @@ -410,13 +404,11 @@ private: res = res / SECONDS_PER_DAY; if (scale_on_precision > scale_multiplier) - { result_data[i] += (origin + res / scale_diff) * scale_diff; - } + else if (scale_on_precision == scale_multiplier && scale_on_precision != 10) /// scale == 10 is default case + result_data[i] += origin + (res * scale_on_precision); else - { result_data[i] += origin + res * scale_diff; - } } } } From ac8d80da515144d2d4fe3ed0c47e2bab1d171927 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 19:28:06 +0100 Subject: [PATCH 0024/1722] fix --- src/Functions/toStartOfInterval.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 94514d1e1a0..e67d3378f6e 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -405,7 +405,8 @@ private: if (scale_on_precision > scale_multiplier) result_data[i] += (origin + res / scale_diff) * scale_diff; - else if (scale_on_precision == scale_multiplier && scale_on_precision != 10) /// scale == 10 is default case + else if (scale_on_precision == scale_multiplier && scale_on_precision % 1000 != 0 && scale_multiplier != 10) /// when it's not a default case with DateTime + /// and when precision is not sub-scale result_data[i] += origin + (res * scale_on_precision); else result_data[i] += origin + res * scale_diff; From 7be47eca5e3ee31ef958f5b9e9804ceb15f35f48 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 19:28:27 +0100 Subject: [PATCH 0025/1722] Update 02916_to_start_of_interval_with_origin.reference --- ...02916_to_start_of_interval_with_origin.reference | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference index 806330743d7..870853bc371 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference @@ -38,3 +38,16 @@ Time and origin as DateTime64(3) 2023-10-09 10:11:12.987 2023-10-09 10:11:12.987000 2023-10-09 10:11:12.987000000 +Non-const arguments +2023-03-01 16:55:00.00 +2023-02-01 16:55:00.00 +2023-03-01 16:55:00.00 +2023-02-01 16:55:00.00 +2023-03-01 16:55:00.00 +2023-03-01 16:55:00 +2023-02-01 16:55:00 +2023-03-01 16:55:00 +2023-02-01 16:55:00 +2023-03-01 16:55:00 +2023-01-02 15:44:30 +2023-02-01 16:44:30.00 From 969c7f36a5db55ef53cbce2def91820b619c23ea Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Dec 2023 19:28:45 +0100 Subject: [PATCH 0026/1722] fix tests --- .../02916_to_start_of_interval_with_origin.sql | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql index 853103ecc77..71f5fb7fb36 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -75,7 +75,8 @@ SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalM SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987', 3), toIntervalNanosecond(1), toDateTime64('2023-10-09 10:11:12.123', 3)); --- SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); --- SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); --- SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); --- SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); +SELECT 'Non-const arguments'; +SELECT toStartOfInterval(number % 2 == 0 ? toDateTime64('2023-03-01 15:55:00', 2) : toDateTime64('2023-02-01 15:55:00', 2), toIntervalMinute(1), toDateTime64('2023-01-01 13:55:00', 2), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(number % 2 == 0 ? toDateTime('2023-03-01 15:55:00') : toDateTime('2023-02-01 15:55:00'), toIntervalHour(1), toDateTime('2023-01-01 13:55:00'), 'Europe/Amsterdam') from numbers(5); +SELECT toStartOfInterval(materialize(toDateTime('2023-01-02 14:45:50')), toIntervalHour(1), toDateTime('2023-01-02 14:44:30'), 'Europe/Amsterdam'); +SELECT toStartOfInterval(materialize(toDateTime64('2023-02-01 15:45:50', 2)), toIntervalHour(1), toDateTime64('2023-01-02 14:44:30', 2), 'Europe/Amsterdam'); From 3027f3a04f99beafdc8ead3641aa84fdf4f6657a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Dec 2023 15:29:11 +0000 Subject: [PATCH 0027/1722] fix tests --- src/Functions/toStartOfInterval.cpp | 23 ++++++++++--------- .../02207_subseconds_intervals.reference | 16 ++++++------- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index e67d3378f6e..7c75d7aa8e9 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -117,9 +117,6 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", getName(), interval_type->getKind().toString()); - - if (arguments[0].type.get() != arguments[2].type.get()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); } else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3)) { @@ -137,7 +134,6 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " "This argument is optional and must be a constant String with timezone name or a Date/Date32/DateTime/DateTime64 with a constant origin", type_arg3->getName(), getName()); - }; auto check_fourth_argument = [&] @@ -365,13 +361,19 @@ private: result_data.resize(size); Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); - Int64 scale_on_precision = decideScaleOnPrecision(scale); - Int64 scale_diff = scale_on_precision > scale_multiplier ? scale_on_precision / scale_multiplier : scale_multiplier / scale_on_precision; + Int64 scale_on_interval = decideScaleOnPrecision(scale); + Int64 scale_diff = scale_on_interval > scale_multiplier ? scale_on_interval / scale_multiplier : scale_multiplier / scale_on_interval; if (origin_column.column == nullptr) { for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + { + result_data[i] = 0; + if (scale_on_interval < scale_multiplier) + result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)) * scale_diff; + else + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + } } else { @@ -403,11 +405,10 @@ private: if (isDate(result_type) || isDate32(result_type)) res = res / SECONDS_PER_DAY; - if (scale_on_precision > scale_multiplier) + if (scale_on_interval > scale_multiplier) result_data[i] += (origin + res / scale_diff) * scale_diff; - else if (scale_on_precision == scale_multiplier && scale_on_precision % 1000 != 0 && scale_multiplier != 10) /// when it's not a default case with DateTime - /// and when precision is not sub-scale - result_data[i] += origin + (res * scale_on_precision); + else if (scale_on_interval == scale_multiplier && scale_on_interval % 1000 != 0 && scale_multiplier != 10) + result_data[i] += origin + (res * scale_on_interval); else result_data[i] += origin + res * scale_diff; } diff --git a/tests/queries/0_stateless/02207_subseconds_intervals.reference b/tests/queries/0_stateless/02207_subseconds_intervals.reference index 91f0ecb8606..b0edbda5e76 100644 --- a/tests/queries/0_stateless/02207_subseconds_intervals.reference +++ b/tests/queries/0_stateless/02207_subseconds_intervals.reference @@ -10,25 +10,25 @@ test intervals - test microseconds 1980-12-12 12:12:12.123456 1980-12-12 12:12:12.123400 -1980-12-12 12:12:12.123456 -1980-12-12 12:12:12.123456 +1980-12-12 12:12:12.12345600 +1980-12-12 12:12:12.12345600 1930-12-12 12:12:12.123456 1930-12-12 12:12:12.123400 -1930-12-12 12:12:12.123456 +1930-12-12 12:12:12.12345600 2220-12-12 12:12:12.123456 2220-12-12 12:12:12.123400 -2220-12-12 12:12:12.123456 +2220-12-12 12:12:12.12345600 - test milliseconds 1980-12-12 12:12:12.123 1980-12-12 12:12:12.120 -1980-12-12 12:12:12.123 -1980-12-12 12:12:12.123 +1980-12-12 12:12:12.123000 +1980-12-12 12:12:12.123000 1930-12-12 12:12:12.123 1930-12-12 12:12:12.120 -1930-12-12 12:12:12.123 +1930-12-12 12:12:12.123000 2220-12-12 12:12:12.123 2220-12-12 12:12:12.120 -2220-12-12 12:12:12.123 +2220-12-12 12:12:12.123000 test add[...]seconds() - test nanoseconds 1980-12-12 12:12:12.123456790 From d08fd931bec203f1b86d89227b766973ad3612c9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Dec 2023 13:42:00 +0000 Subject: [PATCH 0028/1722] fix error --- src/Functions/toStartOfInterval.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 7c75d7aa8e9..e0301f45ded 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -190,8 +190,10 @@ public: } case ResultType::DateTime64: { - auto scale_date_time = assert_cast(*arguments[0].type.get()).getScale(); - UInt32 scale = scale_date_time; + UInt32 scale = 0; + auto scale_date_time = 0; + if (third_argument == ThirdArgument::IsOrigin) + scale_date_time = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Nanosecond) scale = 9 > scale_date_time ? 9 : scale_date_time; else if (interval_type->getKind() == IntervalKind::Microsecond) From e13eec9c5a9cb49004aef8d635034e9ccfd697d4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Dec 2023 15:07:39 +0000 Subject: [PATCH 0029/1722] fix --- src/Functions/toStartOfInterval.cpp | 11 +++++------ .../02207_subseconds_intervals.reference | 16 ++++++++-------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index e0301f45ded..e90f965a6f9 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -191,15 +191,14 @@ public: case ResultType::DateTime64: { UInt32 scale = 0; - auto scale_date_time = 0; - if (third_argument == ThirdArgument::IsOrigin) - scale_date_time = assert_cast(*arguments[0].type.get()).getScale(); + if (isDate32(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)) + scale = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Nanosecond) - scale = 9 > scale_date_time ? 9 : scale_date_time; + scale = 9 > scale ? 9 : scale; else if (interval_type->getKind() == IntervalKind::Microsecond) - scale = 6 > scale_date_time ? 6 : scale_date_time; + scale = 6 > scale ? 6 : scale; else if (interval_type->getKind() == IntervalKind::Millisecond) - scale = 3 > scale_date_time ? 3 : scale_date_time; + scale = 3 > scale ? 3 : scale; const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); diff --git a/tests/queries/0_stateless/02207_subseconds_intervals.reference b/tests/queries/0_stateless/02207_subseconds_intervals.reference index b0edbda5e76..91f0ecb8606 100644 --- a/tests/queries/0_stateless/02207_subseconds_intervals.reference +++ b/tests/queries/0_stateless/02207_subseconds_intervals.reference @@ -10,25 +10,25 @@ test intervals - test microseconds 1980-12-12 12:12:12.123456 1980-12-12 12:12:12.123400 -1980-12-12 12:12:12.12345600 -1980-12-12 12:12:12.12345600 +1980-12-12 12:12:12.123456 +1980-12-12 12:12:12.123456 1930-12-12 12:12:12.123456 1930-12-12 12:12:12.123400 -1930-12-12 12:12:12.12345600 +1930-12-12 12:12:12.123456 2220-12-12 12:12:12.123456 2220-12-12 12:12:12.123400 -2220-12-12 12:12:12.12345600 +2220-12-12 12:12:12.123456 - test milliseconds 1980-12-12 12:12:12.123 1980-12-12 12:12:12.120 -1980-12-12 12:12:12.123000 -1980-12-12 12:12:12.123000 +1980-12-12 12:12:12.123 +1980-12-12 12:12:12.123 1930-12-12 12:12:12.123 1930-12-12 12:12:12.120 -1930-12-12 12:12:12.123000 +1930-12-12 12:12:12.123 2220-12-12 12:12:12.123 2220-12-12 12:12:12.120 -2220-12-12 12:12:12.123000 +2220-12-12 12:12:12.123 test add[...]seconds() - test nanoseconds 1980-12-12 12:12:12.123456790 From 4c83b7e46ffe50409aab35b889839fb6cb92a18e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Dec 2023 15:18:43 +0000 Subject: [PATCH 0030/1722] style fix --- src/Functions/toStartOfInterval.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index e90f965a6f9..b55d92f809b 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -191,7 +191,7 @@ public: case ResultType::DateTime64: { UInt32 scale = 0; - if (isDate32(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)) + if (isDate32(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)) scale = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Nanosecond) scale = 9 > scale ? 9 : scale; From 5793725bc7ae8cfd66bea9c9f82ef0e30c124314 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Dec 2023 16:45:45 +0000 Subject: [PATCH 0031/1722] tests --- .../02207_subseconds_intervals.reference | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02207_subseconds_intervals.reference b/tests/queries/0_stateless/02207_subseconds_intervals.reference index 91f0ecb8606..bedd5d4878b 100644 --- a/tests/queries/0_stateless/02207_subseconds_intervals.reference +++ b/tests/queries/0_stateless/02207_subseconds_intervals.reference @@ -10,25 +10,25 @@ test intervals - test microseconds 1980-12-12 12:12:12.123456 1980-12-12 12:12:12.123400 -1980-12-12 12:12:12.123456 -1980-12-12 12:12:12.123456 +1980-12-12 12:12:12.12345600 +1980-12-12 12:12:12.12345600 1930-12-12 12:12:12.123456 1930-12-12 12:12:12.123400 -1930-12-12 12:12:12.123456 +1930-12-12 12:12:12.12345600 2220-12-12 12:12:12.123456 2220-12-12 12:12:12.123400 -2220-12-12 12:12:12.123456 +2220-12-12 12:12:12.12345600 - test milliseconds 1980-12-12 12:12:12.123 1980-12-12 12:12:12.120 -1980-12-12 12:12:12.123 -1980-12-12 12:12:12.123 +1980-12-12 12:12:12.123000 +1980-12-12 12:12:12.123000 1930-12-12 12:12:12.123 1930-12-12 12:12:12.120 -1930-12-12 12:12:12.123 +1930-12-12 12:12:12.123000 2220-12-12 12:12:12.123 2220-12-12 12:12:12.120 -2220-12-12 12:12:12.123 +2220-12-12 12:12:12.123000 test add[...]seconds() - test nanoseconds 1980-12-12 12:12:12.123456790 @@ -75,4 +75,4 @@ test subtract[...]seconds() 2022-12-31 23:59:59.999 2022-12-31 23:59:59.900 2023-01-01 00:00:00.001 -2023-01-01 00:00:00.100 +2023-01-01 00:00:00.100 \ No newline at end of file From 53ef9c0cb80c734caa449430ff4600f32b900134 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Dec 2023 18:38:25 +0100 Subject: [PATCH 0032/1722] Update 02207_subseconds_intervals.reference --- tests/queries/0_stateless/02207_subseconds_intervals.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02207_subseconds_intervals.reference b/tests/queries/0_stateless/02207_subseconds_intervals.reference index bedd5d4878b..b0edbda5e76 100644 --- a/tests/queries/0_stateless/02207_subseconds_intervals.reference +++ b/tests/queries/0_stateless/02207_subseconds_intervals.reference @@ -75,4 +75,4 @@ test subtract[...]seconds() 2022-12-31 23:59:59.999 2022-12-31 23:59:59.900 2023-01-01 00:00:00.001 -2023-01-01 00:00:00.100 \ No newline at end of file +2023-01-01 00:00:00.100 From 87bda03da17cdf0e3878a1df73175c92bae834c6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Dec 2023 21:36:24 +0100 Subject: [PATCH 0033/1722] logical error --- src/Functions/toStartOfInterval.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index b55d92f809b..b6a3a9389d6 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -191,7 +191,7 @@ public: case ResultType::DateTime64: { UInt32 scale = 0; - if (isDate32(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)) + if (isDate32(arguments[0].type) || isDateTime64(arguments[0].type)) scale = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Nanosecond) scale = 9 > scale ? 9 : scale; From d1c49cc9bcb869030361821e60349e8054a51c4b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 12 Dec 2023 17:55:53 +0100 Subject: [PATCH 0034/1722] Added comments, simplified and fixed review --- src/Functions/toStartOfInterval.cpp | 138 ++++++++++++++++------------ 1 file changed, 81 insertions(+), 57 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index b6a3a9389d6..6c71b357590 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,9 +1,7 @@ -#include -#include #include #include #include -#include +#include #include "DataTypes/IDataType.h" #include #include @@ -16,7 +14,7 @@ #include #include #include -#include +#include namespace DB @@ -31,21 +29,23 @@ namespace ErrorCodes } -namespace -{ - class FunctionToStartOfInterval : public IFunction { public: + enum class Overload + { + Default, /// toStartOfInterval(time, interval) or toStartOfInterval(time, interval, timezone) + Origin /// toStartOfInterval(time, interval, origin) or toStartOfInterval(time, interval, origin, timezone) + }; + mutable Overload overload; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } static constexpr auto name = "toStartOfInterval"; String getName() const override { return name; } - bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } @@ -59,8 +59,9 @@ public: { const DataTypePtr & type_arg1 = arguments[0].type; if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 1st argument of function {}. " - "Should be a date or a date with time", type_arg1->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1st argument of function {}, expected a Date, DateTime or DateTime64", + type_arg1->getName(), getName()); value_is_date = isDate(type_arg1); }; @@ -75,10 +76,14 @@ public: auto check_second_argument = [&] { const DataTypePtr & type_arg2 = arguments[1].type; + interval_type = checkAndGetDataType(type_arg2.get()); if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " - "Should be an interval of time", type_arg2->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 2nd argument of function {}, expected a time interval", + type_arg2->getName(), getName()); + + /// Result here is determined for default overload (without origin) switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Nanosecond: @@ -89,7 +94,7 @@ public: case IntervalKind::Second: case IntervalKind::Minute: case IntervalKind::Hour: - case IntervalKind::Day: + case IntervalKind::Day: /// weird why Day leads to DateTime but too afraid to change it result_type = ResultType::DateTime; break; case IntervalKind::Week: @@ -101,31 +106,26 @@ public: } }; - enum class ThirdArgument - { - IsTimezone, - IsOrigin - }; - ThirdArgument third_argument; /// valid only if 3rd argument is given auto check_third_argument = [&] { const DataTypePtr & type_arg3 = arguments[2].type; if (isString(type_arg3)) { - third_argument = ThirdArgument::IsTimezone; + overload = Overload::Default; + if (value_is_date && result_type == ResultType::Date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", getName(), interval_type->getKind().toString()); } - else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3)) + else if (isDateTimeOrDateTime64(type_arg3) || isDate(type_arg3)) { - third_argument = ThirdArgument::IsOrigin; + overload = Overload::Origin; if (isDateTime64(arguments[0].type) && isDateTime64(arguments[2].type)) result_type = ResultType::DateTime64; else if (isDateTime(arguments[0].type) && isDateTime(arguments[2].type)) result_type = ResultType::DateTime; - else if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && (isDate(arguments[2].type) || isDate32(arguments[2].type))) + else if (isDate(arguments[0].type) && isDate(arguments[2].type)) result_type = ResultType::Date; else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); @@ -138,7 +138,7 @@ public: auto check_fourth_argument = [&] { - if (third_argument != ThirdArgument::IsOrigin) /// sanity check + if (overload != Overload::Origin) /// sanity check throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. " "The third argument must a Date/Date32/DateTime/DateTime64 with a constant origin", arguments[2].type->getName(), getName()); @@ -185,7 +185,7 @@ public: return std::make_shared(); case ResultType::DateTime: { - const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; + const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); } case ResultType::DateTime64: @@ -200,7 +200,7 @@ public: else if (interval_type->getKind() == IntervalKind::Millisecond) scale = 3 > scale ? 3 : scale; - const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3; + const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); } } @@ -278,25 +278,25 @@ private: if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. Must contain dates or dates with time", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); } template ColumnPtr dispatchForIntervalColumn( const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const + const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be an interval of time.", getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a time interval", getName()); const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be a const interval of time.", getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a const time interval", getName()); - Int64 num_units = interval_column_const_int64->getValue(); + const Int64 num_units = interval_column_const_int64->getValue(); if (num_units <= 0) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for second argument of function {} must be positive.", getName()); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for 2nd argument of function {} must be positive", getName()); switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { @@ -328,12 +328,11 @@ private: } template - Int64 decideScaleOnPrecision(const UInt16 scale) const + Int64 decideScaleOnPrecision() const { static constexpr Int64 MILLISECOND_SCALE = 1000; static constexpr Int64 MICROSECOND_SCALE = 1000000; static constexpr Int64 NANOSECOND_SCALE = 1000000000; - Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); switch (unit) { case IntervalKind::Millisecond: @@ -343,37 +342,41 @@ private: case IntervalKind::Nanosecond: return NANOSECOND_SCALE; default: - return scale_multiplier; + return 1; } } - template + template ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const { - using ToColumnType = typename ToDataType::ColumnType; - using ToFieldType = typename ToDataType::FieldType; + using ResultColumnType = typename ResultDataType::ColumnType; + using ResultFieldType = typename ResultDataType::FieldType; const auto & time_data = time_column_type.getData(); size_t size = time_data.size(); auto result_col = result_type->createColumn(); - auto * col_to = assert_cast(result_col.get()); + auto * col_to = assert_cast(result_col.get()); auto & result_data = col_to->getData(); result_data.resize(size); - Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); - Int64 scale_on_interval = decideScaleOnPrecision(scale); - Int64 scale_diff = scale_on_interval > scale_multiplier ? scale_on_interval / scale_multiplier : scale_multiplier / scale_on_interval; + Int64 scale_on_time = DecimalUtils::scaleMultiplier(scale); // scale that depends on type of arguments + Int64 scale_on_interval = decideScaleOnPrecision(); // scale that depends on the Interval + /// In case if we have a difference between time arguments and Interval, we need to calculate the difference between them + /// to get the right precision for the result. + Int64 scale_diff = scale_on_interval > scale_on_time ? scale_on_interval / scale_on_time : scale_on_time / scale_on_interval; if (origin_column.column == nullptr) { for (size_t i = 0; i != size; ++i) { result_data[i] = 0; - if (scale_on_interval < scale_multiplier) - result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)) * scale_diff; + if (scale_on_interval < scale_on_time) + /// if we have a time argument that has bigger scale than the interval can contain, we need + /// to return a value with bigger precision and thus we should multiply result on the scale difference. + result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_interval)) * scale_diff; else - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_time)); } } else @@ -387,31 +390,54 @@ private: throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); t -= origin; - auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_multiplier)); + auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_on_time)); static constexpr size_t SECONDS_PER_DAY = 86400; result_data[i] = 0; if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) { + /// By default, when we use week, month, quarter or year interval, we get date return type. So, simply add values. if (isDate(result_type) || isDate32(result_type)) result_data[i] += origin + res; - else if (isDateTime64(result_type)) - result_data[i] += origin + (res * SECONDS_PER_DAY * scale_multiplier); - else + /// When we use DateTime arguments, we should keep in mind that we also have hours, minutes and seconds there, + /// so we need to multiply result by amount of seconds per day. + else if (isDateTime(result_type)) result_data[i] += origin + res * SECONDS_PER_DAY; + /// When we use DateTime64 arguments, we also should multiply it on right scale. + else + result_data[i] += origin + (res * SECONDS_PER_DAY * scale_on_time); } else { + /// In this case result will be calculated as datetime, so we need to get the amount of days if the arguments are Date. if (isDate(result_type) || isDate32(result_type)) res = res / SECONDS_PER_DAY; - if (scale_on_interval > scale_multiplier) - result_data[i] += (origin + res / scale_diff) * scale_diff; - else if (scale_on_interval == scale_multiplier && scale_on_interval % 1000 != 0 && scale_multiplier != 10) - result_data[i] += origin + (res * scale_on_interval); + /// Case when Interval has default scale + if (scale_on_interval == 1) + { + /// Case when the arguments are DateTime64 with precision like 4,5,7,8. Here res has right precision and origin doesn't. + if (scale_on_time % 1000 != 0 && scale_on_time >= 1000) + result_data[i] += (origin + res / scale_on_time) * scale_on_time; + /// Special case when the arguments are DateTime64 with precision 2. Here origin has right precision and res doesn't + else if (scale_on_time == 100) + result_data[i] += (origin + res * scale_on_time); + /// Cases when precision of DateTime64 is 1, 3, 6, 9 e.g. has right precision in res and origin. + else + result_data[i] += (origin + res); + } + /// Case when Interval has some specific scale (3,6,9) else - result_data[i] += origin + res * scale_diff; + { + /// If we have a time argument that has bigger scale than the interval can contain, we need + /// to return a value with bigger precision and thus we should multiply result on the scale difference. + if (scale_on_interval < scale_on_time) + result_data[i] += origin + res * scale_diff; + /// The other case: interval has bigger scale than the interval or they have the same scale, so res has the right precision and origin doesn't + else + result_data[i] += (origin + res / scale_diff) * scale_diff; + } } } } @@ -419,8 +445,6 @@ private: } }; -} - REGISTER_FUNCTION(ToStartOfInterval) { factory.registerFunction(); From 36fa954f5000c8130f2556abba6052392dd16fa7 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 12 Dec 2023 18:19:11 +0100 Subject: [PATCH 0035/1722] Stly check --- src/Functions/toStartOfInterval.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index f4171748b92..fb071c952ab 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -371,7 +371,7 @@ private: { result_data[i] = 0; if (scale_on_interval < scale_on_time) - /// if we have a time argument that has bigger scale than the interval can contain, we need + /// if we have a time argument that has bigger scale than the interval can contain, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_interval)) * scale_diff; else @@ -429,7 +429,7 @@ private: /// Case when Interval has some specific scale (3,6,9) else { - /// If we have a time argument that has bigger scale than the interval can contain, we need + /// If we have a time argument that has bigger scale than the interval can contain, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. if (scale_on_interval < scale_on_time) result_data[i] += origin + res * scale_diff; From 202ca21e3f1366479ec4f0d786adeab6e54cc3d4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 13 Dec 2023 15:16:51 +0100 Subject: [PATCH 0036/1722] fix tests --- src/Functions/toStartOfInterval.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index fb071c952ab..56e721b7601 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -370,12 +370,12 @@ private: for (size_t i = 0; i != size; ++i) { result_data[i] = 0; - if (scale_on_interval < scale_on_time) - /// if we have a time argument that has bigger scale than the interval can contain, we need + if (scale_on_interval < scale_on_time && scale_on_interval != 1) + /// If we have a time argument that has bigger scale than the interval can contain and interval is not default, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. - result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_interval)) * scale_diff; + result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_time)) * scale_diff; else - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_time)); + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_time)); } } else From 174309821a9813dba1d2a090768332e9de9470e3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 19 Dec 2023 20:48:30 +0000 Subject: [PATCH 0037/1722] Small fixups --- src/Functions/toStartOfInterval.cpp | 181 +++++++++++++--------------- 1 file changed, 85 insertions(+), 96 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 56e721b7601..81a2fd0a75d 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes class FunctionToStartOfInterval : public IFunction { -public: +private: enum class Overload { Default, /// toStartOfInterval(time, interval) or toStartOfInterval(time, interval, timezone) @@ -39,6 +39,7 @@ public: }; mutable Overload overload; +public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } static constexpr auto name = "toStartOfInterval"; @@ -82,7 +83,9 @@ public: "Illegal type {} of 2nd argument of function {}, expected a time interval", type_arg2->getName(), getName()); - /// Result here is determined for default overload (without origin) + overload = Overload::Default; + + /// Determine result type for default overload (no origin) switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Nanosecond: @@ -110,21 +113,20 @@ public: const DataTypePtr & type_arg3 = arguments[2].type; if (isString(type_arg3)) { - overload = Overload::Default; - if (value_is_date && result_type == ResultType::Date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", + "A timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", getName(), interval_type->getKind().toString()); } - else if (isDateTimeOrDateTime64(type_arg3) || isDate(type_arg3)) + else if (isDate(type_arg3) || isDateTime(type_arg3) || isDateTime64(type_arg3)) { overload = Overload::Origin; - if (isDateTime64(arguments[0].type) && isDateTime64(arguments[2].type)) + const DataTypePtr & type_arg1 = arguments[0].type; + if (isDateTime64(type_arg1) && isDateTime64(type_arg3)) result_type = ResultType::DateTime64; - else if (isDateTime(arguments[0].type) && isDateTime(arguments[2].type)) + else if (isDateTime(type_arg1) && isDateTime(type_arg3)) result_type = ResultType::DateTime; - else if (isDate(arguments[0].type) && isDate(arguments[2].type)) + else if (isDate(type_arg1) && isDate(type_arg3)) result_type = ResultType::Date; else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); @@ -149,7 +151,7 @@ public: type_arg4->getName(), getName()); if (value_is_date && result_type == ResultType::Date) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", + "A timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64", getName(), interval_type->getKind().toString()); }; @@ -190,14 +192,14 @@ public: case ResultType::DateTime64: { UInt32 scale = 0; - if (isDate32(arguments[0].type) || isDateTime64(arguments[0].type)) + if (isDateTime64(arguments[0].type)) scale = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Nanosecond) - scale = 9 > scale ? 9 : scale; + scale = (9 > scale) ? 9 : scale; else if (interval_type->getKind() == IntervalKind::Microsecond) - scale = 6 > scale ? 6 : scale; + scale = (6 > scale) ? 6 : scale; else if (interval_type->getKind() == IntervalKind::Millisecond) - scale = 3 > scale ? 3 : scale; + scale = (3 > scale) ? 3 : scale; const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false)); @@ -213,20 +215,19 @@ public: const auto & interval_column = arguments[1]; ColumnWithTypeAndName origin_column; - const bool has_origin_arg = (arguments.size() == 3 && isDateOrDate32OrDateTimeOrDateTime64(arguments[2].type)) || arguments.size() == 4; - if (has_origin_arg) + if (overload == Overload::Origin) origin_column = arguments[2]; - const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3; + const size_t time_zone_arg_num = (overload == Overload::Origin) ? 3 : 2; const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0); - ColumnPtr result_column = nullptr; - if (isDateTime64(result_type)) - result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + ColumnPtr result_column; + if (isDate(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); else if (isDateTime(result_type)) result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); - else - result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); + else if (isDateTime64(result_type)) + result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone); return result_column; } @@ -238,44 +239,24 @@ private: const auto & time_column_type = *time_column.type.get(); const auto & time_column_col = *time_column.column.get(); - if (isDateTime64(time_column_type)) + if (isDate(time_column_type)) { - if (origin_column.column != nullptr && !isDateTime64(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); - - const auto * time_column_vec = checkAndGetColumn(time_column_col); - auto scale = assert_cast(time_column_type).getScale(); - - if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale); - } - else if (isDateTime(time_column_type)) - { - if (origin_column.column != nullptr && !isDateTime(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); - - const auto * time_column_vec = checkAndGetColumn(time_column_col); - if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); - } - else if (isDate(time_column_type)) - { - if (origin_column.column != nullptr && !isDate(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); - const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } - else if (isDate32(time_column_type)) + else if (isDateTime(time_column_type)) { - if (origin_column.column != nullptr) - if (!isDate32(origin_column.type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Datetime argument and origin argument for function {} must have the same type", getName()); - - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); + } + else if (isDateTime64(time_column_type)) + { + const auto * time_column_vec = checkAndGetColumn(time_column_col); + auto scale = assert_cast(time_column_type).getScale(); + if (time_column_vec) + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale); } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); } @@ -327,26 +308,23 @@ private: } template - Int64 decideScaleOnPrecision() const + static Int64 scaleFromInterval() { - static constexpr Int64 MILLISECOND_SCALE = 1000; - static constexpr Int64 MICROSECOND_SCALE = 1000000; - static constexpr Int64 NANOSECOND_SCALE = 1000000000; switch (unit) { case IntervalKind::Millisecond: - return MILLISECOND_SCALE; + return 1'000; case IntervalKind::Microsecond: - return MICROSECOND_SCALE; + return 1'000'000; case IntervalKind::Nanosecond: - return NANOSECOND_SCALE; + return 1'000'000'000; default: return 1; } } template - ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const + ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale) const { using ResultColumnType = typename ResultDataType::ColumnType; using ResultFieldType = typename ResultDataType::FieldType; @@ -359,23 +337,29 @@ private: auto & result_data = col_to->getData(); result_data.resize(size); - Int64 scale_on_time = DecimalUtils::scaleMultiplier(scale); // scale that depends on type of arguments - Int64 scale_on_interval = decideScaleOnPrecision(); // scale that depends on the Interval + const Int64 scale_time = DecimalUtils::scaleMultiplier(scale); + const Int64 scale_interval = scaleFromInterval(); + /// In case if we have a difference between time arguments and Interval, we need to calculate the difference between them /// to get the right precision for the result. - Int64 scale_diff = scale_on_interval > scale_on_time ? scale_on_interval / scale_on_time : scale_on_time / scale_on_interval; + const Int64 scale_diff = (scale_interval > scale_time) ? (scale_interval / scale_time) : (scale_time / scale_interval); if (origin_column.column == nullptr) { - for (size_t i = 0; i != size; ++i) + if (scale_time > scale_interval && scale_interval != 1) { - result_data[i] = 0; - if (scale_on_interval < scale_on_time && scale_on_interval != 1) + for (size_t i = 0; i != size; ++i) + { /// If we have a time argument that has bigger scale than the interval can contain and interval is not default, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. - result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_time)) * scale_diff; - else - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_on_time)); + result_data[i] = 0; + result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_time)) * scale_diff; + } + } + else + { + for (size_t i = 0; i != size; ++i) + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_time)); } } else @@ -386,55 +370,60 @@ private: { auto t = time_data[i]; if (origin > static_cast(t)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date/datetime"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date / date with time"); + + /// The trick to calculate the interval starting from an offset is to + /// 1. subtract the offset, + /// 2. perform the calculation, and + /// 3. add the offset to the result. t -= origin; - auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_on_time)); + auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_time)); - static constexpr size_t SECONDS_PER_DAY = 86400; + static constexpr size_t SECONDS_PER_DAY = 86'400; result_data[i] = 0; if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) { - /// By default, when we use week, month, quarter or year interval, we get date return type. So, simply add values. - if (isDate(result_type) || isDate32(result_type)) + /// For such intervals, ToStartOfInterval::execute() returns days + if (isDate(result_type)) result_data[i] += origin + res; - /// When we use DateTime arguments, we should keep in mind that we also have hours, minutes and seconds there, - /// so we need to multiply result by amount of seconds per day. else if (isDateTime(result_type)) result_data[i] += origin + res * SECONDS_PER_DAY; - /// When we use DateTime64 arguments, we also should multiply it on right scale. - else - result_data[i] += origin + (res * SECONDS_PER_DAY * scale_on_time); + else if (isDateTime64(result_type)) + result_data[i] += origin + (res * SECONDS_PER_DAY * scale_time); } else { - /// In this case result will be calculated as datetime, so we need to get the amount of days if the arguments are Date. - if (isDate(result_type) || isDate32(result_type)) + /// ToStartOfInterval::execute() returns seconds + + if (isDate(result_type)) res = res / SECONDS_PER_DAY; - /// Case when Interval has default scale - if (scale_on_interval == 1) + if (scale_interval == 1) { - /// Case when the arguments are DateTime64 with precision like 4,5,7,8. Here res has right precision and origin doesn't. - if (scale_on_time % 1000 != 0 && scale_on_time >= 1000) - result_data[i] += (origin + res / scale_on_time) * scale_on_time; - /// Special case when the arguments are DateTime64 with precision 2. Here origin has right precision and res doesn't - else if (scale_on_time == 100) - result_data[i] += (origin + res * scale_on_time); - /// Cases when precision of DateTime64 is 1, 3, 6, 9 e.g. has right precision in res and origin. + /// Interval has default scale, i.e. Year - Second + + if (scale_time % 1000 != 0 && scale_time >= 1000) + /// The arguments are DateTime64 with precision like 4,5,7,8. Here res has right precision and origin doesn't. + result_data[i] += (origin + res / scale_time) * scale_time; + else if (scale_time == 100) + /// The arguments are DateTime64 with precision 2. Here origin has right precision and res doesn't + result_data[i] += (origin + res * scale_time); else + /// Precision of DateTime64 is 1, 3, 6, 9, e.g. has right precision in res and origin. result_data[i] += (origin + res); } - /// Case when Interval has some specific scale (3,6,9) else { - /// If we have a time argument that has bigger scale than the interval can contain, we need - /// to return a value with bigger precision and thus we should multiply result on the scale difference. - if (scale_on_interval < scale_on_time) + /// Interval has some specific scale (3,6,9), i.e. Millisecond - Nanosecond + + if (scale_interval < scale_time) + /// If we have a time argument that has bigger scale than the interval can contain, we need + /// to return a value with bigger precision and thus we should multiply result on the scale difference. result_data[i] += origin + res * scale_diff; - /// The other case: interval has bigger scale than the interval or they have the same scale, so res has the right precision and origin doesn't else + /// The other case: interval has bigger scale than the interval or they have the same scale, so res has the right precision and origin doesn't result_data[i] += (origin + res / scale_diff) * scale_diff; } } From 861421d27ac74fe11921c4ba901dcccb94df76e8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jan 2024 16:03:22 +0000 Subject: [PATCH 0038/1722] fixes --- src/Functions/DateTimeTransforms.h | 57 ++++++--- src/Functions/toStartOfInterval.cpp | 117 ++++++++++++------ ...to_start_of_interval_with_origin.reference | 32 ++--- ...02916_to_start_of_interval_with_origin.sql | 8 +- 4 files changed, 139 insertions(+), 75 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 74b37e18907..dbe2b11d7b2 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -487,7 +487,7 @@ struct ToStartOfInterval { throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); } - static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 nanoseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0) { if (scale_multiplier < 1000000000) { @@ -522,7 +522,7 @@ struct ToStartOfInterval { throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); } - static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 microseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0) { if (scale_multiplier < 1000000) { @@ -565,7 +565,7 @@ struct ToStartOfInterval { throwDateTimeIsNotSupported(TO_START_OF_INTERVAL_NAME); } - static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 milliseconds, const DateLUTImpl &, Int64 scale_multiplier, Int64 /*origin*/ = 0) { if (scale_multiplier < 1000) { @@ -608,7 +608,7 @@ struct ToStartOfInterval { return time_zone.toStartOfSecondInterval(t, seconds); } - static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds); } @@ -629,7 +629,7 @@ struct ToStartOfInterval { return time_zone.toStartOfMinuteInterval(t, minutes); } - static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes); } @@ -650,7 +650,7 @@ struct ToStartOfInterval { return time_zone.toStartOfHourInterval(t, hours); } - static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfHourInterval(t / scale_multiplier, hours); } @@ -671,7 +671,7 @@ struct ToStartOfInterval { return static_cast(time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days)); } - static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 days, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 /*origin*/ = 0) { return time_zone.toStartOfDayInterval(time_zone.toDayNum(t / scale_multiplier), days); } @@ -692,9 +692,12 @@ struct ToStartOfInterval { return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks); } - static UInt16 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 weeks, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); + if (origin == 0) + return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); + else + return ToStartOfInterval::execute(t, weeks * 7, time_zone, scale_multiplier, origin); } }; @@ -713,9 +716,24 @@ struct ToStartOfInterval { return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months); } - static UInt16 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 months, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); + if (origin == 0) + return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t / scale_multiplier), months); + else + { + Int64 days = time_zone.toDayOfMonth(t / scale_multiplier + origin) - time_zone.toDayOfMonth(origin); + Int64 months_to_add = time_zone.toMonth(t / scale_multiplier + origin) - time_zone.toMonth(origin); + Int64 years = time_zone.toYear(t / scale_multiplier + origin) - time_zone.toYear(origin); + months_to_add = days < 0 ? months_to_add - 1 : months_to_add; + months_to_add += years * 12; + Int64 month_multiplier = (months_to_add / months) * months; + Int64 a = 0; + + a = time_zone.addMonths(time_zone.toDate(origin), month_multiplier); + // a += time_zone.toTime(origin); + return a - time_zone.toDate(origin); + } } }; @@ -734,9 +752,12 @@ struct ToStartOfInterval { return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters); } - static UInt16 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 quarters, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); + if (origin == 0) + return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); + else + return ToStartOfInterval::execute(t, quarters * 3, time_zone, scale_multiplier, origin); } }; @@ -755,9 +776,15 @@ struct ToStartOfInterval { return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years); } - static UInt16 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier) + static Int64 execute(Int64 t, Int64 years, const DateLUTImpl & time_zone, Int64 scale_multiplier, Int64 origin = 0) { - return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); + if (origin == 0) + return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); + else + { + auto a = ToStartOfInterval::execute(t, years * 12, time_zone, scale_multiplier, origin); + return a; + } } }; diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 81a2fd0a75d..1ba7fed4bee 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,17 +1,17 @@ +#include +#include +#include #include #include #include #include -#include "DataTypes/IDataType.h" #include #include #include #include #include -#include #include #include -#include #include #include #include @@ -270,6 +270,27 @@ private: if (!interval_type) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a time interval", getName()); + if (isDate(time_data_type) || isDateTime(time_data_type)) + { + switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) + { + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: + if (isDate(time_data_type) || isDateTime(time_data_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type {}", isDate(time_data_type) ? "Date" : "DateTime"); + break; + case IntervalKind::Second: + case IntervalKind::Minute: + case IntervalKind::Hour: + if (isDate(time_data_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type Date"); + break; + default: + break; + } + } + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a const time interval", getName()); @@ -337,94 +358,110 @@ private: auto & result_data = col_to->getData(); result_data.resize(size); - const Int64 scale_time = DecimalUtils::scaleMultiplier(scale); + const Int64 scale_endtime = DecimalUtils::scaleMultiplier(scale); const Int64 scale_interval = scaleFromInterval(); /// In case if we have a difference between time arguments and Interval, we need to calculate the difference between them /// to get the right precision for the result. - const Int64 scale_diff = (scale_interval > scale_time) ? (scale_interval / scale_time) : (scale_time / scale_interval); + const Int64 scale_diff = (scale_interval > scale_endtime) ? (scale_interval / scale_endtime) : (scale_endtime / scale_interval); if (origin_column.column == nullptr) { - if (scale_time > scale_interval && scale_interval != 1) + if (scale_endtime > scale_interval && scale_interval != 1) { for (size_t i = 0; i != size; ++i) { /// If we have a time argument that has bigger scale than the interval can contain and interval is not default, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. result_data[i] = 0; - result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_time)) * scale_diff; + result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_endtime)) * scale_diff; } } else { for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_time)); + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_endtime)); } } else { UInt64 origin = origin_column.column->get64(0); + Int64 origin_scale = 1; + if (isDateTime64(origin_column.type.get())) + origin_scale = assert_cast(*origin_column.type.get()).getScale(); for (size_t i = 0; i != size; ++i) { - auto t = time_data[i]; - if (origin > static_cast(t)) + UInt64 end_time = time_data[i]; + + if (origin > static_cast(end_time) && origin_scale == scale) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date / date with time"); + else if (origin_scale > scale) + origin /= static_cast(std::pow(10, origin_scale - scale)); /// If aguments have different scales, we make + else if (origin_scale < scale) /// origin argument to have the same scale as the first argument. + origin *= static_cast(std::pow(10, scale - origin_scale)); /// The trick to calculate the interval starting from an offset is to /// 1. subtract the offset, /// 2. perform the calculation, and /// 3. add the offset to the result. - t -= origin; - auto res = static_cast(ToStartOfInterval::execute(t, num_units, time_zone, scale_time)); - static constexpr size_t SECONDS_PER_DAY = 86'400; - result_data[i] = 0; - if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) + + if (isDate(origin_column.type.get())) /// We need to perform calculations on dateTime (dateTime64) values only. { - /// For such intervals, ToStartOfInterval::execute() returns days - if (isDate(result_type)) - result_data[i] += origin + res; - else if (isDateTime(result_type)) - result_data[i] += origin + res * SECONDS_PER_DAY; - else if (isDateTime64(result_type)) - result_data[i] += origin + (res * SECONDS_PER_DAY * scale_time); + end_time *= SECONDS_PER_DAY; + origin *= SECONDS_PER_DAY; + } + + Int64 delta = (end_time - origin) * (isDateTime64(origin_column.type.get()) ? 1 : scale_endtime); /// No need to multiply on scale endtime if we have dateTime64 argument. + Int64 offset = 0; + + { + auto origin_data = isDateTime64(result_type) ? origin / scale_endtime : origin; + offset = static_cast(ToStartOfInterval::execute(delta, num_units, time_zone, scale_endtime, origin_data)); + } + + + if (isDate(result_type)) /// The result should be a date and the calculations were as datetime. + result_data[i] += (origin + offset) / SECONDS_PER_DAY; + else if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) + { + if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the right scale. + offset *= scale_endtime; + + result_data[i] += origin + offset; } else { - /// ToStartOfInterval::execute() returns seconds - - if (isDate(result_type)) - res = res / SECONDS_PER_DAY; + /// ToStartOfInterval::execute() returns seconds. if (scale_interval == 1) { - /// Interval has default scale, i.e. Year - Second + if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the right scale. + offset *= scale_endtime; - if (scale_time % 1000 != 0 && scale_time >= 1000) - /// The arguments are DateTime64 with precision like 4,5,7,8. Here res has right precision and origin doesn't. - result_data[i] += (origin + res / scale_time) * scale_time; - else if (scale_time == 100) - /// The arguments are DateTime64 with precision 2. Here origin has right precision and res doesn't - result_data[i] += (origin + res * scale_time); + /// Interval has default scale, i.e. Year - Second. + + if (scale_endtime % 1000 != 0 && scale_endtime >= 1000) + /// The arguments are DateTime64 with precision like 4,5,7,8. Here offset has right precision and origin doesn't. + result_data[i] += (origin + offset / scale_endtime) * scale_endtime; else - /// Precision of DateTime64 is 1, 3, 6, 9, e.g. has right precision in res and origin. - result_data[i] += (origin + res); + /// Precision of DateTime64 is 1, 2, 3, 6, 9, e.g. has right precision in offset and origin. + result_data[i] += (origin + offset); } else { - /// Interval has some specific scale (3,6,9), i.e. Millisecond - Nanosecond + /// Interval has some specific scale (3,6,9), i.e. Millisecond - Nanosecond. - if (scale_interval < scale_time) + if (scale_interval < scale_endtime) /// If we have a time argument that has bigger scale than the interval can contain, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. - result_data[i] += origin + res * scale_diff; + result_data[i] += origin + offset * scale_diff; else - /// The other case: interval has bigger scale than the interval or they have the same scale, so res has the right precision and origin doesn't - result_data[i] += (origin + res / scale_diff) * scale_diff; + /// The other case: interval has bigger scale than the interval or they have the same scale, so offset has the right precision and origin doesn't. + result_data[i] += (origin + offset / scale_diff) * scale_diff; } } } diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference index 870853bc371..969e2726902 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference @@ -1,40 +1,40 @@ -- Negative tests -Time and origin as Time +Time and origin as Date 2023-02-01 2023-08-01 -2023-10-09 -2023-10-05 2023-10-08 +2023-10-08 +2023-10-09 Time and origin as DateTime 2023-02-01 09:08:07 2023-08-01 09:08:07 -2023-10-09 09:08:07 -2023-10-05 09:08:07 +2023-10-08 09:08:07 +2023-10-08 09:08:07 2023-10-09 09:08:07 2023-10-09 10:10:07 -2023-10-09 10:11:11 +2023-10-09 10:11:07 2023-10-09 10:11:12 Time and origin as DateTime64(9) 2023-02-01 09:08:07.123456789 2023-08-01 09:08:07.123456789 2023-09-10 09:08:07.123456789 -2023-10-05 09:08:07.123456789 -2023-10-08 09:08:07.123543189 -2023-10-09 09:10:07.123460389 -2023-10-09 10:10:11.123456849 -2023-10-09 10:11:10.123456791 +2023-10-08 09:08:07.123456789 +2023-10-09 09:08:07.123456789 +2023-10-09 10:10:07.123456789 +2023-10-09 10:11:11.123456789 +2023-10-09 10:11:12.123456789 2023-10-09 10:11:12.987456789 2023-10-09 10:11:12.987653789 2023-10-09 10:11:12.987654321 Time and origin as DateTime64(3) 2023-02-01 09:08:07.123 2023-08-01 09:08:07.123 +2023-10-08 09:08:07.123 +2023-10-08 09:08:07.123 2023-10-09 09:08:07.123 -2023-10-05 09:08:07.123 -2023-10-08 09:09:33.523 -2023-10-09 09:10:10.723 -2023-10-09 10:10:11.183 -2023-10-09 10:11:10.125 +2023-10-09 10:10:07.123 +2023-10-09 10:11:11.123 +2023-10-09 10:11:12.123 2023-10-09 10:11:12.987 2023-10-09 10:11:12.987000 2023-10-09 10:11:12.987000000 diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql index 71f5fb7fb36..4f8a96b093d 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.sql @@ -23,7 +23,7 @@ SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), 5 -- too many arguments SELECT toStartOfInterval(toDateTime('2023-01-02 14:45:50'), toIntervalYear(1), toDateTime('2020-01-02 14:44:30'), 'Europe/Amsterdam', 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT 'Time and origin as Time'; +SELECT 'Time and origin as Date'; SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalYear(1), toDate('2022-02-01')); SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalQuarter(1), toDate('2022-02-01')); SELECT toStartOfInterval(toDate('2023-10-09'), toIntervalMonth(1), toDate('2023-09-08')); @@ -43,8 +43,8 @@ SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMonth(1), SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalWeek(1), toDateTime('2023-10-01 09:08:07')); SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalDay(1), toDateTime('2023-10-08 09:08:07')); SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalHour(1), toDateTime('2023-10-09 09:10:07')); -SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 10:10:11')); -SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalSecond(1), toDateTime('2023-10-09 10:11:10')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMinute(1), toDateTime('2023-10-09 09:10:07')); +SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalSecond(1), toDateTime('2023-10-09 09:10:07')); SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMillisecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalMicrosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT toStartOfInterval(toDateTime('2023-10-09 10:11:12'), toIntervalNanosecond(1), toDateTime('2023-10-09 10:11:12')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } @@ -56,7 +56,7 @@ SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toInt SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalWeek(1), toDateTime64('2023-10-01 09:08:07.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalDay(1), toDateTime64('2023-10-08 09:08:07.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalHour(1), toDateTime64('2023-10-09 09:10:07.123456789', 9)); -SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 10:10:11.123456789', 9)); +SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMinute(1), toDateTime64('2023-10-09 09:10:11.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalSecond(1), toDateTime64('2023-10-09 10:11:10.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMillisecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.987654321', 9), toIntervalMicrosecond(1), toDateTime64('2023-10-09 10:11:12.123456789', 9)); From cb645f82198250b119d734291456afe8dcdde27f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jan 2024 17:18:56 +0100 Subject: [PATCH 0039/1722] fix style --- src/Functions/toStartOfInterval.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 1ba7fed4bee..ffabf38ef20 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -397,7 +397,7 @@ private: if (origin > static_cast(end_time) && origin_scale == scale) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The origin must be before the end date / date with time"); else if (origin_scale > scale) - origin /= static_cast(std::pow(10, origin_scale - scale)); /// If aguments have different scales, we make + origin /= static_cast(std::pow(10, origin_scale - scale)); /// If arguments have different scales, we make else if (origin_scale < scale) /// origin argument to have the same scale as the first argument. origin *= static_cast(std::pow(10, scale - origin_scale)); @@ -439,16 +439,16 @@ private: if (scale_interval == 1) { - if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the right scale. + if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the correct scale. offset *= scale_endtime; /// Interval has default scale, i.e. Year - Second. if (scale_endtime % 1000 != 0 && scale_endtime >= 1000) - /// The arguments are DateTime64 with precision like 4,5,7,8. Here offset has right precision and origin doesn't. + /// The arguments are DateTime64 with precision like 4,5,7,8. Here offset has correct precision and origin doesn't. result_data[i] += (origin + offset / scale_endtime) * scale_endtime; else - /// Precision of DateTime64 is 1, 2, 3, 6, 9, e.g. has right precision in offset and origin. + /// Precision of DateTime64 is 1, 2, 3, 6, 9, e.g. has correct precision in offset and origin. result_data[i] += (origin + offset); } else From 1117284be7f72a22ac841af8f1a91ec853adc900 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jan 2024 23:43:58 +0000 Subject: [PATCH 0040/1722] fix overflow --- src/Functions/DateTimeTransforms.h | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index dbe2b11d7b2..e9cee9616fb 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -697,7 +697,12 @@ struct ToStartOfInterval if (origin == 0) return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); else - return ToStartOfInterval::execute(t, weeks * 7, time_zone, scale_multiplier, origin); + { + if (const auto weeks_to_days = weeks * 7; weeks_to_days / 7 == weeks) // Check if multiplication doesn't overflow Int64 value + return ToStartOfInterval::execute(t, weeks_to_days, time_zone, scale_multiplier, origin); + else + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 7 is out of bounds for type Int64", weeks); + } } }; @@ -728,11 +733,8 @@ struct ToStartOfInterval months_to_add = days < 0 ? months_to_add - 1 : months_to_add; months_to_add += years * 12; Int64 month_multiplier = (months_to_add / months) * months; - Int64 a = 0; - a = time_zone.addMonths(time_zone.toDate(origin), month_multiplier); - // a += time_zone.toTime(origin); - return a - time_zone.toDate(origin); + return time_zone.addMonths(time_zone.toDate(origin), month_multiplier) - time_zone.toDate(origin); } } }; @@ -757,7 +759,12 @@ struct ToStartOfInterval if (origin == 0) return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); else - return ToStartOfInterval::execute(t, quarters * 3, time_zone, scale_multiplier, origin); + { + if (const auto quarters_to_months = quarters * 3; quarters_to_months / 3 == quarters) // Check if multiplication doesn't overflow Int64 value + return ToStartOfInterval::execute(t, quarters_to_months, time_zone, scale_multiplier, origin); + else + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 3 is out of bounds for type Int64", quarters); + } } }; @@ -782,8 +789,10 @@ struct ToStartOfInterval return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); else { - auto a = ToStartOfInterval::execute(t, years * 12, time_zone, scale_multiplier, origin); - return a; + if (const auto years_to_months = years * 12; years_to_months / 12 == years) // Check if multiplication doesn't overflow Int64 value + return ToStartOfInterval::execute(t, years_to_months, time_zone, scale_multiplier, origin); + else + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 12 is out of bounds for type Int64", years); } } }; From 07f031ec8a26d0ee7081d725fe59b312101bcae8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 8 Jan 2024 15:34:22 +0100 Subject: [PATCH 0041/1722] fix fuzzer --- src/Functions/DateTimeTransforms.h | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index e9cee9616fb..dd843daed8c 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -20,6 +20,7 @@ namespace DB { +static Int64 Int64_max_value = std::numeric_limits::max(); static constexpr auto microsecond_multiplier = 1000000; static constexpr auto millisecond_multiplier = 1000; @@ -698,8 +699,8 @@ struct ToStartOfInterval return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t / scale_multiplier), weeks); else { - if (const auto weeks_to_days = weeks * 7; weeks_to_days / 7 == weeks) // Check if multiplication doesn't overflow Int64 value - return ToStartOfInterval::execute(t, weeks_to_days, time_zone, scale_multiplier, origin); + if (weeks < Int64_max_value / 7) // Check if multiplication doesn't overflow Int64 value + return ToStartOfInterval::execute(t, weeks * 7, time_zone, scale_multiplier, origin); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 7 is out of bounds for type Int64", weeks); } @@ -760,8 +761,8 @@ struct ToStartOfInterval return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t / scale_multiplier), quarters); else { - if (const auto quarters_to_months = quarters * 3; quarters_to_months / 3 == quarters) // Check if multiplication doesn't overflow Int64 value - return ToStartOfInterval::execute(t, quarters_to_months, time_zone, scale_multiplier, origin); + if (quarters < Int64_max_value / 3) // Check if multiplication doesn't overflow Int64 value + return ToStartOfInterval::execute(t, quarters * 3, time_zone, scale_multiplier, origin); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 3 is out of bounds for type Int64", quarters); } @@ -789,8 +790,8 @@ struct ToStartOfInterval return time_zone.toStartOfYearInterval(time_zone.toDayNum(t / scale_multiplier), years); else { - if (const auto years_to_months = years * 12; years_to_months / 12 == years) // Check if multiplication doesn't overflow Int64 value - return ToStartOfInterval::execute(t, years_to_months, time_zone, scale_multiplier, origin); + if (years < Int64_max_value / 12) // Check if multiplication doesn't overflow Int64 value + return ToStartOfInterval::execute(t, years * 12, time_zone, scale_multiplier, origin); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 12 is out of bounds for type Int64", years); } From 26561c6bdd22085d6fe8537fa37cbf12de573efd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 12 Jan 2024 17:47:17 +0100 Subject: [PATCH 0042/1722] fix due to #58557 --- .../02916_to_start_of_interval_with_origin.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference index 969e2726902..552323be1a5 100644 --- a/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference +++ b/tests/queries/0_stateless/02916_to_start_of_interval_with_origin.reference @@ -24,7 +24,7 @@ Time and origin as DateTime64(9) 2023-10-09 10:11:11.123456789 2023-10-09 10:11:12.123456789 2023-10-09 10:11:12.987456789 -2023-10-09 10:11:12.987653789 +2023-10-09 10:11:12.987654789 2023-10-09 10:11:12.987654321 Time and origin as DateTime64(3) 2023-02-01 09:08:07.123 From 85a35dce28d3a367cf306c2e95edb41a3484a9c8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 12 Jan 2024 17:22:16 +0000 Subject: [PATCH 0043/1722] fix tests --- .../0_stateless/02207_subseconds_intervals.reference | 6 +++--- .../02956_fix_to_start_of_milli_microsecond.reference | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02207_subseconds_intervals.reference b/tests/queries/0_stateless/02207_subseconds_intervals.reference index b0edbda5e76..6cde773c3c4 100644 --- a/tests/queries/0_stateless/02207_subseconds_intervals.reference +++ b/tests/queries/0_stateless/02207_subseconds_intervals.reference @@ -10,14 +10,14 @@ test intervals - test microseconds 1980-12-12 12:12:12.123456 1980-12-12 12:12:12.123400 -1980-12-12 12:12:12.12345600 -1980-12-12 12:12:12.12345600 +1980-12-12 12:12:12.12345700 +1980-12-12 12:12:12.12345700 1930-12-12 12:12:12.123456 1930-12-12 12:12:12.123400 1930-12-12 12:12:12.12345600 2220-12-12 12:12:12.123456 2220-12-12 12:12:12.123400 -2220-12-12 12:12:12.12345600 +2220-12-12 12:12:12.12345700 - test milliseconds 1980-12-12 12:12:12.123 1980-12-12 12:12:12.120 diff --git a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference index d3a002c4fd4..dff0c2a9585 100644 --- a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference +++ b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference @@ -1,4 +1,4 @@ -2023-10-09 10:11:12.001 -2023-10-09 10:11:12.001 -2023-10-09 10:11:12.000 -2023-10-09 10:11:12.000 +2023-10-09 10:11:12.001000 +2023-10-09 10:11:12.001000 +2023-10-09 10:11:12.000000 +2023-10-09 10:11:12.000000 From 0d46e7555b066298603b5e0cd4dc122e74863ebf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Feb 2024 19:08:33 +0100 Subject: [PATCH 0044/1722] Fix unexpected behavior with FORMAT and SETTINGS parsing --- src/Parsers/ParserQueryWithOutput.cpp | 80 ++++++++++++++++++--------- 1 file changed, 54 insertions(+), 26 deletions(-) diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 7a627ae5f6a..340abf27c31 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -150,37 +150,65 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } + /// These two sections are allowed in an arbitrary order. ParserKeyword s_format("FORMAT"); - - if (s_format.ignore(pos, expected)) - { - ParserIdentifier format_p; - - if (!format_p.parse(pos, query_with_output.format, expected)) - return false; - setIdentifierSpecial(query_with_output.format); - - query_with_output.children.push_back(query_with_output.format); - } - - // SETTINGS key1 = value1, key2 = value2, ... ParserKeyword s_settings("SETTINGS"); - if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) - { - ParserSetQuery parser_settings(true); - if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) - return false; - query_with_output.children.push_back(query_with_output.settings_ast); - // SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery) - // Pass them manually, to apply in InterpreterSelectQuery::initSettings() - if (query->as()) + /** Why: let's take the following example: + * SELECT 1 UNION ALL SELECT 2 FORMAT TSV + * Each subquery can be put in parentheses and have its own settings: + * (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV + * And the whole query can have settings: + * (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV SETTINGS e=f + * A single query with output is parsed in the same way as the UNION ALL chain: + * SELECT 1 SETTINGS a=b FORMAT TSV SETTINGS e=f + * So while these forms have a slightly different meaning, they both exist: + * SELECT 1 SETTINGS a=b FORMAT TSV + * SELECT 1 FORMAT TSV SETTINGS e=f + * And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order. + * But while this work: + * (SELECT 1) UNION ALL (SELECT 2) FORMAT TSV SETTINGS d=f + * This does not work automatically, unless we explicitly allow different orders: + * (SELECT 1) UNION ALL (SELECT 2) SETTINGS d=f FORMAT TSV + * Inevitably, we also allow this: + * SELECT 1 SETTINGS a=b SETTINGS d=f FORMAT TSV + * ^^^^^^^^^^^^^^^^^^^^^ + * Because this part is consumed into ASTSelectWithUnionQuery + * and the rest into ASTQueryWithOutput. + */ + + for (size_t i = 0; i < 2; ++i) + { + if (!query_with_output.format && s_format.ignore(pos, expected)) { - auto settings = query_with_output.settings_ast->clone(); - assert_cast(settings.get())->print_in_format = false; - QueryWithOutputSettingsPushDownVisitor::Data data{settings}; - QueryWithOutputSettingsPushDownVisitor(data).visit(query); + ParserIdentifier format_p; + + if (!format_p.parse(pos, query_with_output.format, expected)) + return false; + setIdentifierSpecial(query_with_output.format); + + query_with_output.children.push_back(query_with_output.format); } + else if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) + { + // SETTINGS key1 = value1, key2 = value2, ... + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) + return false; + query_with_output.children.push_back(query_with_output.settings_ast); + + // SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery) + // Pass them manually, to apply in InterpreterSelectQuery::initSettings() + if (query->as()) + { + auto settings = query_with_output.settings_ast->clone(); + assert_cast(settings.get())->print_in_format = false; + QueryWithOutputSettingsPushDownVisitor::Data data{settings}; + QueryWithOutputSettingsPushDownVisitor(data).visit(query); + } + } + else + break; } node = std::move(query); From d7b34a80bbbf98ea11e0d679eeede076421748f1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 16:09:14 +0000 Subject: [PATCH 0045/1722] stash --- .../Net/include/Poco/Net/HTTPServerSession.h | 5 +++++ base/poco/Net/src/HTTPServerSession.cpp | 22 ++++++++++--------- .../library-bridge/LibraryBridgeHandlers.cpp | 16 +++++--------- .../library-bridge/LibraryBridgeHandlers.h | 6 ++--- programs/odbc-bridge/PingHandler.cpp | 2 +- programs/server/Server.cpp | 1 + src/Core/ServerSettings.h | 1 + src/IO/HTTPCommon.cpp | 15 +++++++++---- src/IO/HTTPCommon.h | 2 +- src/Server/HTTP/HTTPServer.cpp | 3 ++- src/Server/HTTP/HTTPServerResponse.h | 2 ++ .../WriteBufferFromHTTPServerResponse.cpp | 4 +--- .../HTTP/WriteBufferFromHTTPServerResponse.h | 2 -- src/Server/HTTPHandler.cpp | 3 +-- src/Server/InterserverIOHTTPHandler.cpp | 3 +-- src/Server/PrometheusRequestHandler.cpp | 10 ++------- src/Server/PrometheusRequestHandler.h | 7 +----- src/Server/ReplicasStatusHandler.cpp | 3 +-- src/Server/StaticRequestHandler.cpp | 10 ++++----- src/Server/WebUIRequestHandler.cpp | 9 +++----- .../00408_http_keep_alive.reference | 6 ++--- .../0_stateless/00501_http_head.reference | 4 ++-- 22 files changed, 63 insertions(+), 73 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index ec928af304f..192d71962bc 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -56,10 +56,15 @@ namespace Net SocketAddress serverAddress(); /// Returns the server's address. + size_t getKeepAliveTimeout() const { return _params->getKeepAliveTimeout().totalSeconds(); } + + size_t getMaxKeepAliveRequests() const { return _params->getMaxKeepAliveRequests(); } + private: bool _firstRequest; Poco::Timespan _keepAliveTimeout; int _maxKeepAliveRequests; + HTTPServerParams::Ptr _params; }; diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index d4f2b24879e..3ea689cb0cf 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -19,11 +19,12 @@ namespace Poco { namespace Net { -HTTPServerSession::HTTPServerSession(const StreamSocket& socket, HTTPServerParams::Ptr pParams): - HTTPSession(socket, pParams->getKeepAlive()), - _firstRequest(true), - _keepAliveTimeout(pParams->getKeepAliveTimeout()), - _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) +HTTPServerSession::HTTPServerSession(const StreamSocket & socket, HTTPServerParams::Ptr pParams) + : HTTPSession(socket, pParams->getKeepAlive()) + , _firstRequest(true) + , _keepAliveTimeout(pParams->getKeepAliveTimeout()) + , _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) + , _params(pParams) { setTimeout(pParams->getTimeout()); } @@ -46,11 +47,12 @@ bool HTTPServerSession::hasMoreRequests() } else if (_maxKeepAliveRequests != 0 && getKeepAlive()) { - if (_maxKeepAliveRequests > 0) - --_maxKeepAliveRequests; - return buffered() > 0 || socket().poll(_keepAliveTimeout, Socket::SELECT_READ); - } - else return false; + if (_maxKeepAliveRequests > 0) + --_maxKeepAliveRequests; + return buffered() > 0 || socket().poll(_keepAliveTimeout, Socket::SELECT_READ); + } + else + return false; } diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index 26d887cfc98..094cef6716d 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -374,10 +374,8 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ } -ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler")) +ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler")) { } @@ -401,7 +399,7 @@ void ExternalDictionaryLibraryBridgeExistsHandler::handleRequest(HTTPServerReque String res = library_handler ? "1" : "0"; - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); LOG_TRACE(log, "Sending ping response: {} (dictionary id: {})", res, dictionary_id); response.sendBuffer(res.data(), res.size()); } @@ -617,10 +615,8 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ } -CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("CatBoostLibraryBridgeExistsHandler")) +CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("CatBoostLibraryBridgeExistsHandler")) { } @@ -634,7 +630,7 @@ void CatBoostLibraryBridgeExistsHandler::handleRequest(HTTPServerRequest & reque String res = "1"; - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); LOG_TRACE(log, "Sending ping response: {}", res); response.sendBuffer(res.data(), res.size()); } diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 1db71eb24cb..83bca24ce1f 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -34,12 +34,11 @@ private: class ExternalDictionaryLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_); + ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - const size_t keep_alive_timeout; LoggerPtr log; }; @@ -77,12 +76,11 @@ private: class CatBoostLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_); + CatBoostLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - const size_t keep_alive_timeout; LoggerPtr log; }; diff --git a/programs/odbc-bridge/PingHandler.cpp b/programs/odbc-bridge/PingHandler.cpp index 80d0e2bf4a9..e5d094fb7eb 100644 --- a/programs/odbc-bridge/PingHandler.cpp +++ b/programs/odbc-bridge/PingHandler.cpp @@ -10,7 +10,7 @@ void PingHandler::handleRequest(HTTPServerRequest & /* request */, HTTPServerRes { try { - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); const char * data = "Ok.\n"; response.sendBuffer(data, strlen(data)); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b67a4eccd15..b741cd7f644 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2251,6 +2251,7 @@ void Server::createServers( Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; http_params->setTimeout(settings.http_receive_timeout); http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); + http_params->setMaxKeepAliveRequests(static_cast(global_context->getServerSettings().max_keep_alive_requests)); Poco::Util::AbstractConfiguration::Keys protocols; config.keys("protocols", protocols); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index da82cdea5a4..7480d94e81d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -113,6 +113,7 @@ namespace DB M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ + M(UInt64, max_keep_alive_requests, 10000, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \ diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 09f7724d613..2b3f7f062bc 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -33,14 +34,20 @@ namespace ErrorCodes extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; } -void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout) +void setResponseDefaultHeaders(HTTPServerResponse & response) { if (!response.getKeepAlive()) return; - Poco::Timespan timeout(keep_alive_timeout, 0); - if (timeout.totalSeconds()) - response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); + const size_t keep_alive_timeout = response.getSession().getKeepAliveTimeout(); + const size_t keep_alive_max_requests = response.getSession().getMaxKeepAliveRequests(); + if (keep_alive_timeout) + { + if (keep_alive_max_requests) + response.set("Keep-Alive", fmt::format("timeout={}, max={}", keep_alive_timeout, keep_alive_max_requests)); + else + response.set("Keep-Alive", fmt::format("timeout={}", keep_alive_timeout)); + } } HTTPSessionPtr makeHTTPSession( diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 63dffcf6878..fa6086224f5 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -54,7 +54,7 @@ private: using HTTPSessionPtr = std::shared_ptr; -void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); +void setResponseDefaultHeaders(HTTPServerResponse & response); /// Create session object to perform requests and set required parameters. HTTPSessionPtr makeHTTPSession( diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 90bdebf6451..9b8feae3e26 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -13,7 +13,8 @@ HTTPServer::HTTPServer( Poco::Net::HTTPServerParams::Ptr params, const ProfileEvents::Event & read_event, const ProfileEvents::Event & write_event) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params), factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params) + , factory(factory_) { } diff --git a/src/Server/HTTP/HTTPServerResponse.h b/src/Server/HTTP/HTTPServerResponse.h index 8edb785e7c5..9793fc8b24b 100644 --- a/src/Server/HTTP/HTTPServerResponse.h +++ b/src/Server/HTTP/HTTPServerResponse.h @@ -245,6 +245,8 @@ public: void attachRequest(HTTPServerRequest * request_) { request = request_; } + const Poco::Net::HTTPServerSession & getSession() const { return session; } + private: Poco::Net::HTTPServerSession & session; HTTPServerRequest * request = nullptr; diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 8098671a903..a39f6de51d0 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -30,7 +30,7 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() if (add_cors_header) response.set("Access-Control-Allow-Origin", "*"); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); std::stringstream header; //STYLE_CHECK_ALLOW_STD_STRING_STREAM response.beginWrite(header); @@ -119,12 +119,10 @@ void WriteBufferFromHTTPServerResponse::nextImpl() WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( HTTPServerResponse & response_, bool is_http_method_head_, - UInt64 keep_alive_timeout_, const ProfileEvents::Event & write_event_) : HTTPWriteBuffer(response_.getSocket(), write_event_) , response(response_) , is_http_method_head(is_http_method_head_) - , keep_alive_timeout(keep_alive_timeout_) { } diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index a3952b7c553..f0c80f24582 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -29,7 +29,6 @@ public: WriteBufferFromHTTPServerResponse( HTTPServerResponse & response_, bool is_http_method_head_, - UInt64 keep_alive_timeout_, const ProfileEvents::Event & write_event_ = ProfileEvents::end()); ~WriteBufferFromHTTPServerResponse() override; @@ -91,7 +90,6 @@ private: bool is_http_method_head; bool add_cors_header = false; - size_t keep_alive_timeout = 0; bool initialized = false; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c112eefec6c..ac6c9d6a0a5 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -621,7 +621,6 @@ void HTTPHandler::processQuery( std::make_shared( response, request.getMethod() == HTTPRequest::HTTP_HEAD, - context->getServerSettings().keep_alive_timeout.totalSeconds(), write_event); used_output.out = used_output.out_holder; used_output.out_maybe_compressed = used_output.out_holder; @@ -926,7 +925,7 @@ try if (!used_output.out_holder && !used_output.exception_is_written) { /// If nothing was sent yet and we don't even know if we must compress the response. - WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT).writeln(s); + WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD).writeln(s); } else if (used_output.out_maybe_compressed) { diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 28045380cd7..9a87992731c 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -87,9 +87,8 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe response.setChunkedTransferEncoding(true); Output used_output; - const auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); used_output.out = std::make_shared( - response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); + response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, write_event); auto finalize_output = [&] { diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index dff960f7031..0ad5f907467 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -18,21 +18,15 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe { try { - /// Raw config reference is used here to avoid dependency on Context and ServerSettings. - /// This is painful, because this class is also used in a build with CLICKHOUSE_KEEPER_STANDALONE_BUILD=1 - /// And there ordinary Context is replaced with a tiny clone. - const auto & config = server.config(); - unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); - /// In order to make keep-alive works. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); - WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); + WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, write_event); try { metrics_writer->write(wb); diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index d120752c8c5..cc7848d1dd0 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -12,15 +12,10 @@ class IServer; class PrometheusRequestHandler : public HTTPRequestHandler { private: - IServer & server; PrometheusMetricsWriterPtr metrics_writer; public: - PrometheusRequestHandler(IServer & server_, PrometheusMetricsWriterPtr metrics_writer_) - : server(server_) - , metrics_writer(std::move(metrics_writer_)) - { - } + PrometheusRequestHandler(IServer &, PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; }; diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index 91c6bd722d3..964e3834037 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -84,8 +84,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe } } - const auto & server_settings = getContext()->getServerSettings(); - setResponseDefaultHeaders(response, server_settings.keep_alive_timeout.totalSeconds()); + setResponseDefaultHeaders(response); if (!ok) { diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 67bf3875de4..3d618031875 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -33,10 +33,9 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } -static inline std::unique_ptr -responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response, UInt64 keep_alive_timeout) +static inline std::unique_ptr responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response) { - auto buf = std::unique_ptr(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout)); + auto buf = std::unique_ptr(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD)); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -89,8 +88,7 @@ static inline void trySendExceptionToClient( void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/) { - auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); - auto out = responseWriteBuffer(request, response, keep_alive_timeout); + auto out = responseWriteBuffer(request, response); try { @@ -105,7 +103,7 @@ void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServer "The Transfer-Encoding is not chunked and there " "is no Content-Length header for POST request"); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status)); writeResponse(*out); } diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index 68d3ff0b325..faad9d57519 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -29,18 +29,15 @@ DashboardWebUIRequestHandler::DashboardWebUIRequestHandler(IServer & server_) : BinaryWebUIRequestHandler::BinaryWebUIRequestHandler(IServer & server_) : server(server_) {} JavaScriptWebUIRequestHandler::JavaScriptWebUIRequestHandler(IServer & server_) : server(server_) {} -static void handle(const IServer & server, HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) +static void handle(const IServer &, HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) { - auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds(); - response.setContentType("text/html; charset=UTF-8"); if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response, keep_alive_timeout); + setResponseDefaultHeaders(response); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout).write(html.data(), html.size()); - + WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD).write(html.data(), html.size()); } void PlayWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) diff --git a/tests/queries/0_stateless/00408_http_keep_alive.reference b/tests/queries/0_stateless/00408_http_keep_alive.reference index 17a7fd690a8..d5d7dacce9e 100644 --- a/tests/queries/0_stateless/00408_http_keep_alive.reference +++ b/tests/queries/0_stateless/00408_http_keep_alive.reference @@ -1,6 +1,6 @@ < Connection: Keep-Alive -< Keep-Alive: timeout=10 +< Keep-Alive: timeout=10, max=10000 < Connection: Keep-Alive -< Keep-Alive: timeout=10 +< Keep-Alive: timeout=10, max=10000 < Connection: Keep-Alive -< Keep-Alive: timeout=10 +< Keep-Alive: timeout=10, max=10000 diff --git a/tests/queries/0_stateless/00501_http_head.reference b/tests/queries/0_stateless/00501_http_head.reference index 8351327b356..db82132b145 100644 --- a/tests/queries/0_stateless/00501_http_head.reference +++ b/tests/queries/0_stateless/00501_http_head.reference @@ -2,11 +2,11 @@ HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10 +Keep-Alive: timeout=10, max=10000 HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10 +Keep-Alive: timeout=10, max=10000 From 7aace4d876173ce18ade57ec1bdc332efff7ce80 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 17:24:25 +0000 Subject: [PATCH 0046/1722] add test --- .../test_server_keep_alive/__init__.py | 0 .../configs/keep_alive_settings.xml | 4 ++ .../test_server_keep_alive/test.py | 46 +++++++++++++++++++ 3 files changed, 50 insertions(+) create mode 100644 tests/integration/test_server_keep_alive/__init__.py create mode 100644 tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml create mode 100644 tests/integration/test_server_keep_alive/test.py diff --git a/tests/integration/test_server_keep_alive/__init__.py b/tests/integration/test_server_keep_alive/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml b/tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml new file mode 100644 index 00000000000..06e68044817 --- /dev/null +++ b/tests/integration/test_server_keep_alive/configs/keep_alive_settings.xml @@ -0,0 +1,4 @@ + + 3600 + 5 + diff --git a/tests/integration/test_server_keep_alive/test.py b/tests/integration/test_server_keep_alive/test.py new file mode 100644 index 00000000000..0f88fe47673 --- /dev/null +++ b/tests/integration/test_server_keep_alive/test.py @@ -0,0 +1,46 @@ +import logging +import pytest +import requests + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/keep_alive_settings.xml"]) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_requests_with_keep_alive(start_cluster): + # In this test we have `keep_alive_timeout` set to one hour to never trigger connection reset by timeout, `max_keep_alive_requests` is set to 5. + # We expect server to close connection after each 5 requests. We detect connection reset by change in src port. + # So the first 5 requests should come from the same port, the following 5 requests should come from another port. + session = requests.Session() + for i in range(10): + session.get( + f"http://{node.ip_address}:8123/?query=select%201&log_comment=test_requests_with_keep_alive_{i}" + ) + + ports = node.query( + """ + SYSTEM FLUSH LOGS; + + SELECT port + FROM system.query_log + WHERE log_comment like 'test_requests_with_keep_alive_%' AND type = 'QueryFinish' + ORDER BY log_comment + """ + ).split("\n")[:-1] + + expected = 5 * [ports[0]] + [ports[5]] * 5 + + assert ports == expected From 146d7603388ca161ee3340ab1f582971a4e45a03 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 17:38:06 +0000 Subject: [PATCH 0047/1722] rm more --- programs/keeper/Keeper.cpp | 2 +- src/IO/HTTPCommon.cpp | 1 - src/Server/HTTP/HTTPServer.cpp | 3 +-- src/Server/HTTPHandlerFactory.cpp | 10 ++++------ src/Server/HTTPHandlerFactory.h | 8 ++------ src/Server/PrometheusRequestHandler.cpp | 13 +++---------- src/Server/PrometheusRequestHandler.h | 2 +- src/Server/WebUIRequestHandler.cpp | 14 +++++++------- 8 files changed, 19 insertions(+), 34 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index a558ed64bf9..238964fb25e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -507,7 +507,7 @@ try "Prometheus: http://" + address.toString(), std::make_unique( std::move(my_http_context), - createPrometheusMainHandlerFactory(*this, config_getter(), metrics_writer, "PrometheusHandler-factory"), + createPrometheusMainHandlerFactory(config_getter(), metrics_writer, "PrometheusHandler-factory"), server_pool, socket, http_params)); diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 2b3f7f062bc..56226941228 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 9b8feae3e26..90bdebf6451 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -13,8 +13,7 @@ HTTPServer::HTTPServer( Poco::Net::HTTPServerParams::Ptr params, const ProfileEvents::Event & read_event, const ProfileEvents::Event & write_event) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params) - , factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_, read_event, write_event), thread_pool, socket_, params), factory(factory_) { } diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 9a67e576345..23d4c081d2d 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -123,7 +123,7 @@ static inline auto createHandlersFactoryFromConfig( } else if (handler_type == "prometheus") { - main_handler_factory->addHandler(createPrometheusHandlerFactory(server, config, async_metrics, prefix + "." + key)); + main_handler_factory->addHandler(createPrometheusHandlerFactory(config, async_metrics, prefix + "." + key)); } else if (handler_type == "replicas_status") { @@ -202,7 +202,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "PrometheusHandler-factory") { auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); + return createPrometheusMainHandlerFactory(config, metrics_writer, name); } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); @@ -294,10 +294,8 @@ void addDefaultHandlersFactory( if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) { auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; + auto creator + = [writer]() -> std::unique_ptr { return std::make_unique(writer); }; auto prometheus_handler = std::make_shared>(std::move(creator)); prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); prometheus_handler->allowGetAndHeadRequest(); diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index ac18c36e6c9..5c1a12d9e06 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -126,14 +126,10 @@ HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); -HTTPRequestHandlerFactoryPtr -createPrometheusHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - AsynchronousMetrics & async_metrics, - const std::string & config_prefix); +HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( + const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & config_prefix); HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name); diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 0ad5f907467..1a04311116f 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -44,16 +44,12 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe } HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & config_prefix) { auto writer = std::make_shared(config, config_prefix + ".handler", async_metrics); - auto creator = [&server, writer]() -> std::unique_ptr - { - return std::make_unique(server, writer); - }; + auto creator = [writer]() -> std::unique_ptr { return std::make_unique(writer); }; auto factory = std::make_shared>(std::move(creator)); factory->addFiltersFromConfig(config, config_prefix); @@ -61,13 +57,10 @@ HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory( } HTTPRequestHandlerFactoryPtr createPrometheusMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name) + const Poco::Util::AbstractConfiguration & config, PrometheusMetricsWriterPtr metrics_writer, const std::string & name) { auto factory = std::make_shared(name); - auto creator = [&server, metrics_writer] - { - return std::make_unique(server, metrics_writer); - }; + auto creator = [metrics_writer] { return std::make_unique(metrics_writer); }; auto handler = std::make_shared>(std::move(creator)); handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index cc7848d1dd0..7f4d3c14f62 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -15,7 +15,7 @@ private: PrometheusMetricsWriterPtr metrics_writer; public: - PrometheusRequestHandler(IServer &, PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } + PrometheusRequestHandler(PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; }; diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index faad9d57519..e43412550f9 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -29,7 +29,7 @@ DashboardWebUIRequestHandler::DashboardWebUIRequestHandler(IServer & server_) : BinaryWebUIRequestHandler::BinaryWebUIRequestHandler(IServer & server_) : server(server_) {} JavaScriptWebUIRequestHandler::JavaScriptWebUIRequestHandler(IServer & server_) : server(server_) {} -static void handle(const IServer &, HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) +static void handle(HTTPServerRequest & request, HTTPServerResponse & response, std::string_view html) { response.setContentType("text/html; charset=UTF-8"); if (request.getVersion() == HTTPServerRequest::HTTP_1_1) @@ -42,7 +42,7 @@ static void handle(const IServer &, HTTPServerRequest & request, HTTPServerRespo void PlayWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) { - handle(server, request, response, {reinterpret_cast(gresource_play_htmlData), gresource_play_htmlSize}); + handle(request, response, {reinterpret_cast(gresource_play_htmlData), gresource_play_htmlSize}); } void DashboardWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) @@ -60,23 +60,23 @@ void DashboardWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HT static re2::RE2 lz_string_url = R"(https://[^\s"'`]+lz-string[^\s"'`]*\.js)"; RE2::Replace(&html, lz_string_url, "/js/lz-string.js"); - handle(server, request, response, html); + handle(request, response, html); } void BinaryWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) { - handle(server, request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); + handle(request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); } void JavaScriptWebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event &) { if (request.getURI() == "/js/uplot.js") { - handle(server, request, response, {reinterpret_cast(gresource_uplot_jsData), gresource_uplot_jsSize}); + handle(request, response, {reinterpret_cast(gresource_uplot_jsData), gresource_uplot_jsSize}); } else if (request.getURI() == "/js/lz-string.js") { - handle(server, request, response, {reinterpret_cast(gresource_lz_string_jsData), gresource_lz_string_jsSize}); + handle(request, response, {reinterpret_cast(gresource_lz_string_jsData), gresource_lz_string_jsSize}); } else { @@ -84,7 +84,7 @@ void JavaScriptWebUIRequestHandler::handleRequest(HTTPServerRequest & request, H *response.send() << "Not found.\n"; } - handle(server, request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); + handle(request, response, {reinterpret_cast(gresource_binary_htmlData), gresource_binary_htmlSize}); } } From bd04fc5346d83e8450fa98578e325923a609abda Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 17:41:53 +0000 Subject: [PATCH 0048/1722] rename test --- tests/integration/test_server_keep_alive/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_server_keep_alive/test.py b/tests/integration/test_server_keep_alive/test.py index 0f88fe47673..96f08a37adb 100644 --- a/tests/integration/test_server_keep_alive/test.py +++ b/tests/integration/test_server_keep_alive/test.py @@ -20,7 +20,7 @@ def start_cluster(): cluster.shutdown() -def test_requests_with_keep_alive(start_cluster): +def test_max_keep_alive_requests_on_user_side(start_cluster): # In this test we have `keep_alive_timeout` set to one hour to never trigger connection reset by timeout, `max_keep_alive_requests` is set to 5. # We expect server to close connection after each 5 requests. We detect connection reset by change in src port. # So the first 5 requests should come from the same port, the following 5 requests should come from another port. From c153fae0b8377aeca5e636a7ad0370e6ada42688 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 19:10:06 +0000 Subject: [PATCH 0049/1722] add docs --- .../server-configuration-parameters/settings.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 07c9a2b88ab..0efb5a9e6e4 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1298,6 +1298,16 @@ The number of seconds that ClickHouse waits for incoming requests before closing 10 ``` +## max_keep_alive_requests {#max-keep-alive-requests} + +Maximal number of requests through a single keep-alive connection until it will be closed by ClickHouse server. Default to 10000. + +**Example** + +``` xml +10 +``` + ## listen_host {#listen_host} Restriction on hosts that requests can come from. If you want the server to answer all of them, specify `::`. From b93f483a0e2f312d50685fd499d2f52717b83925 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 20:07:12 +0000 Subject: [PATCH 0050/1722] fix build --- programs/library-bridge/LibraryBridge.cpp | 2 +- .../LibraryBridgeHandlerFactory.cpp | 10 ++++------ .../LibraryBridgeHandlerFactory.h | 2 -- .../library-bridge/LibraryBridgeHandlers.cpp | 17 ++++++----------- programs/library-bridge/LibraryBridgeHandlers.h | 6 ++---- programs/odbc-bridge/ColumnInfoHandler.cpp | 5 +---- programs/odbc-bridge/ColumnInfoHandler.h | 8 +------- programs/odbc-bridge/IdentifierQuoteHandler.cpp | 2 +- programs/odbc-bridge/IdentifierQuoteHandler.h | 8 +------- programs/odbc-bridge/MainHandler.cpp | 2 +- programs/odbc-bridge/MainHandler.h | 3 --- programs/odbc-bridge/ODBCHandlerFactory.cpp | 10 +++++----- programs/odbc-bridge/SchemaAllowedHandler.cpp | 2 +- programs/odbc-bridge/SchemaAllowedHandler.h | 8 +------- 14 files changed, 25 insertions(+), 60 deletions(-) diff --git a/programs/library-bridge/LibraryBridge.cpp b/programs/library-bridge/LibraryBridge.cpp index 8a07ca57104..f86e469a307 100644 --- a/programs/library-bridge/LibraryBridge.cpp +++ b/programs/library-bridge/LibraryBridge.cpp @@ -25,7 +25,7 @@ std::string LibraryBridge::bridgeName() const LibraryBridge::HandlerFactoryPtr LibraryBridge::getHandlerFactoryPtr(ContextPtr context) const { - return std::make_shared("LibraryRequestHandlerFactory", keep_alive_timeout, context); + return std::make_shared("LibraryRequestHandlerFactory", context); } } diff --git a/programs/library-bridge/LibraryBridgeHandlerFactory.cpp b/programs/library-bridge/LibraryBridgeHandlerFactory.cpp index e5ab22f2d40..234904c6265 100644 --- a/programs/library-bridge/LibraryBridgeHandlerFactory.cpp +++ b/programs/library-bridge/LibraryBridgeHandlerFactory.cpp @@ -9,12 +9,10 @@ namespace DB { LibraryBridgeHandlerFactory::LibraryBridgeHandlerFactory( const std::string & name_, - size_t keep_alive_timeout_, ContextPtr context_) : WithContext(context_) , log(getLogger(name_)) , name(name_) - , keep_alive_timeout(keep_alive_timeout_) { } @@ -26,17 +24,17 @@ std::unique_ptr LibraryBridgeHandlerFactory::createRequestHa if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { if (uri.getPath() == "/extdict_ping") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); else if (uri.getPath() == "/catboost_ping") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); } if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { if (uri.getPath() == "/extdict_request") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); else if (uri.getPath() == "/catboost_request") - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); } return nullptr; diff --git a/programs/library-bridge/LibraryBridgeHandlerFactory.h b/programs/library-bridge/LibraryBridgeHandlerFactory.h index 5b0f088bc29..c65394efa3b 100644 --- a/programs/library-bridge/LibraryBridgeHandlerFactory.h +++ b/programs/library-bridge/LibraryBridgeHandlerFactory.h @@ -13,7 +13,6 @@ class LibraryBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContex public: LibraryBridgeHandlerFactory( const std::string & name_, - size_t keep_alive_timeout_, ContextPtr context_); std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; @@ -21,7 +20,6 @@ public: private: LoggerPtr log; const std::string name; - const size_t keep_alive_timeout; }; } diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index 094cef6716d..bd8faf76188 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -86,10 +86,8 @@ static void writeData(Block data, OutputFormatPtr format) } -ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler")) +ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler")) { } @@ -136,7 +134,7 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ const String & dictionary_id = params.get("dictionary_id"); LOG_TRACE(log, "Library method: '{}', dictionary id: {}", method, dictionary_id); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { @@ -410,11 +408,8 @@ void ExternalDictionaryLibraryBridgeExistsHandler::handleRequest(HTTPServerReque } -CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler( - size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , keep_alive_timeout(keep_alive_timeout_) - , log(getLogger("CatBoostLibraryBridgeRequestHandler")) +CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler(ContextPtr context_) + : WithContext(context_), log(getLogger("CatBoostLibraryBridgeRequestHandler")) { } @@ -453,7 +448,7 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ const String & method = params.get("method"); LOG_TRACE(log, "Library method: '{}'", method); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 83bca24ce1f..70e3c9c78da 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -18,14 +18,13 @@ namespace DB class ExternalDictionaryLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_); + ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: static constexpr inline auto FORMAT = "RowBinary"; - const size_t keep_alive_timeout; LoggerPtr log; }; @@ -62,12 +61,11 @@ private: class CatBoostLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_); + CatBoostLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: - const size_t keep_alive_timeout; LoggerPtr log; }; diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 4cb15de3b2c..438062e8169 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -200,10 +200,7 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (columns.empty()) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Columns definition was not returned"); - WriteBufferFromHTTPServerResponse out( - response, - request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, - keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { writeStringBinary(columns.toString(), out); diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index ca7044fdf32..f16e09ec3f9 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -16,18 +16,12 @@ namespace DB class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext { public: - ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger("ODBCColumnsInfoHandler")) - , keep_alive_timeout(keep_alive_timeout_) - { - } + ODBCColumnsInfoHandler(ContextPtr context_) : WithContext(context_), log(getLogger("ODBCColumnsInfoHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: LoggerPtr log; - size_t keep_alive_timeout; }; } diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/programs/odbc-bridge/IdentifierQuoteHandler.cpp index cf5acdc4534..0bd1e8758cd 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -73,7 +73,7 @@ void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServ auto identifier = getIdentifierQuote(std::move(connection)); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { writeStringBinary(identifier, out); diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index 7b78c5b4b93..c0e07795ea5 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -14,18 +14,12 @@ namespace DB class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext { public: - IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger("IdentifierQuoteHandler")) - , keep_alive_timeout(keep_alive_timeout_) - { - } + IdentifierQuoteHandler(ContextPtr context_) : WithContext(context_), log(getLogger("IdentifierQuoteHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: LoggerPtr log; - size_t keep_alive_timeout; }; } diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index e350afa2b10..b086397446e 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -131,7 +131,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse return; } - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { diff --git a/programs/odbc-bridge/MainHandler.h b/programs/odbc-bridge/MainHandler.h index ed0c6b2e28c..0fcad61d274 100644 --- a/programs/odbc-bridge/MainHandler.h +++ b/programs/odbc-bridge/MainHandler.h @@ -20,12 +20,10 @@ class ODBCHandler : public HTTPRequestHandler, WithContext { public: ODBCHandler( - size_t keep_alive_timeout_, ContextPtr context_, const String & mode_) : WithContext(context_) , log(getLogger("ODBCHandler")) - , keep_alive_timeout(keep_alive_timeout_) , mode(mode_) { } @@ -35,7 +33,6 @@ public: private: LoggerPtr log; - size_t keep_alive_timeout; String mode; static inline std::mutex mutex; diff --git a/programs/odbc-bridge/ODBCHandlerFactory.cpp b/programs/odbc-bridge/ODBCHandlerFactory.cpp index eebb0c24c7a..7f095666447 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.cpp +++ b/programs/odbc-bridge/ODBCHandlerFactory.cpp @@ -30,26 +30,26 @@ std::unique_ptr ODBCBridgeHandlerFactory::createRequestHandl if (uri.getPath() == "/columns_info") #if USE_ODBC - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); #else return nullptr; #endif else if (uri.getPath() == "/identifier_quote") #if USE_ODBC - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); #else return nullptr; #endif else if (uri.getPath() == "/schema_allowed") #if USE_ODBC - return std::make_unique(keep_alive_timeout, getContext()); + return std::make_unique(getContext()); #else return nullptr; #endif else if (uri.getPath() == "/write") - return std::make_unique(keep_alive_timeout, getContext(), "write"); + return std::make_unique(getContext(), "write"); else - return std::make_unique(keep_alive_timeout, getContext(), "read"); + return std::make_unique(getContext(), "read"); } return nullptr; } diff --git a/programs/odbc-bridge/SchemaAllowedHandler.cpp b/programs/odbc-bridge/SchemaAllowedHandler.cpp index c7025ca4311..5dc0cb3aa2b 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.cpp +++ b/programs/odbc-bridge/SchemaAllowedHandler.cpp @@ -86,7 +86,7 @@ void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServer bool result = isSchemaAllowed(std::move(connection)); - WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); + WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD); try { writeBoolText(result, out); diff --git a/programs/odbc-bridge/SchemaAllowedHandler.h b/programs/odbc-bridge/SchemaAllowedHandler.h index 8dc725dbb33..e73c0a2cb26 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.h +++ b/programs/odbc-bridge/SchemaAllowedHandler.h @@ -17,18 +17,12 @@ class Context; class SchemaAllowedHandler : public HTTPRequestHandler, WithContext { public: - SchemaAllowedHandler(size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger("SchemaAllowedHandler")) - , keep_alive_timeout(keep_alive_timeout_) - { - } + SchemaAllowedHandler(ContextPtr context_) : WithContext(context_), log(getLogger("SchemaAllowedHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; private: LoggerPtr log; - size_t keep_alive_timeout; }; } From c33511dcb9314da6b64b11cf21d231c9d3896dad Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Mar 2024 20:24:53 +0000 Subject: [PATCH 0051/1722] remove more --- programs/odbc-bridge/ODBCBridge.cpp | 2 +- programs/odbc-bridge/ODBCHandlerFactory.cpp | 9 +++------ programs/odbc-bridge/ODBCHandlerFactory.h | 3 +-- programs/odbc-bridge/PingHandler.h | 4 ---- 4 files changed, 5 insertions(+), 13 deletions(-) diff --git a/programs/odbc-bridge/ODBCBridge.cpp b/programs/odbc-bridge/ODBCBridge.cpp index e91cc3158df..2cde5bbf9f5 100644 --- a/programs/odbc-bridge/ODBCBridge.cpp +++ b/programs/odbc-bridge/ODBCBridge.cpp @@ -25,7 +25,7 @@ std::string ODBCBridge::bridgeName() const ODBCBridge::HandlerFactoryPtr ODBCBridge::getHandlerFactoryPtr(ContextPtr context) const { - return std::make_shared("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context); + return std::make_shared("ODBCRequestHandlerFactory-factory", context); } } diff --git a/programs/odbc-bridge/ODBCHandlerFactory.cpp b/programs/odbc-bridge/ODBCHandlerFactory.cpp index 7f095666447..b5d0be908f4 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.cpp +++ b/programs/odbc-bridge/ODBCHandlerFactory.cpp @@ -9,11 +9,8 @@ namespace DB { -ODBCBridgeHandlerFactory::ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_) - : WithContext(context_) - , log(getLogger(name_)) - , name(name_) - , keep_alive_timeout(keep_alive_timeout_) +ODBCBridgeHandlerFactory::ODBCBridgeHandlerFactory(const std::string & name_, ContextPtr context_) + : WithContext(context_), log(getLogger(name_)), name(name_) { } @@ -23,7 +20,7 @@ std::unique_ptr ODBCBridgeHandlerFactory::createRequestHandl LOG_TRACE(log, "Request URI: {}", uri.toString()); if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - return std::make_unique(keep_alive_timeout); + return std::make_unique(); if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { diff --git a/programs/odbc-bridge/ODBCHandlerFactory.h b/programs/odbc-bridge/ODBCHandlerFactory.h index 4aaf1b55453..f4a2717dc9f 100644 --- a/programs/odbc-bridge/ODBCHandlerFactory.h +++ b/programs/odbc-bridge/ODBCHandlerFactory.h @@ -17,14 +17,13 @@ namespace DB class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext { public: - ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_); + ODBCBridgeHandlerFactory(const std::string & name_, ContextPtr context_); std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: LoggerPtr log; std::string name; - size_t keep_alive_timeout; }; } diff --git a/programs/odbc-bridge/PingHandler.h b/programs/odbc-bridge/PingHandler.h index c5447107e0c..4c557bd3cf6 100644 --- a/programs/odbc-bridge/PingHandler.h +++ b/programs/odbc-bridge/PingHandler.h @@ -9,11 +9,7 @@ namespace DB class PingHandler : public HTTPRequestHandler { public: - explicit PingHandler(size_t keep_alive_timeout_) : keep_alive_timeout(keep_alive_timeout_) {} void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; - -private: - size_t keep_alive_timeout; }; } From 1115fa4bc74d840e8fc3230908310cb7311dd0d0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Mar 2024 15:13:13 +0000 Subject: [PATCH 0052/1722] fix tidy --- src/Server/PrometheusRequestHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 7f4d3c14f62..a1bd18b394a 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -15,7 +15,7 @@ private: PrometheusMetricsWriterPtr metrics_writer; public: - PrometheusRequestHandler(PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } + explicit PrometheusRequestHandler(PrometheusMetricsWriterPtr metrics_writer_) : metrics_writer(std::move(metrics_writer_)) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; }; From 1cfbc548bb415e89e294a22da0eea59302269c37 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Mar 2024 17:28:51 +0100 Subject: [PATCH 0053/1722] Fix copy-paste Co-authored-by: Michael Lex --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 7480d94e81d..4a22082cdda 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -113,7 +113,7 @@ namespace DB M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ - M(UInt64, max_keep_alive_requests, 10000, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ + M(UInt64, max_keep_alive_requests, 10000, "The maximum number of requests handled via a single http keepalive connection before the server closes this connection.", 0) \ M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \ M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \ From 3c2915934f31d82176b65e1e998eca030671d872 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 26 Mar 2024 04:29:34 +0000 Subject: [PATCH 0054/1722] update fuzzers --- ..._function_state_deserialization_fuzzer.cpp | 23 +++++++++++++++++++ src/Client/ClientBase.cpp | 2 +- src/Core/fuzzers/names_and_types_fuzzer.cpp | 22 ++++++++++++++++++ .../data_type_deserialization_fuzzer.cpp | 22 ++++++++++++++++++ src/Formats/fuzzers/format_fuzzer.cpp | 20 ++++++++++++++++ .../fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- .../codegen_fuzzer/codegen_select_fuzzer.cpp | 2 +- src/Parsers/fuzzers/create_parser_fuzzer.cpp | 2 +- .../fuzzers/columns_description_fuzzer.cpp | 22 ++++++++++++++++++ 9 files changed, 113 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 425364efb9c..9d490432c60 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -12,10 +12,33 @@ #include +#include + #include #include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 767a9b2b9f9..bdee2233b27 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2729,7 +2729,7 @@ void ClientBase::runLibFuzzer() for (auto & arg : fuzzer_args_holder) fuzzer_args.emplace_back(arg.data()); - int fuzzer_argc = fuzzer_args.size(); + int fuzzer_argc = static_cast(fuzzer_args.size()); char ** fuzzer_argv = fuzzer_args.data(); LLVMFuzzerRunDriver(&fuzzer_argc, &fuzzer_argv, [](const uint8_t * data, size_t size) diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index 6fdd8703014..bc8cb7af61f 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -1,7 +1,29 @@ +#include +#include #include #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 0ae325871fb..f1b03147929 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -8,11 +8,33 @@ #include #include +#include + #include #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 46661e4828c..4426301b6e7 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -21,6 +21,26 @@ #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 20fd951d390..74fdcff79f7 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -39,7 +39,7 @@ set(CMAKE_INCLUDE_CURRENT_DIR TRUE) clickhouse_add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) -set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") +set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier -Wno-extra-semi-stmt -Wno-used-but-marked-unused") # contrib/libprotobuf-mutator/src/libfuzzer/libfuzzer_macro.h:143:44: error: no newline at end of file [-Werror,-Wnewline-eof] target_compile_options (codegen_select_fuzzer PRIVATE -Wno-newline-eof) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 9310d7d59f7..55daa370651 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -27,7 +27,7 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) DB::ParserQueryWithOutput parser(input.data() + input.size()); try { - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, 0); DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); diff --git a/src/Parsers/fuzzers/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp index 854885ad33b..1d5c3e27232 100644 --- a/src/Parsers/fuzzers/create_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/create_parser_fuzzer.cpp @@ -14,7 +14,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::string input = std::string(reinterpret_cast(data), size); DB::ParserCreateQuery parser; - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, 0); const UInt64 max_ast_depth = 1000; ast->checkDepth(max_ast_depth); diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index b703a1e7051..cb0c6168225 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -1,4 +1,26 @@ +#include #include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + +FunctionBasePtr createFunctionBaseCast( + ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); +} + +} extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) From cb40fd7d0c9096a8df8d5e7c2e9924f66d51e061 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 26 Mar 2024 15:27:01 +0000 Subject: [PATCH 0055/1722] minor fixes --- .../fuzzers/aggregate_function_state_deserialization_fuzzer.cpp | 2 +- src/Core/fuzzers/names_and_types_fuzzer.cpp | 2 +- src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp | 2 +- src/Formats/fuzzers/format_fuzzer.cpp | 2 +- src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 2 +- src/Parsers/fuzzers/create_parser_fuzzer.cpp | 2 +- src/Storages/fuzzers/columns_description_fuzzer.cpp | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 9d490432c60..a956d9906bc 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -33,7 +33,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for aggregate_function_state_deserialization_fuzzer"); } } diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index bc8cb7af61f..74debedf2a3 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -18,7 +18,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for names_and_types_fuzzer"); } } diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index f1b03147929..7d9a0513d18 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -29,7 +29,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for data_type_deserialization_fuzzer"); } } diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 4426301b6e7..2c1ec65e54d 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -35,7 +35,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for format_fuzzer"); } } diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 55daa370651..6b25b581532 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -27,7 +27,7 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) DB::ParserQueryWithOutput parser(input.data() + input.size()); try { - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0, DB::DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); diff --git a/src/Parsers/fuzzers/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp index 1d5c3e27232..bab8db5671d 100644 --- a/src/Parsers/fuzzers/create_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/create_parser_fuzzer.cpp @@ -14,7 +14,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) std::string input = std::string(reinterpret_cast(data), size); DB::ParserCreateQuery parser; - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000, DB::DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); const UInt64 max_ast_depth = 1000; ast->checkDepth(max_ast_depth); diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index cb0c6168225..ac285ea50f7 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -17,7 +17,7 @@ using FunctionBasePtr = std::shared_ptr; FunctionBasePtr createFunctionBaseCast( ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for Library Bridge"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for columns_description_fuzzer"); } } From 64e6c6a2fcf2f7017ec3749ad05eed2daeeb4b42 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Mar 2024 22:31:40 +0000 Subject: [PATCH 0056/1722] fix tidy --- programs/library-bridge/LibraryBridgeHandlers.h | 8 ++++---- programs/odbc-bridge/ColumnInfoHandler.h | 2 +- programs/odbc-bridge/IdentifierQuoteHandler.h | 2 +- programs/odbc-bridge/SchemaAllowedHandler.h | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/programs/library-bridge/LibraryBridgeHandlers.h b/programs/library-bridge/LibraryBridgeHandlers.h index 70e3c9c78da..582619e174e 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.h +++ b/programs/library-bridge/LibraryBridgeHandlers.h @@ -18,7 +18,7 @@ namespace DB class ExternalDictionaryLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_); + explicit ExternalDictionaryLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -33,7 +33,7 @@ private: class ExternalDictionaryLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_); + explicit ExternalDictionaryLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -61,7 +61,7 @@ private: class CatBoostLibraryBridgeRequestHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeRequestHandler(ContextPtr context_); + explicit CatBoostLibraryBridgeRequestHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; @@ -74,7 +74,7 @@ private: class CatBoostLibraryBridgeExistsHandler : public HTTPRequestHandler, WithContext { public: - CatBoostLibraryBridgeExistsHandler(ContextPtr context_); + explicit CatBoostLibraryBridgeExistsHandler(ContextPtr context_); void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; diff --git a/programs/odbc-bridge/ColumnInfoHandler.h b/programs/odbc-bridge/ColumnInfoHandler.h index f16e09ec3f9..bbbf0da218b 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.h +++ b/programs/odbc-bridge/ColumnInfoHandler.h @@ -16,7 +16,7 @@ namespace DB class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext { public: - ODBCColumnsInfoHandler(ContextPtr context_) : WithContext(context_), log(getLogger("ODBCColumnsInfoHandler")) { } + explicit ODBCColumnsInfoHandler(ContextPtr context_) : WithContext(context_), log(getLogger("ODBCColumnsInfoHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.h b/programs/odbc-bridge/IdentifierQuoteHandler.h index c0e07795ea5..a85b56a9f6a 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.h +++ b/programs/odbc-bridge/IdentifierQuoteHandler.h @@ -14,7 +14,7 @@ namespace DB class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext { public: - IdentifierQuoteHandler(ContextPtr context_) : WithContext(context_), log(getLogger("IdentifierQuoteHandler")) { } + explicit IdentifierQuoteHandler(ContextPtr context_) : WithContext(context_), log(getLogger("IdentifierQuoteHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; diff --git a/programs/odbc-bridge/SchemaAllowedHandler.h b/programs/odbc-bridge/SchemaAllowedHandler.h index e73c0a2cb26..59022151b53 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.h +++ b/programs/odbc-bridge/SchemaAllowedHandler.h @@ -17,7 +17,7 @@ class Context; class SchemaAllowedHandler : public HTTPRequestHandler, WithContext { public: - SchemaAllowedHandler(ContextPtr context_) : WithContext(context_), log(getLogger("SchemaAllowedHandler")) { } + explicit SchemaAllowedHandler(ContextPtr context_) : WithContext(context_), log(getLogger("SchemaAllowedHandler")) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override; From 3e2f41f3010ca7b68762518738a35dee0f84f8e0 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 18:35:44 +0300 Subject: [PATCH 0057/1722] support ATTACH PARTITION `ALL` FROM `TABLE` --- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 19 +- src/Storages/StorageReplicatedMergeTree.cpp | 451 +++++++++--------- ...626_replace_partition_from_table.reference | 11 +- .../00626_replace_partition_from_table.sql | 6 +- ...e_partition_from_table_zookeeper.reference | 1 + ..._replace_partition_from_table_zookeeper.sh | 9 + 7 files changed, 273 insertions(+), 228 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6cc8063d90a..7cdb18ce0b9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4876,7 +4876,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( const auto * partition_ast = command.partition->as(); if (partition_ast && partition_ast->all) { - if (command.type != PartitionCommand::DROP_PARTITION && command.type != PartitionCommand::ATTACH_PARTITION) + if (command.type != PartitionCommand::DROP_PARTITION && command.type != PartitionCommand::ATTACH_PARTITION && (command.type == PartitionCommand::REPLACE_PARTITION && command.replace)) throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); } else @@ -5625,7 +5625,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc const auto & partition_ast = ast->as(); if (partition_ast.all) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only Support DETACH PARTITION ALL currently"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only Support DROP/DETACH/ATTACH PARTITION ALL currently"); if (!partition_ast.value) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 788f250e6a9..821b02b5b5f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2078,9 +2078,21 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, local_context); + DataPartsVector src_parts; + String partition_id; + bool is_all = partition->as()->all; + if (is_all) + { + if (replace) + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); + + src_parts = src_data.getVisibleDataPartsVector(local_context); + } else + { + partition_id = getPartitionIDFromQuery(partition, local_context); + src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); + } - DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); MutableDataPartsVector dst_parts; std::vector dst_parts_locks; @@ -2088,6 +2100,9 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con for (const DataPartPtr & src_part : src_parts) { + if (is_all) + partition_id = src_part->partition.getID(src_data); + if (!canReplacePartition(src_part)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot replace partition '{}' because part '{}' has inconsistent granularity with table", diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ce6735d9176..7d7ab712163 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7961,232 +7961,247 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); - /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. - DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); - - static const String TMP_PREFIX = "tmp_replace_from_"; + const String TMP_PREFIX = "tmp_replace_from_"; auto zookeeper = getZooKeeper(); - /// Retry if alter_partition_version changes - for (size_t retry = 0; retry < 1000; ++retry) + std::unordered_set partitions; + if (partition->as()->all) { - DataPartsVector src_parts; - MutableDataPartsVector dst_parts; - std::vector dst_parts_locks; - Strings block_id_paths; - Strings part_checksums; - std::vector ephemeral_locks; - String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; - Coordination::Stat alter_partition_version_stat; - zookeeper->get(alter_partition_version_path, &alter_partition_version_stat); - - /// Firstly, generate last block number and compute drop_range - /// NOTE: Even if we make ATTACH PARTITION instead of REPLACE PARTITION drop_range will not be empty, it will contain a block. - /// So, such case has special meaning, if drop_range contains only one block it means that nothing to drop. - /// TODO why not to add normal DROP_RANGE entry to replication queue if `replace` is true? - MergeTreePartInfo drop_range; - std::optional delimiting_block_lock; - bool partition_was_empty = !getFakePartCoveringAllPartsInPartition(partition_id, drop_range, delimiting_block_lock, true); - if (replace && partition_was_empty) - { - /// Nothing to drop, will just attach new parts - LOG_INFO(log, "Partition {} was empty, REPLACE PARTITION will work as ATTACH PARTITION FROM", drop_range.partition_id); - replace = false; - } - - if (!replace) - { - /// It's ATTACH PARTITION FROM, not REPLACE PARTITION. We have to reset drop range - drop_range = makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(partition_id); - } - - assert(replace == !LogEntry::ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range)); - - String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range); - - std::set replaced_parts; - for (const auto & src_part : src_all_parts) - { - /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION - /// Assume that merges in the partition are quite rare - /// Save deduplication block ids with special prefix replace_partition - - if (!canReplacePartition(src_part)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot replace partition '{}' because part '{}" - "' has inconsistent granularity with table", partition_id, src_part->name); - - String hash_hex = src_part->checksums.getTotalChecksumHex(); - const bool is_duplicated_part = replaced_parts.contains(hash_hex); - replaced_parts.insert(hash_hex); - - if (replace) - LOG_INFO(log, "Trying to replace {} with hash_hex {}", src_part->name, hash_hex); - else - LOG_INFO(log, "Trying to attach {} with hash_hex {}", src_part->name, hash_hex); - - String block_id_path = (replace || is_duplicated_part) ? "" : (fs::path(zookeeper_path) / "blocks" / (partition_id + "_replace_from_" + hash_hex)); - - auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); - if (!lock) - { - LOG_INFO(log, "Part {} (hash {}) has been already attached", src_part->name, hash_hex); - continue; - } - - UInt64 index = lock->getNumber(); - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication - || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; - - IDataPartStorage::ClonePartParams clone_params - { - .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), - .metadata_version_to_write = metadata_snapshot->getMetadataVersion() - }; - - auto [dst_part, part_lock] = cloneAndLoadDataPart( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - src_parts.emplace_back(src_part); - ephemeral_locks.emplace_back(std::move(*lock)); - block_id_paths.emplace_back(block_id_path); - part_checksums.emplace_back(hash_hex); - } - - ReplicatedMergeTreeLogEntryData entry; - { - auto src_table_id = src_data.getStorageID(); - entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; - entry.source_replica = replica_name; - entry.create_time = time(nullptr); - entry.replace_range_entry = std::make_shared(); - - auto & entry_replace = *entry.replace_range_entry; - entry_replace.drop_range_part_name = drop_range_fake_part_name; - entry_replace.from_database = src_table_id.database_name; - entry_replace.from_table = src_table_id.table_name; - for (const auto & part : src_parts) - entry_replace.src_part_names.emplace_back(part->name); - for (const auto & part : dst_parts) - entry_replace.new_part_names.emplace_back(part->name); - for (const String & checksum : part_checksums) - entry_replace.part_names_checksums.emplace_back(checksum); - entry_replace.columns_version = -1; - } - if (replace) - { - /// Cancel concurrent inserts in range - clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); - /// Remove deduplication block_ids of replacing parts - clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); - } + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); - Coordination::Responses op_results; - DataPartsVector parts_holder; - - try - { - Coordination::Requests ops; - for (size_t i = 0; i < dst_parts.size(); ++i) - { - getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); - ephemeral_locks[i].getUnlockOp(ops); - } - - if (auto txn = query_context->getZooKeeperMetadataTransaction()) - txn->moveOpsTo(ops); - - delimiting_block_lock->getUnlockOp(ops); - /// Check and update version to avoid race with DROP_RANGE - ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); - /// Just update version, because merges assignment relies on it - ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Transaction transaction(*this, NO_TRANSACTION_RAW); - { - auto data_parts_lock = lockParts(); - for (auto & part : dst_parts) - renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock); - } - - for (const auto & dst_part : dst_parts) - lockSharedData(*dst_part, false, /*hardlinked_files*/ {}); - - Coordination::Error code = zookeeper->tryMulti(ops, op_results); - if (code == Coordination::Error::ZOK) - delimiting_block_lock->assumeUnlocked(); - else if (code == Coordination::Error::ZBADVERSION) - { - /// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level. - if (query_context->getZooKeeperMetadataTransaction()) - throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, - "Cannot execute alter, because alter partition version was suddenly changed due " - "to concurrent alter"); - continue; - } - else - zkutil::KeeperMultiException::check(code, ops, op_results); - - { - auto data_parts_lock = lockParts(); - transaction.commit(&data_parts_lock); - if (replace) - { - parts_holder = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, drop_range.partition_id, &data_parts_lock); - /// We ignore the list of parts returned from the function below. We will remove them from zk when executing REPLACE_RANGE - removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); - } - } - - PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); - } - catch (...) - { - PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); - for (const auto & dst_part : dst_parts) - unlockSharedData(*dst_part); - - throw; - } - - String log_znode_path = dynamic_cast(*op_results.back()).path_created; - entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - for (auto & lock : ephemeral_locks) - lock.assumeUnlocked(); - - lock2.reset(); - lock1.reset(); - - /// We need to pull the DROP_RANGE before cleaning the replaced parts (otherwise CHeckThread may decide that parts are lost) - queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); - parts_holder.clear(); - cleanup_thread.wakeup(); - - - waitForLogEntryToBeProcessedIfNecessary(entry, query_context); - - return; + partitions = src_data.getAllPartitionIds(); + LOG_INFO(log, "Will try to attach {} partitions", partitions.size()); + } else + { + partitions = std::unordered_set(); + partitions.emplace(getPartitionIDFromQuery(partition, query_context)); } - throw Exception( - ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed"); + for (const auto & partition_id : partitions) { + auto src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); + LOG_DEBUG(log, "Cloning {} parts from partition '{}'", src_all_parts.size(), partition_id); + + auto ok = false; + /// Retry if alter_partition_version changes + for (size_t retry = 0; retry < 1000; ++retry) + { + DataPartsVector src_parts; + MutableDataPartsVector dst_parts; + std::vector dst_parts_locks; + Strings block_id_paths; + Strings part_checksums; + std::vector ephemeral_locks; + String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; + Coordination::Stat alter_partition_version_stat; + zookeeper->get(alter_partition_version_path, &alter_partition_version_stat); + + /// Firstly, generate last block number and compute drop_range + /// NOTE: Even if we make ATTACH PARTITION instead of REPLACE PARTITION drop_range will not be empty, it will contain a block. + /// So, such case has special meaning, if drop_range contains only one block it means that nothing to drop. + /// TODO why not to add normal DROP_RANGE entry to replication queue if `replace` is true? + MergeTreePartInfo drop_range; + std::optional delimiting_block_lock; + bool partition_was_empty = !getFakePartCoveringAllPartsInPartition(partition_id, drop_range, delimiting_block_lock, true); + if (replace && partition_was_empty) + { + /// Nothing to drop, will just attach new parts + LOG_INFO(log, "Partition {} was empty, REPLACE PARTITION will work as ATTACH PARTITION FROM", drop_range.partition_id); + replace = false; + } + + if (!replace) + { + /// It's ATTACH PARTITION FROM, not REPLACE PARTITION. We have to reset drop range + drop_range = makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(partition_id); + } + + assert(replace == !LogEntry::ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range)); + + String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range); + + std::set replaced_parts; + for (const auto & src_part : src_all_parts) + { + /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION + /// Assume that merges in the partition are quite rare + /// Save deduplication block ids with special prefix replace_partition + + if (!canReplacePartition(src_part)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot replace partition '{}' because part '{}" + "' has inconsistent granularity with table", partition_id, src_part->name); + + String hash_hex = src_part->checksums.getTotalChecksumHex(); + const bool is_duplicated_part = replaced_parts.contains(hash_hex); + replaced_parts.insert(hash_hex); + + if (replace) + LOG_INFO(log, "Trying to replace '{}' with hash_hex '{}'", src_part->name, hash_hex); + else + LOG_INFO(log, "Trying to attach '{}' with hash_hex '{}'", src_part->name, hash_hex); + + String block_id_path = (replace || is_duplicated_part) ? "" : (fs::path(zookeeper_path) / "blocks" / (partition_id + "_replace_from_" + hash_hex)); + + auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); + if (!lock) + { + LOG_INFO(log, "Part '{}' (hash '{}') in partition '{}' has been already attached", src_part->name, hash_hex, partition_id); + continue; + } + + UInt64 index = lock->getNumber(); + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + + IDataPartStorage::ClonePartParams clone_params + { + .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), + .metadata_version_to_write = metadata_snapshot->getMetadataVersion() + }; + + auto [dst_part, part_lock] = cloneAndLoadDataPart( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + src_parts.emplace_back(src_part); + ephemeral_locks.emplace_back(std::move(*lock)); + block_id_paths.emplace_back(block_id_path); + part_checksums.emplace_back(hash_hex); + } + + ReplicatedMergeTreeLogEntryData entry; + { + auto src_table_id = src_data.getStorageID(); + entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; + entry.source_replica = replica_name; + entry.create_time = time(nullptr); + entry.replace_range_entry = std::make_shared(); + + auto & entry_replace = *entry.replace_range_entry; + entry_replace.drop_range_part_name = drop_range_fake_part_name; + entry_replace.from_database = src_table_id.database_name; + entry_replace.from_table = src_table_id.table_name; + for (const auto & part : src_parts) + entry_replace.src_part_names.emplace_back(part->name); + for (const auto & part : dst_parts) + entry_replace.new_part_names.emplace_back(part->name); + for (const String & checksum : part_checksums) + entry_replace.part_names_checksums.emplace_back(checksum); + entry_replace.columns_version = -1; + } + + if (replace) + { + /// Cancel concurrent inserts in range + clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); + /// Remove deduplication block_ids of replacing parts + clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); + } + + Coordination::Responses op_results; + DataPartsVector parts_holder; + + try + { + Coordination::Requests ops; + for (size_t i = 0; i < dst_parts.size(); ++i) + { + getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); + ephemeral_locks[i].getUnlockOp(ops); + } + + if (auto txn = query_context->getZooKeeperMetadataTransaction()) + txn->moveOpsTo(ops); + + delimiting_block_lock->getUnlockOp(ops); + /// Check and update version to avoid race with DROP_RANGE + ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); + /// Just update version, because merges assignment relies on it + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Transaction transaction(*this, NO_TRANSACTION_RAW); + { + auto data_parts_lock = lockParts(); + for (auto & part : dst_parts) + renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock); + } + + for (const auto & dst_part : dst_parts) + lockSharedData(*dst_part, false, /*hardlinked_files*/ {}); + + Coordination::Error code = zookeeper->tryMulti(ops, op_results); + if (code == Coordination::Error::ZOK) + delimiting_block_lock->assumeUnlocked(); + else if (code == Coordination::Error::ZBADVERSION) + { + /// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level. + if (query_context->getZooKeeperMetadataTransaction()) + throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, + "Cannot execute alter on partition '{}', because alter partition version was suddenly changed due " + "to concurrent alter", partition_id); + continue; + } + else + zkutil::KeeperMultiException::check(code, ops, op_results); + + { + auto data_parts_lock = lockParts(); + transaction.commit(&data_parts_lock); + if (replace) + { + parts_holder = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, drop_range.partition_id, &data_parts_lock); + /// We ignore the list of parts returned from the function below. We will remove them from zk when executing REPLACE_RANGE + removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); + } + } + + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); + } + catch (...) + { + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); + for (const auto & dst_part : dst_parts) + unlockSharedData(*dst_part); + + throw; + } + + String log_znode_path = dynamic_cast(*op_results.back()).path_created; + entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + for (auto & lock : ephemeral_locks) + lock.assumeUnlocked(); + + /// We need to pull the DROP_RANGE before cleaning the replaced parts (otherwise CHeckThread may decide that parts are lost) + queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); + parts_holder.clear(); + cleanup_thread.wakeup(); + + waitForLogEntryToBeProcessedIfNecessary(entry, query_context); + + ok = true; + break; + } + + if (!ok) + throw Exception( + ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION '{}', because another ALTER PARTITION query was concurrently executed", partition_id); + } + + lock2.reset(); + lock1.reset(); } void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table.reference b/tests/queries/0_stateless/00626_replace_partition_from_table.reference index 611f3a93ced..0f8ded245d0 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table.reference +++ b/tests/queries/0_stateless/00626_replace_partition_from_table.reference @@ -10,11 +10,12 @@ REPLACE recursive 4 8 1 ATTACH FROM -5 8 +6 8 +10 12 OPTIMIZE -5 8 5 -5 8 3 +10 12 9 +10 12 5 After restart -5 8 +10 12 DETACH+ATTACH PARTITION -3 4 +7 7 diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table.sql b/tests/queries/0_stateless/00626_replace_partition_from_table.sql index 7224224334e..3f712f48c06 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table.sql +++ b/tests/queries/0_stateless/00626_replace_partition_from_table.sql @@ -53,12 +53,16 @@ DROP TABLE src; CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k; INSERT INTO src VALUES (1, '0', 1); INSERT INTO src VALUES (1, '1', 1); +INSERT INTO src VALUES (2, '2', 1); +INSERT INTO src VALUES (3, '3', 1); SYSTEM STOP MERGES dst; -INSERT INTO dst VALUES (1, '1', 2); +INSERT INTO dst VALUES (1, '1', 2), (1, '2', 0); ALTER TABLE dst ATTACH PARTITION 1 FROM src; SELECT count(), sum(d) FROM dst; +ALTER TABLE dst ATTACH PARTITION ALL FROM src; +SELECT count(), sum(d) FROM dst; SELECT 'OPTIMIZE'; SELECT count(), sum(d), uniqExact(_part) FROM dst; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference index c6208941ac6..6a7c3478f86 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference @@ -16,6 +16,7 @@ REPLACE recursive ATTACH FROM 5 8 5 8 +7 12 REPLACE with fetch 4 6 4 6 diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index ffbf4df4ba7..49976df83b7 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -82,6 +82,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE src;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (3, '1', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (4, '1', 2);" $CLICKHOUSE_CLIENT --query="INSERT INTO dst_r2 VALUES (1, '1', 2);" query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION 1 FROM src;" @@ -90,6 +92,13 @@ $CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" +query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION ALL FROM src;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" +query_with_retry "ALTER TABLE dst_r2 DROP PARTITION 3;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +query_with_retry "ALTER TABLE dst_r2 DROP PARTITION 4;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch';" $CLICKHOUSE_CLIENT --query="DROP TABLE src;" From 0fbf612e150d83d9d0dfb7b157ea4cef6127c86a Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 18:41:24 +0300 Subject: [PATCH 0058/1722] support ATTACH PARTITION `ALL` FROM `TABLE` --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7d7ab712163..5434a8bd63e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7962,7 +7962,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - const String TMP_PREFIX = "tmp_replace_from_"; + static const String TMP_PREFIX = "tmp_replace_from_"; auto zookeeper = getZooKeeper(); std::unordered_set partitions; From 9c5e094289df6f0c0063819b68ae96a23f032d5e Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 19:57:41 +0300 Subject: [PATCH 0059/1722] fix cs --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5434a8bd63e..b4ffd5eff0a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7979,7 +7979,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( partitions.emplace(getPartitionIDFromQuery(partition, query_context)); } - for (const auto & partition_id : partitions) { + for (const auto & partition_id : partitions) + { auto src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts from partition '{}'", src_all_parts.size(), partition_id); From 1d4aa10099fec19f2b60f9094a9f9c004ab06421 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Sun, 31 Mar 2024 00:09:40 +0300 Subject: [PATCH 0060/1722] fix cs --- src/Storages/StorageMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 821b02b5b5f..04115cf0ede 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2087,7 +2087,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); src_parts = src_data.getVisibleDataPartsVector(local_context); - } else + } + else { partition_id = getPartitionIDFromQuery(partition, local_context); src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); From 8357bc7b1b2d48e808b63cc0aa6fb7c7aa36e98b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 31 Mar 2024 23:33:35 +0000 Subject: [PATCH 0061/1722] fix build --- base/base/CMakeLists.txt | 2 +- cmake/sanitize.cmake | 2 +- programs/CMakeLists.txt | 2 +- src/CMakeLists.txt | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 27aa0bd6baf..7b1da9ab4ad 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -1,4 +1,4 @@ -add_compile_options($<$,$>:${COVERAGE_FLAGS}>) +add_compile_options("$<$,$>:${COVERAGE_FLAGS}>") if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 9d53b2004b4..227d96357b5 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -64,7 +64,7 @@ option(WITH_COVERAGE "Instrumentation for code coverage with default implementat if (WITH_COVERAGE) message (STATUS "Enabled instrumentation for code coverage") - set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping") + set(COVERAGE_FLAGS -fprofile-instr-generate -fcoverage-mapping) endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 0d91de2dad8..aa7781498c8 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -1,4 +1,4 @@ -add_compile_options($<$,$>:${COVERAGE_FLAGS}>) +add_compile_options("$<$,$>:${COVERAGE_FLAGS}>") if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 73aa409e995..bd603c9f15e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -1,4 +1,4 @@ -add_compile_options($<$,$>:${COVERAGE_FLAGS}>) +add_compile_options("$<$,$>:${COVERAGE_FLAGS}>") if (USE_INCLUDE_WHAT_YOU_USE) set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH}) From 99e25d762c2db3c544dd5590726fc039b1828d16 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Apr 2024 12:28:51 +0000 Subject: [PATCH 0062/1722] remove WITH_COVERAGE for fuzzers build --- docker/packager/packager | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 23fc26bc1a4..355149df38c 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -276,7 +276,6 @@ def parse_env_variables( elif package_type == "fuzzers": cmake_flags.append("-DENABLE_FUZZING=1") cmake_flags.append("-DENABLE_PROTOBUF=1") - cmake_flags.append("-DWITH_COVERAGE=1") # Reduce linking and building time by avoid *install/all dependencies cmake_flags.append("-DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON") From 8d667ad5a34d1ba3d9008a5a6308598483281b35 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Apr 2024 22:55:51 +0000 Subject: [PATCH 0063/1722] fix build.sh --- docker/packager/binary-builder/build.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary-builder/build.sh b/docker/packager/binary-builder/build.sh index 032aceb0af3..cbd14b1eac2 100755 --- a/docker/packager/binary-builder/build.sh +++ b/docker/packager/binary-builder/build.sh @@ -108,7 +108,8 @@ if [ -n "$MAKE_DEB" ]; then bash -x /build/packages/build fi -mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output +mv ./programs/clickhouse* /output ||: +mv ./programs/*_fuzzer /output ||: [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output [ -x ./programs/self-extracting/clickhouse-stripped ] && mv ./programs/self-extracting/clickhouse-stripped /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds From db3d923d4cae57254cadcef7f6997f3912d46515 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 3 Apr 2024 20:25:29 +0000 Subject: [PATCH 0064/1722] return WITH_COVERAGE, fix build --- cmake/sanitize.cmake | 3 ++- docker/packager/packager | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 227d96357b5..9f4fa7081c6 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -64,7 +64,8 @@ option(WITH_COVERAGE "Instrumentation for code coverage with default implementat if (WITH_COVERAGE) message (STATUS "Enabled instrumentation for code coverage") - set(COVERAGE_FLAGS -fprofile-instr-generate -fcoverage-mapping) + set (COVERAGE_FLAGS -fprofile-instr-generate -fcoverage-mapping) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) diff --git a/docker/packager/packager b/docker/packager/packager index 355149df38c..23fc26bc1a4 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -276,6 +276,7 @@ def parse_env_variables( elif package_type == "fuzzers": cmake_flags.append("-DENABLE_FUZZING=1") cmake_flags.append("-DENABLE_PROTOBUF=1") + cmake_flags.append("-DWITH_COVERAGE=1") # Reduce linking and building time by avoid *install/all dependencies cmake_flags.append("-DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON") From cc5456c649a4bd1f95321f4320fadfa382236d1b Mon Sep 17 00:00:00 2001 From: cangyin Date: Sat, 6 Apr 2024 19:38:37 +0000 Subject: [PATCH 0065/1722] Fix projection merge for Collapsing/Replacing/VersionedCollapsing MergeTree --- .../MergeTree/MergeProjectionPartsTask.cpp | 95 +++++++++++ .../MergeTree/MergeProjectionPartsTask.h | 84 ++++++++++ src/Storages/MergeTree/MergeTask.cpp | 157 +++++++++++++++--- src/Storages/MergeTree/MergeTask.h | 25 ++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 3 +- .../MergeTree/MergeTreeDataMergerMutator.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 139 +--------------- 7 files changed, 349 insertions(+), 156 deletions(-) create mode 100644 src/Storages/MergeTree/MergeProjectionPartsTask.cpp create mode 100644 src/Storages/MergeTree/MergeProjectionPartsTask.h diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp new file mode 100644 index 00000000000..bdc2ba8b9ca --- /dev/null +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp @@ -0,0 +1,95 @@ +#include + +#include +#include + +namespace DB +{ + +bool MergeProjectionPartsTask::executeStep() +{ + auto & current_level_parts = level_parts[current_level]; + auto & next_level_parts = level_parts[next_level]; + + MergeTreeData::MutableDataPartsVector selected_parts; + while (selected_parts.size() < max_parts_to_merge_in_one_level && !current_level_parts.empty()) + { + selected_parts.push_back(std::move(current_level_parts.back())); + current_level_parts.pop_back(); + } + + if (selected_parts.empty()) + { + if (next_level_parts.empty()) + { + LOG_WARNING(log, "There is no projection parts merged"); + + /// Task is finished + return false; + } + current_level = next_level; + ++next_level; + } + else if (selected_parts.size() == 1) + { + if (next_level_parts.empty()) + { + LOG_DEBUG(log, "Merged a projection part in level {}", current_level); + selected_parts[0]->renameTo(projection.name + ".proj", true); + selected_parts[0]->setName(projection.name); + selected_parts[0]->is_temp = false; + new_data_part->addProjectionPart(name, std::move(selected_parts[0])); + + /// Task is finished + return false; + } + else + { + LOG_DEBUG(log, "Forwarded part {} in level {} to next level", selected_parts[0]->name, current_level); + next_level_parts.push_back(std::move(selected_parts[0])); + } + } + else if (selected_parts.size() > 1) + { + // Generate a unique part name + ++block_num; + auto projection_future_part = std::make_shared(); + MergeTreeData::DataPartsVector const_selected_parts( + std::make_move_iterator(selected_parts.begin()), std::make_move_iterator(selected_parts.end())); + projection_future_part->assign(std::move(const_selected_parts)); + projection_future_part->name = fmt::format("{}_{}", projection.name, ++block_num); + projection_future_part->part_info = {"all", 0, 0, 0}; + + MergeTreeData::MergingParams projection_merging_params; + projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; + if (projection.type == ProjectionDescription::Type::Aggregate) + projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; + + LOG_DEBUG(log, "Merged {} parts in level {} to {}", selected_parts.size(), current_level, projection_future_part->name); + auto tmp_part_merge_task = mutator->mergePartsToTemporaryPart( + projection_future_part, + projection.metadata, + merge_entry, + std::make_unique((*merge_entry)->table_id, projection_future_part, context), + *table_lock_holder, + time_of_merge, + context, + space_reservation, + false, // TODO Do we need deduplicate for projections + {}, + false, // no cleanup + projection_merging_params, + NO_TRANSACTION_PTR, + /* need_prefix */ true, + new_data_part.get(), + ".tmp_proj"); + + next_level_parts.push_back(executeHere(tmp_part_merge_task)); + next_level_parts.back()->is_temp = true; + } + + /// Need execute again + return true; +} + +} diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.h b/src/Storages/MergeTree/MergeProjectionPartsTask.h new file mode 100644 index 00000000000..47cafe01151 --- /dev/null +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.h @@ -0,0 +1,84 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class MergeProjectionPartsTask : public IExecutableTask +{ +public: + + MergeProjectionPartsTask( + String name_, + MergeTreeData::MutableDataPartsVector && parts_, + const ProjectionDescription & projection_, + size_t & block_num_, + ContextPtr context_, + TableLockHolder * table_lock_holder_, + MergeTreeDataMergerMutator * mutator_, + MergeListEntry * merge_entry_, + time_t time_of_merge_, + MergeTreeData::MutableDataPartPtr new_data_part_, + ReservationSharedPtr space_reservation_) + : name(std::move(name_)) + , parts(std::move(parts_)) + , projection(projection_) + , block_num(block_num_) + , context(context_) + , table_lock_holder(table_lock_holder_) + , mutator(mutator_) + , merge_entry(merge_entry_) + , time_of_merge(time_of_merge_) + , new_data_part(new_data_part_) + , space_reservation(space_reservation_) + , log(getLogger("MergeProjectionPartsTask")) + { + LOG_DEBUG(log, "Selected {} projection_parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); + level_parts[current_level] = std::move(parts); + } + + void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + StorageID getStorageID() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + Priority getPriority() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + String getQueryId() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + + bool executeStep() override; + +private: + String name; + MergeTreeData::MutableDataPartsVector parts; + const ProjectionDescription & projection; + size_t & block_num; + + ContextPtr context; + TableLockHolder * table_lock_holder; + MergeTreeDataMergerMutator * mutator; + MergeListEntry * merge_entry; + time_t time_of_merge; + + MergeTreeData::MutableDataPartPtr new_data_part; + ReservationSharedPtr space_reservation; + + LoggerPtr log; + + std::map level_parts; + size_t current_level = 0; + size_t next_level = 1; + + /// TODO(nikitamikhaylov): make this constant a setting + static constexpr size_t max_parts_to_merge_in_one_level = 10; +}; + +} diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 34e17e40a74..4be1b003573 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -18,6 +18,8 @@ #include #include #include +#include +#include #include #include #include @@ -47,12 +49,12 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } - /// PK columns are sorted and merged, ordinary columns are gathered using info from merge step static void extractMergingAndGatheringColumns( const NamesAndTypesList & storage_columns, const ExpressionActionsPtr & sorting_key_expr, const IndicesDescription & indexes, + const std::vector & projections, const MergeTreeData::MergingParams & merging_params, NamesAndTypesList & gathering_columns, Names & gathering_column_names, NamesAndTypesList & merging_columns, Names & merging_column_names) @@ -65,6 +67,12 @@ static void extractMergingAndGatheringColumns( std::copy(index_columns_vec.cbegin(), index_columns_vec.cend(), std::inserter(key_columns, key_columns.end())); } + for (const auto & projection : projections) + { + Names projection_columns_vec = projection->getRequiredColumns(); + std::copy(projection_columns_vec.cbegin(), projection_columns_vec.cend(), + std::inserter(key_columns, key_columns.end())); + } /// Force sign column for Collapsing mode if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) @@ -203,10 +211,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() extendObjectColumns(global_ctx->storage_columns, object_columns, false); global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, std::move(object_columns)); + prepareProjectionsToMergeAndRebuild(); + extractMergingAndGatheringColumns( global_ctx->storage_columns, global_ctx->metadata_snapshot->getSortingKey().expression, global_ctx->metadata_snapshot->getSecondaryIndices(), + global_ctx->projections_to_rebuild, ctx->merging_params, global_ctx->gathering_columns, global_ctx->gathering_column_names, @@ -453,6 +464,65 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() } +void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Block & block) const +{ + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + const auto & projection = *global_ctx->projections_to_rebuild[i]; + auto projection_block = ctx->projection_squashes[i].add(projection.calculate(block, global_ctx->context)); + if (projection_block) + { + auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( + *global_ctx->data, ctx->log, projection_block, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + tmp_part.finalize(); + tmp_part.part->getDataPartStorage().commitTransaction(); + ctx->projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + } + } +} + + +void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPartsMerge() const +{ + auto && [name, parts] = *ctx->projection_parts_iterator; + const auto & projection = global_ctx->metadata_snapshot->projections.get(name); + + ctx->merge_projection_parts_task_ptr = std::make_unique + ( + name, + std::move(parts), + projection, + ctx->projection_block_num, + global_ctx->context, + global_ctx->holder, + global_ctx->mutator, + global_ctx->merge_entry, + global_ctx->time_of_merge, + global_ctx->new_data_part, + global_ctx->space_reservation + ); +} + + +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() +{ + /// In case if there are no projections we didn't construct a task + if (!ctx->merge_projection_parts_task_ptr) + return false; + + if (ctx->merge_projection_parts_task_ptr->executeStep()) + return true; + + ++ctx->projection_parts_iterator; + + if (ctx->projection_parts_iterator == std::make_move_iterator(ctx->projection_parts.end())) + return false; + + constructTaskForProjectionPartsMerge(); + + return true; +} + bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { Block block; @@ -462,6 +532,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const_cast(*global_ctx->to).write(block); + calculateProjections(block); + UInt64 result_rows = 0; UInt64 result_bytes = 0; global_ctx->merged_pipeline.tryGetResultRowsAndBytes(result_rows, result_bytes); @@ -484,6 +556,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() return true; } + // finalize projections + calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); + + ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); + if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) + constructTaskForProjectionPartsMerge(); + global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -732,24 +811,9 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c } - const auto & projections = global_ctx->metadata_snapshot->getProjections(); - - for (const auto & projection : projections) + for (const auto & projection : global_ctx->projections_to_merge) { - MergeTreeData::DataPartsVector projection_parts; - for (const auto & part : global_ctx->future_part->parts) - { - auto actual_projection_parts = part->getProjectionParts(); - auto it = actual_projection_parts.find(projection.name); - if (it != actual_projection_parts.end() && !it->second->is_broken) - projection_parts.push_back(it->second); - } - if (projection_parts.size() < global_ctx->future_part->parts.size()) - { - LOG_DEBUG(ctx->log, "Projection {} is not merged because some parts don't have it", projection.name); - continue; - } - + MergeTreeData::DataPartsVector projection_parts = global_ctx->projections_to_merge_parts[projection->name]; LOG_DEBUG( ctx->log, "Selected {} projection_parts from {} to {}", @@ -759,7 +823,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c auto projection_future_part = std::make_shared(); projection_future_part->assign(std::move(projection_parts)); - projection_future_part->name = projection.name; + projection_future_part->name = projection->name; // TODO (ab): path in future_part is only for merge process introspection, which is not available for merges of projection parts. // Let's comment this out to avoid code inconsistency and add it back after we implement projection merge introspection. // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; @@ -767,16 +831,17 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; - if (projection.type == ProjectionDescription::Type::Aggregate) + if (projection->type == ProjectionDescription::Type::Aggregate) projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; ctx->tasks_for_projections.emplace_back(std::make_shared( projection_future_part, - projection.metadata, + projection->metadata, global_ctx->merge_entry, std::make_unique((*global_ctx->merge_entry)->table_id, projection_future_part, global_ctx->context), global_ctx->time_of_merge, global_ctx->context, + *global_ctx->holder, global_ctx->space_reservation, global_ctx->deduplicate, global_ctx->deduplicate_by_columns, @@ -1136,6 +1201,56 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->merging_executor = std::make_unique(global_ctx->merged_pipeline); } +void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const +{ + // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. + const bool merge_may_reduce_rows = + global_ctx->deduplicate || + ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || + ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || + ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; + + const auto & projections = global_ctx->metadata_snapshot->getProjections(); + + for (const auto & projection : projections) + { + if (merge_may_reduce_rows) + { + global_ctx->projections_to_rebuild.push_back(&projection); + continue; + } + + MergeTreeData::DataPartsVector projection_parts; + for (const auto & part : global_ctx->future_part->parts) + { + auto it = part->getProjectionParts().find(projection.name); + if (it != part->getProjectionParts().end()) + projection_parts.push_back(it->second); + } + if (projection_parts.size() == global_ctx->future_part->parts.size()) + { + global_ctx->projections_to_merge.push_back(&projection); + global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); + } + else if (projection_parts.empty()) + { + LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); + } + else + { + LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); + global_ctx->projections_to_rebuild.push_back(&projection); + } + } + + const auto & settings = global_ctx->context->getSettingsRef(); + + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + ctx->projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + } +} + MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm() const { diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index f6268886b14..e373b6b33c5 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -64,6 +65,7 @@ public: std::unique_ptr projection_merge_list_element_, time_t time_of_merge_, ContextPtr context_, + TableLockHolder & holder, ReservationSharedPtr space_reservation_, bool deduplicate_, Names deduplicate_by_columns_, @@ -88,6 +90,7 @@ public: = global_ctx->projection_merge_list_element ? global_ctx->projection_merge_list_element.get() : (*global_ctx->merge_entry)->ptr(); global_ctx->time_of_merge = std::move(time_of_merge_); global_ctx->context = std::move(context_); + global_ctx->holder = &holder; global_ctx->space_reservation = std::move(space_reservation_); global_ctx->deduplicate = std::move(deduplicate_); global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_); @@ -142,6 +145,7 @@ private: /// Proper initialization is responsibility of the author struct GlobalRuntimeContext : public IStageRuntimeContext { + TableLockHolder * holder; MergeList::Entry * merge_entry{nullptr}; /// If not null, use this instead of the global MergeList::Entry. This is for merging projections. std::unique_ptr projection_merge_list_element; @@ -173,6 +177,10 @@ private: MergeAlgorithm chosen_merge_algorithm{MergeAlgorithm::Undecided}; size_t gathering_column_names_size{0}; + std::vector projections_to_rebuild{}; + std::vector projections_to_merge{}; + std::map projections_to_merge_parts{}; + std::unique_ptr horizontal_stage_progress{nullptr}; std::unique_ptr column_progress{nullptr}; @@ -219,6 +227,14 @@ private: std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; + // For projections to rebuild + using ProjectionNameToItsBlocks = std::map; + ProjectionNameToItsBlocks projection_parts; + std::move_iterator projection_parts_iterator; + std::vector projection_squashes; + size_t projection_block_num = 0; + ExecutableTaskPtr merge_projection_parts_task_ptr; + size_t initial_reservation{0}; bool read_with_direct_io{false}; @@ -247,16 +263,21 @@ private: bool executeImpl(); /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable - using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; + using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; const ExecuteAndFinalizeHorizontalPartSubtasks subtasks { &ExecuteAndFinalizeHorizontalPart::prepare, - &ExecuteAndFinalizeHorizontalPart::executeImpl + &ExecuteAndFinalizeHorizontalPart::executeImpl, + &ExecuteAndFinalizeHorizontalPart::executeMergeProjections }; ExecuteAndFinalizeHorizontalPartSubtasks::const_iterator subtasks_iterator = subtasks.begin(); + void prepareProjectionsToMergeAndRebuild() const; + void calculateProjections(const Block & block) const; + void constructTaskForProjectionPartsMerge() const; + bool executeMergeProjections(); MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream(); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 53d49b51e8f..765c2c5e428 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -670,7 +670,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const StorageMetadataPtr & metadata_snapshot, MergeList::Entry * merge_entry, std::unique_ptr projection_merge_list_element, - TableLockHolder, + TableLockHolder & holder, time_t time_of_merge, ContextPtr context, ReservationSharedPtr space_reservation, @@ -690,6 +690,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( std::move(projection_merge_list_element), time_of_merge, context, + holder, space_reservation, deduplicate, deduplicate_by_columns, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 669ee040af3..fc910f401b4 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -159,7 +159,7 @@ public: const StorageMetadataPtr & metadata_snapshot, MergeListEntry * merge_entry, std::unique_ptr projection_merge_list_element, - TableLockHolder table_lock_holder, + TableLockHolder & table_lock_holder, time_t time_of_merge, ContextPtr context, ReservationSharedPtr space_reservation, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a971c4fda1c..aaf4e723494 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -1030,136 +1031,6 @@ struct MutationContext using MutationContextPtr = std::shared_ptr; -class MergeProjectionPartsTask : public IExecutableTask -{ -public: - - MergeProjectionPartsTask( - String name_, - MergeTreeData::MutableDataPartsVector && parts_, - const ProjectionDescription & projection_, - size_t & block_num_, - MutationContextPtr ctx_) - : name(std::move(name_)) - , parts(std::move(parts_)) - , projection(projection_) - , block_num(block_num_) - , ctx(ctx_) - , log(getLogger("MergeProjectionPartsTask")) - { - LOG_DEBUG(log, "Selected {} projection_parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); - level_parts[current_level] = std::move(parts); - } - - void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - StorageID getStorageID() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - Priority getPriority() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - String getQueryId() const override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - - bool executeStep() override - { - auto & current_level_parts = level_parts[current_level]; - auto & next_level_parts = level_parts[next_level]; - - MergeTreeData::MutableDataPartsVector selected_parts; - while (selected_parts.size() < max_parts_to_merge_in_one_level && !current_level_parts.empty()) - { - selected_parts.push_back(std::move(current_level_parts.back())); - current_level_parts.pop_back(); - } - - if (selected_parts.empty()) - { - if (next_level_parts.empty()) - { - LOG_WARNING(log, "There is no projection parts merged"); - - /// Task is finished - return false; - } - current_level = next_level; - ++next_level; - } - else if (selected_parts.size() == 1) - { - if (next_level_parts.empty()) - { - LOG_DEBUG(log, "Merged a projection part in level {}", current_level); - selected_parts[0]->renameTo(projection.name + ".proj", true); - selected_parts[0]->setName(projection.name); - selected_parts[0]->is_temp = false; - ctx->new_data_part->addProjectionPart(name, std::move(selected_parts[0])); - - /// Task is finished - return false; - } - else - { - LOG_DEBUG(log, "Forwarded part {} in level {} to next level", selected_parts[0]->name, current_level); - next_level_parts.push_back(std::move(selected_parts[0])); - } - } - else if (selected_parts.size() > 1) - { - // Generate a unique part name - ++block_num; - auto projection_future_part = std::make_shared(); - MergeTreeData::DataPartsVector const_selected_parts( - std::make_move_iterator(selected_parts.begin()), std::make_move_iterator(selected_parts.end())); - projection_future_part->assign(std::move(const_selected_parts)); - projection_future_part->name = fmt::format("{}_{}", projection.name, ++block_num); - projection_future_part->part_info = {"all", 0, 0, 0}; - - MergeTreeData::MergingParams projection_merging_params; - projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; - if (projection.type == ProjectionDescription::Type::Aggregate) - projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; - - LOG_DEBUG(log, "Merged {} parts in level {} to {}", selected_parts.size(), current_level, projection_future_part->name); - auto tmp_part_merge_task = ctx->mutator->mergePartsToTemporaryPart( - projection_future_part, - projection.metadata, - ctx->mutate_entry, - std::make_unique((*ctx->mutate_entry)->table_id, projection_future_part, ctx->context), - *ctx->holder, - ctx->time_of_mutation, - ctx->context, - ctx->space_reservation, - false, // TODO Do we need deduplicate for projections - {}, - false, // no cleanup - projection_merging_params, - NO_TRANSACTION_PTR, - /* need_prefix */ true, - ctx->new_data_part.get(), - ".tmp_proj"); - - next_level_parts.push_back(executeHere(tmp_part_merge_task)); - next_level_parts.back()->is_temp = true; - } - - /// Need execute again - return true; - } - -private: - String name; - MergeTreeData::MutableDataPartsVector parts; - const ProjectionDescription & projection; - size_t & block_num; - MutationContextPtr ctx; - - LoggerPtr log; - - std::map level_parts; - size_t current_level = 0; - size_t next_level = 1; - - /// TODO(nikitamikhaylov): make this constant a setting - static constexpr size_t max_parts_to_merge_in_one_level = 10; -}; - - // This class is responsible for: // 1. get projection pipeline and a sink to write parts // 2. build an executor that can write block to the input stream (actually we can write through it to generate as many parts as possible) @@ -1356,7 +1227,13 @@ void PartMergerWriter::constructTaskForProjectionPartsMerge() std::move(parts), projection, block_num, - ctx + ctx->context, + ctx->holder, + ctx->mutator, + ctx->mutate_entry, + ctx->time_of_mutation, + ctx->new_data_part, + ctx->space_reservation ); } From 603a52caa06552c50e3b4d29989e177d2a0a3efa Mon Sep 17 00:00:00 2001 From: cangyin Date: Mon, 8 Apr 2024 12:34:59 +0800 Subject: [PATCH 0066/1722] Add tests --- .../02968_projection_merge.reference | 28 +++++ .../0_stateless/02968_projection_merge.sql | 112 ++++++++++++++++++ 2 files changed, 140 insertions(+) create mode 100644 tests/queries/0_stateless/02968_projection_merge.reference create mode 100644 tests/queries/0_stateless/02968_projection_merge.sql diff --git a/tests/queries/0_stateless/02968_projection_merge.reference b/tests/queries/0_stateless/02968_projection_merge.reference new file mode 100644 index 00000000000..40cb572c95a --- /dev/null +++ b/tests/queries/0_stateless/02968_projection_merge.reference @@ -0,0 +1,28 @@ +ReplacingMergeTree +0 2 +1 2 +2 2 +0 2 +1 2 +2 2 +CollapsingMergeTree +0 2 +1 2 +2 2 +0 2 +1 2 +2 2 +VersionedCollapsingMergeTree +0 2 +1 2 +2 2 +0 2 +1 2 +2 2 +DEDUPLICATE ON MergeTree +0 1 +1 1 +2 1 +0 1 +1 1 +2 1 diff --git a/tests/queries/0_stateless/02968_projection_merge.sql b/tests/queries/0_stateless/02968_projection_merge.sql new file mode 100644 index 00000000000..07d40e30c2f --- /dev/null +++ b/tests/queries/0_stateless/02968_projection_merge.sql @@ -0,0 +1,112 @@ +SELECT 'ReplacingMergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = ReplacingMergeTree +ORDER BY type; + +INSERT INTO tp SELECT number%3, 1 FROM numbers(3); +INSERT INTO tp SELECT number%3, 2 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + + +SELECT 'CollapsingMergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + `sign` Int8, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = CollapsingMergeTree(sign) +ORDER BY type; + +INSERT INTO tp SELECT number % 3, 1, 1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 1, -1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 2, 1 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +-- Actually we don't need to test all 3 engines Replacing/Collapsing/VersionedCollapsing, +-- Because they share the same logic of 'reduce number of rows during merges' +SELECT 'VersionedCollapsingMergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + `sign` Int8, + `version` UInt8, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = VersionedCollapsingMergeTree(sign,version) +ORDER BY type; + +INSERT INTO tp SELECT number % 3, 1, -1, 0 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 2, 1, 1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 1, 1, 0 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +SELECT 'DEDUPLICATE ON MergeTree'; +DROP TABLE IF EXISTS tp; +CREATE TABLE tp +( + `type` Int32, + `eventcnt` UInt64, + PROJECTION p + ( + SELECT type,sum(eventcnt) + GROUP BY type + ) +) +ENGINE = MergeTree +ORDER BY type; + +INSERT INTO tp SELECT number % 3, 1 FROM numbers(3); +INSERT INTO tp SELECT number % 3, 2 FROM numbers(3); + +OPTIMIZE TABLE tp FINAL DEDUPLICATE BY type; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 0, force_optimize_projection = 0; + +SELECT type,sum(eventcnt) FROM tp GROUP BY type ORDER BY type +SETTINGS allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + From 8c2a371eaa8e51c0d382a13f4413ab9a2796e02b Mon Sep 17 00:00:00 2001 From: cangyin Date: Mon, 8 Apr 2024 18:52:08 +0800 Subject: [PATCH 0067/1722] no readability-make-member-function-const --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 4be1b003573..66629163ebb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -504,7 +504,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPart } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // NOLINT { /// In case if there are no projections we didn't construct a task if (!ctx->merge_projection_parts_task_ptr) From 8b2bd3cfd7654fee98df6f024bcf7e4b6b4f2b49 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Mar 2024 20:48:49 +0000 Subject: [PATCH 0068/1722] impl --- src/Common/AsynchronousMetrics.cpp | 187 +++++++++++++++++++++-------- src/Common/AsynchronousMetrics.h | 6 + 2 files changed, 142 insertions(+), 51 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ab54b180fbf..cf9e8d21bd8 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1,18 +1,19 @@ -#include -#include -#include -#include -#include -#include -#include -#include +#include #include #include +#include #include #include #include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include #include "config.h" @@ -78,6 +79,7 @@ AsynchronousMetrics::AsynchronousMetrics( openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes); } openFileIfExists("/sys/fs/cgroup/cpu.max", cgroupcpu_max); + openFileIfExists("/sys/fs/cgroup/cpu.stat", cgroupcpu_stat); /// CGroups v1 if (!cgroupmem_limit_in_bytes) @@ -90,6 +92,8 @@ AsynchronousMetrics::AsynchronousMetrics( openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_period_us", cgroupcpu_cfs_period); openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", cgroupcpu_cfs_quota); } + if (!cgroupcpu_stat) + openFileIfExists("/sys/fs/cgroup/cpuacct/cpuacct.stat", cgroupcpuacct_stat); openFileIfExists("/proc/sys/vm/max_map_count", vm_max_map_count); openFileIfExists("/proc/self/maps", vm_maps); @@ -561,6 +565,82 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif +void AsynchronousMetrics::applyCPUMetricsUpdate( + AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier) +{ + new_values["OSUserTime" + cpu_suffix] + = {delta_values.user * multiplier, + "The ratio of time the CPU core was running userspace code. This is a system-wide metric, it includes all the processes on the " + "host machine, not just clickhouse-server." + " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline " + "stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSNiceTime" + cpu_suffix] + = {delta_values.nice * multiplier, + "The ratio of time the CPU core was running userspace code with higher priority. This is a system-wide metric, it includes all " + "the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSSystemTime" + cpu_suffix] + = {delta_values.system * multiplier, + "The ratio of time the CPU core was running OS kernel (system) code. This is a system-wide metric, it includes all the " + "processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSIdleTime" + cpu_suffix] + = {delta_values.idle * multiplier, + "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This " + "is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline " + "stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSIOWaitTime" + cpu_suffix] + = {delta_values.iowait * multiplier, + "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as " + "the processes were waiting for IO. This is a system-wide metric, it includes all the processes on the host machine, not just " + "clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSIrqTime" + cpu_suffix] + = {delta_values.irq * multiplier, + "The ratio of time spent for running hardware interrupt requests on the CPU. This is a system-wide metric, it includes all the " + "processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate hardware misconfiguration or a very high network load." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSSoftIrqTime" + cpu_suffix] + = {delta_values.softirq * multiplier, + "The ratio of time spent for running software interrupt requests on the CPU. This is a system-wide metric, it includes all the " + "processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate inefficient software running on the system." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSStealTime" + cpu_suffix] + = {delta_values.steal * multiplier, + "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is a system-wide " + "metric, it includes all the processes on the host machine, not just clickhouse-server." + " Not every virtualized environments present this metric, and most of them don't." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSGuestTime" + cpu_suffix] + = {delta_values.guest * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man " + "procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; + new_values["OSGuestNiceTime" + cpu_suffix] + = {delta_values.guest_nice * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest " + "was set to a higher priority (See `man procfs`). This is a system-wide metric, it includes all the processes on the host " + "machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across " + "them [0..num cores]."}; +} + void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { Stopwatch watch; @@ -821,16 +901,57 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) new_values["CGroupMaxCPU"] = { max_cpu_cgroups, "The maximum number of CPU cores according to CGroups."}; } - if (proc_stat) + int64_t hz = sysconf(_SC_CLK_TCK); + if (-1 == hz) + throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); + + if (cgroupcpu_stat || cgroupcpuacct_stat) + { + ReadBufferFromFilePRead & in = cgroupcpu_stat ? *cgroupcpu_stat : *cgroupcpuacct_stat; + ProcStatValuesCPU current_values{}; + + /// We re-read the file from the beginning each time + in.rewind(); + + while (!in.eof()) + { + String name; + readStringUntilWhitespace(name, in); + skipWhitespaceIfAny(in); + + /// `user_usec` for cgroup v2 and `user` for cgroup v1 + if (name.starts_with("user")) + { + readText(current_values.user, in); + skipToNextLineOrEOF(in); + } + /// `system_usec` for cgroup v2 and `system` for cgroup v1 + else if (name.starts_with("system")) + { + readText(current_values.system, in); + skipToNextLineOrEOF(in); + } + else + skipToNextLineOrEOF(in); + } + + if (!first_run) + { + const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; + const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; + const double multiplier = 1.0 / cgroup_specific_divisor + / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); + applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); + } + + proc_stat_values_all_cpus = current_values; + } + else if (proc_stat) { try { proc_stat->rewind(); - int64_t hz = sysconf(_SC_CLK_TCK); - if (-1 == hz) - throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); - double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); size_t num_cpus = 0; @@ -876,43 +997,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) else delta_values_all_cpus = delta_values; - new_values["OSUserTime" + cpu_suffix] = { delta_values.user * multiplier, - "The ratio of time the CPU core was running userspace code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSNiceTime" + cpu_suffix] = { delta_values.nice * multiplier, - "The ratio of time the CPU core was running userspace code with higher priority. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSSystemTime" + cpu_suffix] = { delta_values.system * multiplier, - "The ratio of time the CPU core was running OS kernel (system) code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSIdleTime" + cpu_suffix] = { delta_values.idle * multiplier, - "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSIOWaitTime" + cpu_suffix] = { delta_values.iowait * multiplier, - "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSIrqTime" + cpu_suffix] = { delta_values.irq * multiplier, - "The ratio of time spent for running hardware interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " A high number of this metric may indicate hardware misconfiguration or a very high network load." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSSoftIrqTime" + cpu_suffix] = { delta_values.softirq * multiplier, - "The ratio of time spent for running software interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " A high number of this metric may indicate inefficient software running on the system." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSStealTime" + cpu_suffix] = { delta_values.steal * multiplier, - "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " Not every virtualized environments present this metric, and most of them don't." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSGuestTime" + cpu_suffix] = { delta_values.guest * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This metric is irrelevant for ClickHouse, but still exists for completeness." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; - new_values["OSGuestNiceTime" + cpu_suffix] = { delta_values.guest_nice * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." - " This metric is irrelevant for ClickHouse, but still exists for completeness." - " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + applyCPUMetricsUpdate(new_values, cpu_suffix, delta_values, multiplier); } prev_values = current_values; diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 4b3d28e80c5..caebcd4cdef 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -122,6 +122,8 @@ private: std::optional cgroupcpu_cfs_period TSA_GUARDED_BY(data_mutex); std::optional cgroupcpu_cfs_quota TSA_GUARDED_BY(data_mutex); std::optional cgroupcpu_max TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_stat TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpuacct_stat TSA_GUARDED_BY(data_mutex); std::optional vm_max_map_count TSA_GUARDED_BY(data_mutex); std::optional vm_maps TSA_GUARDED_BY(data_mutex); @@ -217,6 +219,10 @@ private: void openBlockDevices(); void openSensorsChips(); void openEDAC(); + + void applyCPUMetricsUpdate( + AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier); + #endif void run(); From 85e8a5678783521442a6e61bcd00ba6167302b6a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 27 Mar 2024 17:02:52 +0000 Subject: [PATCH 0069/1722] normalized metrics --- src/Common/AsynchronousMetrics.cpp | 170 ++++++++++++++++++----------- src/Common/AsynchronousMetrics.h | 6 + 2 files changed, 115 insertions(+), 61 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index cf9e8d21bd8..59595e701c1 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -641,6 +641,73 @@ void AsynchronousMetrics::applyCPUMetricsUpdate( "them [0..num cores]."}; } +void AsynchronousMetrics::applyNormalizedCPUMetricsUpdate( + AsynchronousMetricValues & new_values, double num_cpus_to_normalize, const ProcStatValuesCPU & delta_values_all_cpus, double multiplier) +{ + chassert(num_cpus_to_normalize); + + new_values["OSUserTimeNormalized"] + = {delta_values_all_cpus.user * multiplier / num_cpus_to_normalize, + "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSNiceTimeNormalized"] + = {delta_values_all_cpus.nice * multiplier / num_cpus_to_normalize, + "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSSystemTimeNormalized"] + = {delta_values_all_cpus.system * multiplier / num_cpus_to_normalize, + "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSIdleTimeNormalized"] + = {delta_values_all_cpus.idle * multiplier / num_cpus_to_normalize, + "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSIOWaitTimeNormalized"] + = {delta_values_all_cpus.iowait * multiplier / num_cpus_to_normalize, + "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSIrqTimeNormalized"] + = {delta_values_all_cpus.irq * multiplier / num_cpus_to_normalize, + "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of " + "the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSSoftIrqTimeNormalized"] + = {delta_values_all_cpus.softirq * multiplier / num_cpus_to_normalize, + "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval " + "regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSStealTimeNormalized"] + = {delta_values_all_cpus.steal * multiplier / num_cpus_to_normalize, + "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestTimeNormalized"] + = {delta_values_all_cpus.guest * multiplier / num_cpus_to_normalize, + "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless " + "of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestNiceTimeNormalized"] + = {delta_values_all_cpus.guest_nice * multiplier / num_cpus_to_normalize, + "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval " + "regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " + "non-uniform, and still get the average resource utilization metric."}; +} + void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { Stopwatch watch; @@ -907,44 +974,56 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) if (cgroupcpu_stat || cgroupcpuacct_stat) { - ReadBufferFromFilePRead & in = cgroupcpu_stat ? *cgroupcpu_stat : *cgroupcpuacct_stat; - ProcStatValuesCPU current_values{}; - - /// We re-read the file from the beginning each time - in.rewind(); - - while (!in.eof()) + try { - String name; - readStringUntilWhitespace(name, in); - skipWhitespaceIfAny(in); + ReadBufferFromFilePRead & in = cgroupcpu_stat ? *cgroupcpu_stat : *cgroupcpuacct_stat; + ProcStatValuesCPU current_values{}; - /// `user_usec` for cgroup v2 and `user` for cgroup v1 - if (name.starts_with("user")) + /// We re-read the file from the beginning each time + in.rewind(); + + while (!in.eof()) { - readText(current_values.user, in); - skipToNextLineOrEOF(in); + String name; + readStringUntilWhitespace(name, in); + skipWhitespaceIfAny(in); + + /// `user_usec` for cgroup v2 and `user` for cgroup v1 + if (name.starts_with("user")) + { + readText(current_values.user, in); + skipToNextLineOrEOF(in); + } + /// `system_usec` for cgroup v2 and `system` for cgroup v1 + else if (name.starts_with("system")) + { + readText(current_values.system, in); + skipToNextLineOrEOF(in); + } + else + skipToNextLineOrEOF(in); } - /// `system_usec` for cgroup v2 and `system` for cgroup v1 - else if (name.starts_with("system")) + + if (!first_run) { - readText(current_values.system, in); - skipToNextLineOrEOF(in); + const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; + const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; + const double multiplier = 1.0 / cgroup_specific_divisor + / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); + applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); + if (max_cpu_cgroups > 0) + applyNormalizedCPUMetricsUpdate(new_values, max_cpu_cgroups, delta_values, multiplier); } - else - skipToNextLineOrEOF(in); + + proc_stat_values_all_cpus = current_values; } - - if (!first_run) + catch (...) { - const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; - const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; - const double multiplier = 1.0 / cgroup_specific_divisor - / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); - applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); + tryLogCurrentException(__PRETTY_FUNCTION__); + openFileIfExists("/sys/fs/cgroup/cpu.stat", cgroupcpu_stat); + if (!cgroupcpu_stat) + openFileIfExists("/sys/fs/cgroup/cpuacct/cpuacct.stat", cgroupcpuacct_stat); } - - proc_stat_values_all_cpus = current_values; } else if (proc_stat) { @@ -1053,38 +1132,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) Float64 num_cpus_to_normalize = max_cpu_cgroups > 0 ? max_cpu_cgroups : num_cpus; if (num_cpus_to_normalize > 0) - { - new_values["OSUserTimeNormalized"] = { delta_values_all_cpus.user * multiplier / num_cpus_to_normalize, - "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSNiceTimeNormalized"] = { delta_values_all_cpus.nice * multiplier / num_cpus_to_normalize, - "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSSystemTimeNormalized"] = { delta_values_all_cpus.system * multiplier / num_cpus_to_normalize, - "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIdleTimeNormalized"] = { delta_values_all_cpus.idle * multiplier / num_cpus_to_normalize, - "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIOWaitTimeNormalized"] = { delta_values_all_cpus.iowait * multiplier / num_cpus_to_normalize, - "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIrqTimeNormalized"] = { delta_values_all_cpus.irq * multiplier / num_cpus_to_normalize, - "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSSoftIrqTimeNormalized"] = { delta_values_all_cpus.softirq * multiplier / num_cpus_to_normalize, - "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSStealTimeNormalized"] = { delta_values_all_cpus.steal * multiplier / num_cpus_to_normalize, - "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSGuestTimeNormalized"] = { delta_values_all_cpus.guest * multiplier / num_cpus_to_normalize, - "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSGuestNiceTimeNormalized"] = { delta_values_all_cpus.guest_nice * multiplier / num_cpus_to_normalize, - "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." - " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - } + applyNormalizedCPUMetricsUpdate(new_values, num_cpus_to_normalize, delta_values_all_cpus, multiplier); } proc_stat_values_other = current_other_values; diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index caebcd4cdef..2b58fd78044 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -223,6 +223,12 @@ private: void applyCPUMetricsUpdate( AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier); + void applyNormalizedCPUMetricsUpdate( + AsynchronousMetricValues & new_values, + double num_cpus_to_normalize, + const ProcStatValuesCPU & delta_values_all_cpus, + double multiplier); + #endif void run(); From 4aaae7fd4d3340131515be83764e56b5f5c17c13 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 27 Mar 2024 19:49:00 +0000 Subject: [PATCH 0070/1722] add test --- .../test_async_metrics_in_cgroup/test.py | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 tests/integration/test_async_metrics_in_cgroup/test.py diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py new file mode 100644 index 00000000000..1bba42cb980 --- /dev/null +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -0,0 +1,77 @@ +import pytest +import subprocess +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node") + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_client_logs_level(start_cluster): + # check that our metrics sources actually exist + assert ( + subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 + or subprocess.Popen( + "test -f /sys/fs/cgroup/cpuacct/cpuacct.stat".split(" ") + ).wait() + == 0 + ) + + # first let's spawn some cpu-intensive process outside of the container and check that it doesn't accounted by ClickHouse server + proc = subprocess.Popen( + "openssl speed -multi 8".split(" "), + stdout=subprocess.DEVNULL, + stderr=subprocess.DEVNULL, + ) + + time.sleep(5) + + metric = node.query( + """ + SYSTEM FLUSH LOGS; + + SELECT max(value) + FROM ( + SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t + ) + """ + ).strip("\n") + + assert float(metric) <= 2 + + proc.kill() + + # then let's test that we will account cpu time spent by the server itself + node.query( + "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=5, max_threads=8", + ignore_error=True, + ) + + metric = node.query( + """ + SYSTEM FLUSH LOGS; + + SELECT max(value) + FROM ( + SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t + ) + """ + ).strip("\n") + + assert 4 <= float(metric) <= 12 From 75011d6f21e4948bf86fd52e2330fe2f2d8fa922 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 27 Mar 2024 20:22:15 +0000 Subject: [PATCH 0071/1722] fix style --- tests/integration/test_async_metrics_in_cgroup/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_async_metrics_in_cgroup/__init__.py diff --git a/tests/integration/test_async_metrics_in_cgroup/__init__.py b/tests/integration/test_async_metrics_in_cgroup/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From d84a01cabfbb97a8b875620292f843c1247e6382 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 20:46:02 +0000 Subject: [PATCH 0072/1722] better --- src/Common/AsynchronousMetrics.cpp | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 59595e701c1..0943232e776 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -565,6 +565,7 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif +#if defined(OS_LINUX) void AsynchronousMetrics::applyCPUMetricsUpdate( AsynchronousMetricValues & new_values, const std::string & cpu_suffix, const ProcStatValuesCPU & delta_values, double multiplier) { @@ -707,6 +708,7 @@ void AsynchronousMetrics::applyNormalizedCPUMetricsUpdate( " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is " "non-uniform, and still get the average resource utilization metric."}; } +#endif void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { @@ -968,10 +970,6 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) new_values["CGroupMaxCPU"] = { max_cpu_cgroups, "The maximum number of CPU cores according to CGroups."}; } - int64_t hz = sysconf(_SC_CLK_TCK); - if (-1 == hz) - throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); - if (cgroupcpu_stat || cgroupcpuacct_stat) { try @@ -1006,10 +1004,14 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) if (!first_run) { - const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; - const auto cgroup_specific_divisor = cgroupcpu_stat ? 1e6 : hz; - const double multiplier = 1.0 / cgroup_specific_divisor + int64_t hz = sysconf(_SC_CLK_TCK); + if (-1 == hz) + throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); + const auto cgroup_version_specific_divisor = cgroupcpu_stat ? 1e6 : hz; + const double multiplier = 1.0 / cgroup_version_specific_divisor / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); + + const ProcStatValuesCPU delta_values = current_values - proc_stat_values_all_cpus; applyCPUMetricsUpdate(new_values, /*cpu_suffix=*/"", delta_values, multiplier); if (max_cpu_cgroups > 0) applyNormalizedCPUMetricsUpdate(new_values, max_cpu_cgroups, delta_values, multiplier); @@ -1031,6 +1033,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { proc_stat->rewind(); + int64_t hz = sysconf(_SC_CLK_TCK); + if (-1 == hz) + throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); + double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); size_t num_cpus = 0; From 77e3ff7ff50b0e78235ab9a8ee88b258bdcaf510 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 28 Mar 2024 21:18:29 +0000 Subject: [PATCH 0073/1722] fix test --- tests/integration/test_async_metrics_in_cgroup/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py index 1bba42cb980..e63d53e1485 100644 --- a/tests/integration/test_async_metrics_in_cgroup/test.py +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -17,7 +17,7 @@ def start_cluster(): cluster.shutdown() -def test_check_client_logs_level(start_cluster): +def test_user_cpu_accounting(start_cluster): # check that our metrics sources actually exist assert ( subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 @@ -50,7 +50,7 @@ def test_check_client_logs_level(start_cluster): """ ).strip("\n") - assert float(metric) <= 2 + assert float(metric) < 2 proc.kill() @@ -74,4 +74,5 @@ def test_check_client_logs_level(start_cluster): """ ).strip("\n") - assert 4 <= float(metric) <= 12 + # this check is really weak, but CI is tough place and we cannot guarantee that test process will get many cpu time + assert float(metric) > 1 From bc6a82d9cd68a8a4af3ef92b9a91eaa3be0aa347 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 9 Apr 2024 18:35:11 +0000 Subject: [PATCH 0074/1722] fix test --- .../test_async_metrics_in_cgroup/test.py | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py index e63d53e1485..00951c95a0e 100644 --- a/tests/integration/test_async_metrics_in_cgroup/test.py +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -18,6 +18,9 @@ def start_cluster(): def test_user_cpu_accounting(start_cluster): + if node.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + # check that our metrics sources actually exist assert ( subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 @@ -38,14 +41,12 @@ def test_user_cpu_accounting(start_cluster): metric = node.query( """ - SYSTEM FLUSH LOGS; - SELECT max(value) FROM ( SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t ) """ ).strip("\n") @@ -56,20 +57,18 @@ def test_user_cpu_accounting(start_cluster): # then let's test that we will account cpu time spent by the server itself node.query( - "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=5, max_threads=8", + "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=10", ignore_error=True, ) metric = node.query( """ - SYSTEM FLUSH LOGS; - SELECT max(value) FROM ( SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = 'OSUserTime' + GROUP BY t ) """ ).strip("\n") From 9aa0fa11f843bf82432f93d497ec9a68dc756db6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 10 Apr 2024 14:01:53 +0200 Subject: [PATCH 0075/1722] IntervalKind -> IntervalKind::Kind --- src/Functions/DateTimeTransforms.h | 6 +-- src/Functions/toStartOfInterval.cpp | 70 ++++++++++++++--------------- 2 files changed, 38 insertions(+), 38 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 8d70dbea685..20dc1bc21f2 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -703,7 +703,7 @@ struct ToStartOfInterval else { if (weeks < Int64_max_value / 7) // Check if multiplication doesn't overflow Int64 value - return ToStartOfInterval::execute(t, weeks * 7, time_zone, scale_multiplier, origin); + return ToStartOfInterval::execute(t, weeks * 7, time_zone, scale_multiplier, origin); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 7 is out of bounds for type Int64", weeks); } @@ -765,7 +765,7 @@ struct ToStartOfInterval else { if (quarters < Int64_max_value / 3) // Check if multiplication doesn't overflow Int64 value - return ToStartOfInterval::execute(t, quarters * 3, time_zone, scale_multiplier, origin); + return ToStartOfInterval::execute(t, quarters * 3, time_zone, scale_multiplier, origin); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 3 is out of bounds for type Int64", quarters); } @@ -794,7 +794,7 @@ struct ToStartOfInterval else { if (years < Int64_max_value / 12) // Check if multiplication doesn't overflow Int64 value - return ToStartOfInterval::execute(t, years * 12, time_zone, scale_multiplier, origin); + return ToStartOfInterval::execute(t, years * 12, time_zone, scale_multiplier, origin); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} * 12 is out of bounds for type Int64", years); } diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index bdf947977b6..ab8dfef58ca 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -192,11 +192,11 @@ public: UInt32 scale = 0; if (isDateTime64(arguments[0].type)) scale = assert_cast(*arguments[0].type.get()).getScale(); - if (interval_type->getKind() == IntervalKind::Nanosecond) + if (interval_type->getKind() == IntervalKind::Kind::Nanosecond) scale = (9 > scale) ? 9 : scale; - else if (interval_type->getKind() == IntervalKind::Microsecond) + else if (interval_type->getKind() == IntervalKind::Kind::Microsecond) scale = (6 > scale) ? 6 : scale; - else if (interval_type->getKind() == IntervalKind::Millisecond) + else if (interval_type->getKind() == IntervalKind::Kind::Millisecond) scale = (3 > scale) ? 3 : scale; const size_t time_zone_arg_num = (overload == Overload::Default) ? 2 : 3; @@ -272,15 +272,15 @@ private: { switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { - case IntervalKind::Nanosecond: - case IntervalKind::Microsecond: - case IntervalKind::Millisecond: + case IntervalKind::Kind::Nanosecond: + case IntervalKind::Kind::Microsecond: + case IntervalKind::Kind::Millisecond: if (isDate(time_data_type) || isDateTime(time_data_type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type {}", isDate(time_data_type) ? "Date" : "DateTime"); break; - case IntervalKind::Second: - case IntervalKind::Minute: - case IntervalKind::Hour: + case IntervalKind::Kind::Second: + case IntervalKind::Kind::Minute: + case IntervalKind::Kind::Hour: if (isDate(time_data_type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal interval kind for argument data type Date"); break; @@ -299,28 +299,28 @@ private: switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { - case IntervalKind::Nanosecond: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Microsecond: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Millisecond: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Second: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Minute: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Hour: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Day: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Week: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Month: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Quarter: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); - case IntervalKind::Year: - return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Nanosecond: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Microsecond: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Millisecond: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Second: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Minute: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Hour: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Day: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Week: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Month: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Quarter: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); + case IntervalKind::Kind::Year: + return execute(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale); } std::unreachable(); @@ -331,11 +331,11 @@ private: { switch (unit) { - case IntervalKind::Millisecond: + case IntervalKind::Kind::Millisecond: return 1'000; - case IntervalKind::Microsecond: + case IntervalKind::Kind::Microsecond: return 1'000'000; - case IntervalKind::Nanosecond: + case IntervalKind::Kind::Nanosecond: return 1'000'000'000; default: return 1; @@ -424,7 +424,7 @@ private: if (isDate(result_type)) /// The result should be a date and the calculations were as datetime. result_data[i] += (origin + offset) / SECONDS_PER_DAY; - else if (unit == IntervalKind::Week || unit == IntervalKind::Month || unit == IntervalKind::Quarter || unit == IntervalKind::Year) + else if (unit == IntervalKind::Kind::Week || unit == IntervalKind::Kind::Month || unit == IntervalKind::Kind::Quarter || unit == IntervalKind::Kind::Year) { if (isDateTime64(result_type)) /// We need to have the right scale for offset, origin already has the right scale. offset *= scale_endtime; From 6109da248f9a4a3d26e0ca8c41225129202e9688 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 10 Apr 2024 14:29:13 +0200 Subject: [PATCH 0076/1722] fix test --- .../02956_fix_to_start_of_milli_microsecond.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference index b005ce6dfb0..95a05a24981 100644 --- a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference +++ b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference @@ -2,6 +2,6 @@ 2023-10-09 10:11:12.001000 2023-10-09 10:11:12.000000 2023-10-09 10:11:12.000000 -2023-10-09 00:00:00.000000 -2023-10-09 00:00:00.000 +2023-10-09 00:00:00.000000000 +2023-10-09 00:00:00.000000000 2023-10-09 00:00:00 From 5ecb5da648afb4d2d62aa7edee55af4ae89e7459 Mon Sep 17 00:00:00 2001 From: cangyin Date: Wed, 10 Apr 2024 19:30:28 +0000 Subject: [PATCH 0077/1722] Also rebuild for OPTIMIZE CLEANUP --- src/Storages/MergeTree/MergeTask.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 66629163ebb..72a75e4a32e 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1205,6 +1205,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe { // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. const bool merge_may_reduce_rows = + global_ctx->cleanup || global_ctx->deduplicate || ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || From 938c888b131d15b311c8368337d36a52ad2b0a02 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Apr 2024 17:01:01 +0200 Subject: [PATCH 0078/1722] Reload CI to remove build error (empty commit) From 115d7cfa572556f321373e441e5becf3221426ea Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 26 Apr 2024 15:22:34 +0300 Subject: [PATCH 0079/1722] fix after merge --- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 84bb2e78eb6..b0d2faf1651 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7982,13 +7982,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); - - /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. - DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); - static const String TMP_PREFIX = "tmp_replace_from_"; auto zookeeper = getZooKeeper(); @@ -7999,7 +7992,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); partitions = src_data.getAllPartitionIds(); - LOG_INFO(log, "Will try to attach {} partitions", partitions.size()); + LOG_INFO(log, "Will try to attach {} partitions without replace", partitions.size()); } else { partitions = std::unordered_set(); @@ -8008,6 +8001,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( for (const auto & partition_id : partitions) { + /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. auto src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts from partition '{}'", src_all_parts.size(), partition_id); From 05e823a1e9eff9d0df0b6473c19eddc03811d016 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 15:37:20 +0000 Subject: [PATCH 0080/1722] add chunked wrapper to native protocol --- src/Client/Connection.cpp | 30 ++++-- src/Client/Connection.h | 8 +- src/Core/ProtocolDefines.h | 5 +- src/IO/ReadBufferFromPocoSocket.cpp | 54 +++++++--- src/IO/ReadBufferFromPocoSocket.h | 3 + src/IO/ReadBufferFromPocoSocketChunked.cpp | 114 +++++++++++++++++++++ src/IO/ReadBufferFromPocoSocketChunked.h | 32 ++++++ src/IO/WriteBufferFromPocoSocketChunked.h | 56 ++++++++++ src/Server/TCPHandler.cpp | 50 +++++++-- src/Server/TCPHandler.h | 6 +- 10 files changed, 322 insertions(+), 36 deletions(-) create mode 100644 src/IO/ReadBufferFromPocoSocketChunked.cpp create mode 100644 src/IO/ReadBufferFromPocoSocketChunked.h create mode 100644 src/IO/WriteBufferFromPocoSocketChunked.h diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 483201509c4..970768e515e 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -4,8 +4,6 @@ #include #include #include -#include -#include #include #include #include @@ -191,10 +189,10 @@ void Connection::connect(const ConnectionTimeouts & timeouts) , tcp_keep_alive_timeout_in_sec); } - in = std::make_shared(*socket); + in = std::make_shared(*socket); in->setAsyncCallback(async_callback); - out = std::make_shared(*socket); + out = std::make_shared(*socket); out->setAsyncCallback(async_callback); connected = true; setDescription(); @@ -205,6 +203,12 @@ void Connection::connect(const ConnectionTimeouts & timeouts) if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + in->enableChunked(); + out->enableChunked(); + } + LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); } @@ -567,6 +571,7 @@ bool Connection::ping(const ConnectionTimeouts & timeouts) UInt64 pong = 0; writeVarUInt(Protocol::Client::Ping, *out); + out->finishPacket(); out->next(); if (in->eof()) @@ -611,6 +616,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); + out->finishPacket(); out->next(); UInt64 response_type = 0; @@ -762,6 +768,8 @@ void Connection::sendQuery( block_profile_events_in.reset(); block_out.reset(); + out->finishPacket(); + /// Send empty block which means end of data. if (!with_pending_data) { @@ -778,6 +786,7 @@ void Connection::sendCancel() return; writeVarUInt(Protocol::Client::Cancel, *out); + out->finishPacket(); out->next(); } @@ -804,6 +813,8 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) block_out->write(block); maybe_compressed_out->next(); + if (!block) + out->finishPacket(); out->next(); if (throttler) @@ -814,6 +825,7 @@ void Connection::sendIgnoredPartUUIDs(const std::vector & uuids) { writeVarUInt(Protocol::Client::IgnoredPartUUIDs, *out); writeVectorBinary(uuids, *out); + out->finishPacket(); out->next(); } @@ -823,6 +835,7 @@ void Connection::sendReadTaskResponse(const String & response) writeVarUInt(Protocol::Client::ReadTaskResponse, *out); writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out); writeStringBinary(response, *out); + out->finishPacket(); out->next(); } @@ -831,6 +844,7 @@ void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & resp { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); response.serialize(*out); + out->finishPacket(); out->next(); } @@ -848,6 +862,8 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String copyData(input, *out); else copyData(input, *out, size); + + out->finishPacket(); out->next(); } @@ -876,6 +892,8 @@ void Connection::sendScalarsData(Scalars & data) sendData(elem.second, elem.first, true /* scalar */); } + out->finishPacket(); + out_bytes = out->count() - out_bytes; maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes; double elapsed = watch.elapsedSeconds(); @@ -1018,13 +1036,13 @@ std::optional Connection::getResolvedAddress() const bool Connection::poll(size_t timeout_microseconds) { - return static_cast(*in).poll(timeout_microseconds); + return in->poll(timeout_microseconds); } bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); + return last_input_packet_type.has_value() || in->hasPendingData(); } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 9632eb9d948..e7a6d948204 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -8,8 +8,8 @@ #include -#include -#include +#include +#include #include #include @@ -207,8 +207,8 @@ private: String server_display_name; std::unique_ptr socket; - std::shared_ptr in; - std::shared_ptr out; + std::shared_ptr in; + std::shared_ptr out; std::optional last_input_packet_type; String query_id; diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 159a4c28b6d..837801edcbb 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -79,6 +79,9 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION = 54466; /// Send read-only flag for Replicated tables as well static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; +/// Packets size header +static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS = 54468; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -86,6 +89,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54467; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54468; } diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 26cdee4140c..5fb7ea0440c 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -32,25 +32,13 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -bool ReadBufferFromPocoSocket::nextImpl() +size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) { ssize_t bytes_read = 0; - Stopwatch watch; - - SCOPE_EXIT({ - /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one - ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); - }); /// Add more details to exceptions. try { - CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - - if (internal_buffer.size() > INT_MAX) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - /// If async_callback is specified, set socket to non-blocking mode /// and try to read data from it, if socket is not ready for reading, /// run async_callback and try again later. @@ -61,7 +49,7 @@ bool ReadBufferFromPocoSocket::nextImpl() socket.setBlocking(false); SCOPE_EXIT(socket.setBlocking(true)); bool secure = socket.secure(); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) @@ -73,12 +61,12 @@ bool ReadBufferFromPocoSocket::nextImpl() async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); /// Try to read again. - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); } } else { - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); } } catch (const Poco::Net::NetException & e) @@ -99,6 +87,40 @@ bool ReadBufferFromPocoSocket::nextImpl() if (bytes_read < 0) throw NetException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot read from socket (peer: {}, local: {})", peer_address.toString(), socket.address().toString()); + return bytes_read; +} + +bool ReadBufferFromPocoSocket::readSocketExact(Position begin, size_t size) +{ + for (size_t bytes_left = size; bytes_left > 0;) + { + size_t ret = readSocket(begin + size - bytes_left, bytes_left); + if (ret == 0) + return false; + bytes_left -= ret; + } + + return true; +} + +bool ReadBufferFromPocoSocket::nextImpl() +{ + ssize_t bytes_read = 0; + Stopwatch watch; + + SCOPE_EXIT({ + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); + }); + + CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); + + if (internal_buffer.size() > INT_MAX) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); + + bytes_read = readSocket(internal_buffer.begin(), internal_buffer.size()); + if (read_event != ProfileEvents::end()) ProfileEvents::increment(read_event, bytes_read); diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 76156612764..c40a54ed7ae 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -32,6 +32,9 @@ public: void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } + size_t readSocket(Position begin, size_t size); + bool readSocketExact(Position begin, size_t size); + private: AsyncCallback async_callback; std::string socket_description; diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp new file mode 100644 index 00000000000..f0a157a7e1c --- /dev/null +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -0,0 +1,114 @@ +#include +#include + + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB +{ +ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) + : ReadBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) +{} + +ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) + : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), buffer_socket(socket_, read_event_, buf_size) +{ + chassert(buf_size <= std::numeric_limits::max()); + + working_buffer = buffer_socket.buffer(); + pos = buffer_socket.position(); +} + +void ReadBufferFromPocoSocketChunked::enableChunked() +{ + chunked = true; +} + +bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) +{ + buffer_socket.position() = pos + skip_next; + return buffer_socket.poll(timeout_microseconds); +} + +void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callback_) +{ + buffer_socket.setAsyncCallback(async_callback_); +} + +bool ReadBufferFromPocoSocketChunked::startChunk() +{ + do { + if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) + return false; + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + } while (chunk_left == 0); + + return nextChunk(); +} + +bool ReadBufferFromPocoSocketChunked::nextChunk() +{ + static bool start = false; + + if (chunk_left == 0) { + start = true; + return startChunk(); + } + + if (buffer_socket.available() == 0) + if (!buffer_socket.next()) + return false; + if (start) + LOG_TEST(log, "Packet recieve started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); + else + LOG_TEST(log, "Packet recieve continued. Size {}", chunk_left); + + start = false; + + nextimpl_working_buffer_offset = buffer_socket.offset(); + + if (buffer_socket.available() < chunk_left) + { + working_buffer.resize(buffer_socket.offset() + buffer_socket.available()); + chunk_left -= buffer_socket.available(); + return true; + } + + working_buffer.resize(buffer_socket.offset() + chunk_left); + skip_next = std::min(static_cast(4), buffer_socket.available() - chunk_left); + + if (skip_next > 0) + std::memcpy(&chunk_left, buffer_socket.position() + chunk_left, skip_next); + if (4 > skip_next) + if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + skip_next, 4 - skip_next)) + return false; + + if (chunk_left == 0) + LOG_TEST(log, "Packet recieve ended."); + + return true; +} + + +bool ReadBufferFromPocoSocketChunked::nextImpl() +{ + buffer_socket.position() = pos + skip_next; + skip_next = 0; + + if (chunked) + return nextChunk(); + + if (!buffer_socket.next()) + return false; + + pos = buffer_socket.position(); + working_buffer.resize(offset() + buffer_socket.available()); + + return true; +} + +} diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h new file mode 100644 index 00000000000..3d7d91ac93a --- /dev/null +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ReadBufferFromPocoSocketChunked: public ReadBuffer +{ +public: + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + void enableChunked(); + bool poll(size_t timeout_microseconds); + void setAsyncCallback(AsyncCallback async_callback_); + +protected: + bool startChunk(); + bool nextChunk(); + bool nextImpl() override; + +private: + LoggerPtr log; + ReadBufferFromPocoSocket buffer_socket; + bool chunked = false; + UInt32 chunk_left = 0; // chunk left to read from socket + UInt8 skip_next = 0; // skip already processed bytes in buffer_socket +}; + +} diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h new file mode 100644 index 00000000000..b316393aab6 --- /dev/null +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket +{ +public: + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} + + void enableChunked() { chunked = true; } + void finishPacket() + { + if (!chunked) + return; + + next(); + + if (finished) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); + + LOG_TEST(log, "Packet send ended."); + finished = true; + + UInt32 s = 0; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + } +protected: + void nextImpl() override + { + if (chunked) + { + UInt32 s = static_cast(offset()); + if (finished) + LOG_TEST(log, "Packet send started. Message {}, size {}", static_cast(*buffer().begin()), s); + else + LOG_TEST(log, "Packet send continued. Size {}", s); + + finished = false; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + } + + WriteBufferFromPocoSocket::nextImpl(); + } +private: + LoggerPtr log; + bool chunked = false; + bool finished = true; +}; + +} diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ae2f150c4a1..aa33988fdc4 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -19,8 +19,6 @@ #include #include #include -#include -#include #include #include #include @@ -253,8 +251,8 @@ void TCPHandler::runImpl() socket().setSendTimeout(send_timeout); socket().setNoDelay(true); - in = std::make_shared(socket(), read_event); - out = std::make_shared(socket(), write_event); + in = std::make_shared(socket(), read_event); + out = std::make_shared(socket(), write_event); /// Support for PROXY protocol if (parse_proxy_protocol && !receiveProxyHeader()) @@ -289,6 +287,12 @@ void TCPHandler::runImpl() if (!default_database.empty()) session->sessionContext()->setCurrentDatabase(default_database); } + + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + in->enableChunked(); + out->enableChunked(); + } } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { @@ -320,7 +324,7 @@ void TCPHandler::runImpl() { Stopwatch idle_time; UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000; - while (tcp_server.isOpen() && !server.isCancelled() && !static_cast(*in).poll(timeout_ms)) + while (tcp_server.isOpen() && !server.isCancelled() && !in->poll(timeout_ms)) { if (idle_time.elapsedSeconds() > idle_connection_timeout) { @@ -788,7 +792,7 @@ bool TCPHandler::readDataNext() /// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down. while (true) { - if (static_cast(*in).poll(timeout_us)) + if (in->poll(timeout_us)) { /// If client disconnected. if (in->eof()) @@ -1154,6 +1158,8 @@ void TCPHandler::processTablesStatusRequest() } response.write(*out, client_tcp_protocol_version); + + out->finishPacket(); } void TCPHandler::receiveUnexpectedTablesStatusRequest() @@ -1174,6 +1180,8 @@ void TCPHandler::sendPartUUIDs() writeVarUInt(Protocol::Server::PartUUIDs, *out); writeVectorBinary(uuids, *out); + + out->finishPacket(); out->next(); } } @@ -1182,6 +1190,8 @@ void TCPHandler::sendPartUUIDs() void TCPHandler::sendReadTaskRequestAssumeLocked() { writeVarUInt(Protocol::Server::ReadTaskRequest, *out); + + out->finishPacket(); out->next(); } @@ -1190,6 +1200,8 @@ void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRanges { writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); announcement.serialize(*out); + + out->finishPacket(); out->next(); } @@ -1198,6 +1210,8 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re { writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); request.serialize(*out); + + out->finishPacket(); out->next(); } @@ -1206,6 +1220,8 @@ void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out); + + out->finishPacket(); out->next(); } @@ -1221,6 +1237,8 @@ void TCPHandler::sendTotals(const Block & totals) state.block_out->write(totals); state.maybe_compressed_out->next(); + + out->finishPacket(); out->next(); } } @@ -1237,6 +1255,8 @@ void TCPHandler::sendExtremes(const Block & extremes) state.block_out->write(extremes); state.maybe_compressed_out->next(); + + out->finishPacket(); out->next(); } } @@ -1254,6 +1274,8 @@ void TCPHandler::sendProfileEvents() writeStringBinary("", *out); state.profile_events_block_out->write(block); + + out->finishPacket(); out->next(); auto elapsed_milliseconds = stopwatch.elapsedMilliseconds(); @@ -1291,6 +1313,8 @@ void TCPHandler::sendTimezone() LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); + + out->finishPacket(); out->next(); } @@ -1636,6 +1660,7 @@ bool TCPHandler::receivePacket() case Protocol::Client::Ping: writeVarUInt(Protocol::Server::Pong, *out); + out->finishPacket(); out->next(); return false; @@ -2152,7 +2177,7 @@ QueryState::CancellationStatus TCPHandler::getQueryCancellationStatus() after_check_cancelled.restart(); /// During request execution the only packet that can come from the client is stopping the query. - if (static_cast(*in).poll(0)) + if (in->poll(0)) { if (in->eof()) { @@ -2216,6 +2241,8 @@ void TCPHandler::sendData(const Block & block) state.block_out->write(block); state.maybe_compressed_out->next(); + + out->finishPacket(); out->next(); } catch (...) @@ -2251,6 +2278,8 @@ void TCPHandler::sendLogData(const Block & block) writeStringBinary("", *out); state.logs_block_out->write(block); + + out->finishPacket(); out->next(); } @@ -2262,6 +2291,7 @@ void TCPHandler::sendTableColumns(const ColumnsDescription & columns) writeStringBinary("", *out); writeStringBinary(columns.toString(), *out); + out->finishPacket(); out->next(); } @@ -2271,6 +2301,8 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace) writeVarUInt(Protocol::Server::Exception, *out); writeException(e, *out, with_stack_trace); + + out->finishPacket(); out->next(); } @@ -2281,6 +2313,8 @@ void TCPHandler::sendEndOfStream() state.io.setAllDataSent(); writeVarUInt(Protocol::Server::EndOfStream, *out); + + out->finishPacket(); out->next(); } @@ -2299,6 +2333,8 @@ void TCPHandler::sendProgress() increment.elapsed_ns = current_elapsed_ns - state.prev_elapsed_ns; state.prev_elapsed_ns = current_elapsed_ns; increment.write(*out, client_tcp_protocol_version); + + out->finishPacket(); out->next(); } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 191617f1905..67d77381167 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include "IServer.h" #include "Interpreters/AsynchronousInsertQueue.h" @@ -204,8 +206,8 @@ private: ClientInfo::QueryKind query_kind = ClientInfo::QueryKind::NO_QUERY; /// Streams for reading/writing from/to client connection socket. - std::shared_ptr in; - std::shared_ptr out; + std::shared_ptr in; + std::shared_ptr out; ProfileEvents::Event read_event; ProfileEvents::Event write_event; From daf8277e55058e42fddafc49416164d5cb0ab601 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 16:00:58 +0000 Subject: [PATCH 0081/1722] fix --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 17 ++++++++--------- src/IO/ReadBufferFromPocoSocketChunked.h | 1 + 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index f0a157a7e1c..33bed2a32c4 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -52,22 +52,21 @@ bool ReadBufferFromPocoSocketChunked::startChunk() bool ReadBufferFromPocoSocketChunked::nextChunk() { - static bool start = false; - - if (chunk_left == 0) { - start = true; + if (chunk_left == 0) + { + started = true; return startChunk(); } if (buffer_socket.available() == 0) if (!buffer_socket.next()) return false; - if (start) - LOG_TEST(log, "Packet recieve started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); + if (started) + LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); else - LOG_TEST(log, "Packet recieve continued. Size {}", chunk_left); + LOG_TEST(log, "Packet receive continued. Size {}", chunk_left); - start = false; + started = false; nextimpl_working_buffer_offset = buffer_socket.offset(); @@ -88,7 +87,7 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() return false; if (chunk_left == 0) - LOG_TEST(log, "Packet recieve ended."); + LOG_TEST(log, "Packet receive ended."); return true; } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 3d7d91ac93a..5930285e18a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -27,6 +27,7 @@ private: bool chunked = false; UInt32 chunk_left = 0; // chunk left to read from socket UInt8 skip_next = 0; // skip already processed bytes in buffer_socket + bool started = false; }; } From 3fb45ff1762867cbeb53e31cd4492dda5cf8dbb7 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 14 May 2024 19:17:56 +0200 Subject: [PATCH 0082/1722] Add setting and implementation --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Formats/SchemaInferenceUtils.cpp | 44 ++++++++++++++++++- .../0_stateless/03150_infer_type_variant.sql | 4 ++ 5 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03150_infer_type_variant.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a0de354a03..0c0614550e5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1066,6 +1066,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ + M(Bool, input_format_json_infer_variant_from_multitype_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 43ccee173f0..e027c693094 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -137,6 +137,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.json.infer_incomplete_types_as_strings = settings.input_format_json_infer_incomplete_types_as_strings; + format_settings.json.infer_variant_from_multitype_array = settings.input_format_json_infer_variant_from_multitype_array; format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d5fedf99adb..6cac41bd63e 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -223,6 +223,7 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; + bool infer_variant_from_multitype_array = false; bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; } json{}; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 02c0aa6dd77..fc1a26f9b2f 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -307,6 +308,22 @@ namespace type_indexes.erase(TypeIndex::UInt64); } + /// if setting input_format_json_infer_variant_from_multitype_array is true + /// and nested types are not equal then we convert to type variant. + void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) + { + auto variant_type = std::make_shared(data_types); + /// replace separate types with a single variant type + data_types.clear(); + type_indexes.clear(); + data_types.push_back(variant_type); + type_indexes.insert(TypeIndex::Variant); + + // push it back again + data_types.push_back(variant_type); + type_indexes.insert(TypeIndex::Variant); + } + /// If we have only Date and DateTime types, convert Date to DateTime, /// otherwise, convert all Date and DateTime to String. void transformDatesAndDateTimes(DataTypes & data_types, TypeIndexesSet & type_indexes) @@ -649,6 +666,12 @@ namespace /// Check settings specific for JSON formats. + if (settings.json.infer_variant_from_multitype_array) + { + transformVariant(data_types, type_indexes); + return; + } + /// Convert numbers inferred from strings back to strings if needed. if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings) transformJSONNumbersBackToString(data_types, settings, type_indexes, json_info); @@ -677,6 +700,12 @@ namespace if constexpr (!is_json) return; + if (settings.json.infer_variant_from_multitype_array) + { + transformVariant(data_types, type_indexes); + return; + } + /// Convert JSON tuples with same nested types to arrays. transformTuplesWithEqualNestedTypesToArrays(data_types, type_indexes); @@ -822,7 +851,6 @@ namespace if (checkIfTypesAreEqual(nested_types_copy)) return std::make_shared(nested_types_copy.back()); - return std::make_shared(nested_types); } else @@ -1482,6 +1510,20 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) return nested_type ? std::make_shared(nested_type) : nullptr; } + if (which.isVariant()) + { + const auto * variant_type = assert_cast(type.get()); + DataTypes nested_types; + for (const auto & nested_type: variant_type->getVariants()) + { + /// unlike tuple or array, here we do not want to make any of the variants nullable + /// so we do not call makeNullableRecursively + nested_types.push_back(nested_type); + } + + return std::make_shared(nested_types); + } + if (which.isTuple()) { const auto * tuple_type = assert_cast(type.get()); diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql new file mode 100644 index 00000000000..ac544d04f6e --- /dev/null +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -0,0 +1,4 @@ +SET input_format_json_infer_variant_from_multitype_array=1; +SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From dfdf31f1b6efbbda847a693a22969c2187a949f7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 18:09:11 +0000 Subject: [PATCH 0083/1722] host-net conversion --- src/IO/NetUtils.h | 26 ++++++++++++++++++++++ src/IO/ReadBufferFromPocoSocketChunked.cpp | 16 ++++++++----- src/IO/WriteBufferFromPocoSocketChunked.h | 2 ++ 3 files changed, 38 insertions(+), 6 deletions(-) create mode 100644 src/IO/NetUtils.h diff --git a/src/IO/NetUtils.h b/src/IO/NetUtils.h new file mode 100644 index 00000000000..ac6b5eec9a7 --- /dev/null +++ b/src/IO/NetUtils.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +constexpr T netToHost(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T hostToNet(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +} diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 33bed2a32c4..27903761934 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB::ErrorCodes @@ -9,6 +10,7 @@ namespace DB::ErrorCodes namespace DB { + ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) : ReadBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) {} @@ -40,12 +42,12 @@ void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callb bool ReadBufferFromPocoSocketChunked::startChunk() { - do { - if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) - return false; - if (chunk_left == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); - } while (chunk_left == 0); + if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) + return false; + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + + chunk_left = netToHost(chunk_left); return nextChunk(); } @@ -86,6 +88,8 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + skip_next, 4 - skip_next)) return false; + chunk_left = netToHost(chunk_left); + if (chunk_left == 0) LOG_TEST(log, "Packet receive ended."); diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index b316393aab6..4481dfdedfc 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -42,6 +43,7 @@ protected: LOG_TEST(log, "Packet send continued. Size {}", s); finished = false; + s = hostToNet(s); socketSendBytes(reinterpret_cast(&s), sizeof(s)); } From 467366af990215e11b4b0309b90b3e6d9ebca5fd Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 14 May 2024 21:55:56 +0200 Subject: [PATCH 0084/1722] Fix unimplemented serialization error and update reference file --- src/Formats/SchemaInferenceUtils.cpp | 13 ++++++++--- .../03150_infer_type_variant.reference | 22 +++++++++++++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03150_infer_type_variant.reference diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index fc1a26f9b2f..d0d29892dec 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -319,7 +319,7 @@ namespace data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); - // push it back again + // make the second type variant as well data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); } @@ -669,7 +669,6 @@ namespace if (settings.json.infer_variant_from_multitype_array) { transformVariant(data_types, type_indexes); - return; } /// Convert numbers inferred from strings back to strings if needed. @@ -703,7 +702,6 @@ namespace if (settings.json.infer_variant_from_multitype_array) { transformVariant(data_types, type_indexes); - return; } /// Convert JSON tuples with same nested types to arrays. @@ -1440,6 +1438,15 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F return; } + + if (const auto * variant_type = typeid_cast(data_type.get())) + { + auto nested_types = variant_type->getVariants(); + for (auto & nested_type : nested_types) + transformFinalInferredJSONTypeIfNeededImpl(nested_type, settings, json_info, remain_nothing_types); + data_type = std::make_shared(nested_types); + return; + } } void transformFinalInferredJSONTypeIfNeeded(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference new file mode 100644 index 00000000000..ffb4209eadb --- /dev/null +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -0,0 +1,22 @@ + â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ arr ┃ toTypeName(arr) ┃ + ┡â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple(…│ + │ │… a Int64))) │ + └──────────────────┴─────────────────────────────────────┘ + â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ x ┃ toTypeName(x) ┃ + ┡â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ 42 │ Variant(Int64, String) │ + ├───────┼────────────────────────┤ +2. │ Hello │ Variant(Int64, String) │ + └───────┴────────────────────────┘ + â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ x ┃ toTypeName(x) ┃ + ┡â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ [1,2,3] │ Variant(Array(Int64), Tuple(…│ + │ │… a Int64)) │ + ├─────────┼──────────────────────────────┤ +2. │ (42) │ Variant(Array(Int64), Tuple(…│ + │ │… a Int64)) │ + └─────────┴──────────────────────────────┘ From 88a833335f7e7e9fae85e74d250677f415905292 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 14 May 2024 20:25:26 +0000 Subject: [PATCH 0085/1722] fix --- src/IO/WriteBufferFromPocoSocketChunked.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 4481dfdedfc..39cdd93501b 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: From 4066c6bc548979703f45ba264437f5966c403d6a Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 02:13:53 +0000 Subject: [PATCH 0086/1722] Update setting name --- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 6 +++--- tests/queries/0_stateless/03150_infer_type_variant.sql | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0c0614550e5..ffc337b674f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1066,7 +1066,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_infer_variant_from_multitype_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ + M(Bool, input_format_json_infer_variant_from_multi_type_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index e027c693094..792ac08a5df 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -137,7 +137,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.json.infer_incomplete_types_as_strings = settings.input_format_json_infer_incomplete_types_as_strings; - format_settings.json.infer_variant_from_multitype_array = settings.input_format_json_infer_variant_from_multitype_array; + format_settings.json.infer_variant_from_multi_type_array = settings.input_format_json_infer_variant_from_multi_type_array; format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 6cac41bd63e..d2c75872326 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -223,7 +223,7 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; - bool infer_variant_from_multitype_array = false; + bool infer_variant_from_multi_type_array = false; bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; } json{}; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index d0d29892dec..f693916c584 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -308,7 +308,7 @@ namespace type_indexes.erase(TypeIndex::UInt64); } - /// if setting input_format_json_infer_variant_from_multitype_array is true + /// if setting input_format_json_infer_variant_from_multi_type_array is true /// and nested types are not equal then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { @@ -666,7 +666,7 @@ namespace /// Check settings specific for JSON formats. - if (settings.json.infer_variant_from_multitype_array) + if (settings.json.infer_variant_from_multi_type_array) { transformVariant(data_types, type_indexes); } @@ -699,7 +699,7 @@ namespace if constexpr (!is_json) return; - if (settings.json.infer_variant_from_multitype_array) + if (settings.json.infer_variant_from_multi_type_array) { transformVariant(data_types, type_indexes); } diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index ac544d04f6e..2ea849248f7 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ -SET input_format_json_infer_variant_from_multitype_array=1; +SET input_format_json_infer_variant_from_multi_type_array=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From 2762cf86d2ba3f5c1ac86040b6ef484feb40837b Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 14:19:17 +0200 Subject: [PATCH 0087/1722] fix test file --- tests/queries/0_stateless/03150_infer_type_variant.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index 2ea849248f7..ac544d04f6e 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ -SET input_format_json_infer_variant_from_multi_type_array=1; +SET input_format_json_infer_variant_from_multitype_array=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From 4800aa6a6cdf5a8431fcc3d6fd96672590da0fff Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 14:21:52 +0200 Subject: [PATCH 0088/1722] rename setting in test file --- tests/queries/0_stateless/03150_infer_type_variant.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index ac544d04f6e..2ea849248f7 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ -SET input_format_json_infer_variant_from_multitype_array=1; +SET input_format_json_infer_variant_from_multi_type_array=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); From fbf34519a5e72bb03c57ee6bc0feea1adddcb309 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 15 May 2024 16:03:23 +0200 Subject: [PATCH 0089/1722] Add setting to SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ece48620618..0665d1d6ca6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,6 +93,7 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, + {"input_format_json_infer_variant_from_multi_type_array", 0, 0, "Allows inference of variant type if columns/arrays have multiple types."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, From 777e0b313a48975fa51d645fb09a4f6ebfac1d1e Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 18:51:29 +0200 Subject: [PATCH 0090/1722] Update 03150_infer_type_variant.sql Fix failing test --- tests/queries/0_stateless/03150_infer_type_variant.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index 2ea849248f7..3253ddfe179 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,4 @@ SET input_format_json_infer_variant_from_multi_type_array=1; -SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}'); -SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}'); -SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}'); +SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}') FORMAT Pretty; +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}') FORMAT Pretty; +SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}') FORMAT Pretty; From dd8d5c46c4678f570b9357dbdd912bcd6f4a267e Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 15 May 2024 18:55:34 +0200 Subject: [PATCH 0091/1722] Fix style --- src/Formats/SchemaInferenceUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index f693916c584..2cbb680af97 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -319,7 +319,7 @@ namespace data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); - // make the second type variant as well + /// make the second type variant as well data_types.push_back(variant_type); type_indexes.insert(TypeIndex::Variant); } From ad204887a2516e5053035c709735bf6c99ddba21 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 15 May 2024 20:47:54 +0000 Subject: [PATCH 0092/1722] bugs fixed, switch chunk length to little endian --- src/Client/Connection.cpp | 2 +- src/IO/NetUtils.h | 32 +++++++++++++++++ src/IO/ReadBufferFromPocoSocketChunked.cpp | 42 +++++++++++++++------- src/IO/ReadBufferFromPocoSocketChunked.h | 3 +- src/IO/WriteBufferFromPocoSocketChunked.h | 2 +- 5 files changed, 65 insertions(+), 16 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 970768e515e..3a0f3771e7a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1042,7 +1042,7 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || in->hasPendingData(); + return last_input_packet_type.has_value() || in->hasBufferedData(); } diff --git a/src/IO/NetUtils.h b/src/IO/NetUtils.h index ac6b5eec9a7..12f09524ae7 100644 --- a/src/IO/NetUtils.h +++ b/src/IO/NetUtils.h @@ -23,4 +23,36 @@ constexpr T hostToNet(T value) noexcept return value; } +template +constexpr T toLittleEndian(T value) noexcept +{ + if constexpr (std::endian::native == std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T toBigEndian(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T fromLittleEndian(T value) noexcept +{ + if constexpr (std::endian::native == std::endian::big) + return std::byteswap(value); + return value; +} + +template +constexpr T fromBigEndian(T value) noexcept +{ + if constexpr (std::endian::native != std::endian::big) + return std::byteswap(value); + return value; +} + } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 27903761934..247d8c8ec6a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -27,11 +27,14 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock void ReadBufferFromPocoSocketChunked::enableChunked() { chunked = true; + buffer_socket.position() = pos; } bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) { - buffer_socket.position() = pos + skip_next; + if (!chunked) + buffer_socket.position() = pos; + return buffer_socket.poll(timeout_microseconds); } @@ -42,12 +45,12 @@ void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callb bool ReadBufferFromPocoSocketChunked::startChunk() { - if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) == 0) + if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) < sizeof(chunk_left)) return false; if (chunk_left == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); - chunk_left = netToHost(chunk_left); + chunk_left = fromLittleEndian(chunk_left); return nextChunk(); } @@ -76,19 +79,23 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() { working_buffer.resize(buffer_socket.offset() + buffer_socket.available()); chunk_left -= buffer_socket.available(); + buffer_socket.position() += buffer_socket.available(); return true; } working_buffer.resize(buffer_socket.offset() + chunk_left); - skip_next = std::min(static_cast(4), buffer_socket.available() - chunk_left); + UInt8 buffered = std::min(static_cast(4), buffer_socket.available() - chunk_left); - if (skip_next > 0) - std::memcpy(&chunk_left, buffer_socket.position() + chunk_left, skip_next); - if (4 > skip_next) - if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + skip_next, 4 - skip_next)) + buffer_socket.position() += chunk_left; + if (buffered > 0) + std::memcpy(&chunk_left, buffer_socket.position(), buffered); + buffer_socket.position() += buffered; + + if (4 > buffered) + if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + buffered, 4 - buffered)) return false; - chunk_left = netToHost(chunk_left); + chunk_left = fromLittleEndian(chunk_left); if (chunk_left == 0) LOG_TEST(log, "Packet receive ended."); @@ -99,14 +106,23 @@ bool ReadBufferFromPocoSocketChunked::nextChunk() bool ReadBufferFromPocoSocketChunked::nextImpl() { - buffer_socket.position() = pos + skip_next; - skip_next = 0; - if (chunked) - return nextChunk(); + { + if (!nextChunk()) + { + pos = buffer_socket.position(); + return false; + } + return true; + } + + buffer_socket.position() = pos; if (!buffer_socket.next()) + { + pos = buffer_socket.position(); return false; + } pos = buffer_socket.position(); working_buffer.resize(offset() + buffer_socket.available()); diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 5930285e18a..6f99db4489a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -16,6 +16,8 @@ public: bool poll(size_t timeout_microseconds); void setAsyncCallback(AsyncCallback async_callback_); + bool hasBufferedData() const { return hasPendingData() || buffer_socket.hasPendingData(); } + protected: bool startChunk(); bool nextChunk(); @@ -26,7 +28,6 @@ private: ReadBufferFromPocoSocket buffer_socket; bool chunked = false; UInt32 chunk_left = 0; // chunk left to read from socket - UInt8 skip_next = 0; // skip already processed bytes in buffer_socket bool started = false; }; diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 39cdd93501b..070e87feff2 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -48,7 +48,7 @@ protected: LOG_TEST(log, "Packet send continued. Size {}", s); finished = false; - s = hostToNet(s); + s = toLittleEndian(s); socketSendBytes(reinterpret_cast(&s), sizeof(s)); } From 6378184c7f004e211d86c3fd7a4f482e45b01a59 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 20 May 2024 14:15:47 +0000 Subject: [PATCH 0093/1722] fix, add some introspection functionality --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 3 ++- src/IO/ReadBufferFromPocoSocketChunked.h | 5 +++++ src/IO/WriteBufferFromPocoSocketChunked.h | 10 ++++++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 247d8c8ec6a..4d40d8b4f14 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -16,7 +16,7 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock {} ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), buffer_socket(socket_, read_event_, buf_size) + : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), peer_address(socket_.peerAddress()), our_address(socket_.address()), buffer_socket(socket_, read_event_, buf_size) { chassert(buf_size <= std::numeric_limits::max()); @@ -28,6 +28,7 @@ void ReadBufferFromPocoSocketChunked::enableChunked() { chunked = true; buffer_socket.position() = pos; + working_buffer.resize(offset()); } bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 6f99db4489a..c70363cf7d8 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -18,6 +18,9 @@ public: bool hasBufferedData() const { return hasPendingData() || buffer_socket.hasPendingData(); } + Poco::Net::SocketAddress peerAddress() { return peer_address; } + Poco::Net::SocketAddress ourAddress() { return our_address; } + protected: bool startChunk(); bool nextChunk(); @@ -25,6 +28,8 @@ protected: private: LoggerPtr log; + Poco::Net::SocketAddress peer_address; + Poco::Net::SocketAddress our_address; ReadBufferFromPocoSocket buffer_socket; bool chunked = false; UInt32 chunk_left = 0; // chunk left to read from socket diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 070e87feff2..6c35db62c0c 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -54,6 +54,16 @@ protected: WriteBufferFromPocoSocket::nextImpl(); } + + Poco::Net::SocketAddress peerAddress() + { + return peer_address; + } + + Poco::Net::SocketAddress ourAddress() + { + return our_address; + } private: LoggerPtr log; bool chunked = false; From 5308256c67c5781916018c321273f04fd21c4545 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 20 May 2024 16:25:19 +0000 Subject: [PATCH 0094/1722] enable chunked before processing defaul database --- src/Server/TCPHandler.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b3dbd118d8b..070cd0e3247 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -277,6 +277,12 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + in->enableChunked(); + out->enableChunked(); + } + if (!is_interserver_mode) { /// If session created, then settings in session context has been updated. @@ -287,12 +293,6 @@ void TCPHandler::runImpl() if (!default_database.empty()) session->sessionContext()->setCurrentDatabase(default_database); } - - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) - { - in->enableChunked(); - out->enableChunked(); - } } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { From 9e747cd45312302935cbf15ea518808d4ac9c8c8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 01:20:00 +0000 Subject: [PATCH 0095/1722] fix bug with profile stats in WriteBufferFromPocoSocket --- src/IO/WriteBufferFromPocoSocket.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 10d9fd131cd..e29b3b2cddd 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -183,6 +183,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ , socket(socket_) , peer_address(socket.peerAddress()) , our_address(socket.address()) + , write_event(ProfileEvents::end()) , socket_description("socket (" + peer_address.toString() + ")") { } From 34702b30bcfe3401991fe7c792c02a80185acdf2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 03:21:10 +0000 Subject: [PATCH 0096/1722] fix test --- .../0_stateless/02532_send_logs_level_test.reference | 3 --- tests/queries/0_stateless/02532_send_logs_level_test.sh | 8 ++++++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.reference b/tests/queries/0_stateless/02532_send_logs_level_test.reference index 7e51b888d9c..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02532_send_logs_level_test.reference +++ b/tests/queries/0_stateless/02532_send_logs_level_test.reference @@ -1,3 +0,0 @@ - MergeTreeMarksLoader: Loading marks from path data.cmrk3 - MergeTreeRangeReader: First reader returned: num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), requested columns: key - MergeTreeRangeReader: read() returned num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), sample block key diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index 4afc6d4496b..f2940e9c005 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -17,6 +17,10 @@ $CLICKHOUSE_CLIENT -nm -q " # instead of "last" value, hence you cannot simply append another # --send_logs_level here. CLICKHOUSE_CLIENT_CLEAN=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=test/g') -$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -o -e '.*' -e '.*' -$CLICKHOUSE_CLIENT -q "drop table data" +set -e + +trap "$CLICKHOUSE_CLIENT -q 'drop table data'" EXIT + +$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& (! grep -q -o -e '.*') +$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -q -o -e '.*' From 6c3556dfda92ea9d04ff5db8427a58aa7ab35750 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 04:07:52 +0000 Subject: [PATCH 0097/1722] fix test --- tests/queries/0_stateless/02532_send_logs_level_test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index f2940e9c005..b74fcf78ad1 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -20,7 +20,7 @@ CLICKHOUSE_CLIENT_CLEAN=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level set -e -trap "$CLICKHOUSE_CLIENT -q 'drop table data'" EXIT +trap '$CLICKHOUSE_CLIENT -q "drop table data"' EXIT $CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& (! grep -q -o -e '.*') $CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -q -o -e '.*' From 69cd5ae549cf7acc4de756a70c9b632d139e50fe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 16:39:25 +0000 Subject: [PATCH 0098/1722] process possibly remaining message after network error --- src/Client/ClientBase.cpp | 35 +++++++++++++++++++++++------------ 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b6f821794f1..f3e53efd994 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -951,6 +951,8 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) } catch (Exception & e) { + if (server_exception) + server_exception->rethrow(); if (!is_interactive) e.addMessage("(in query: {})", full_query); throw; @@ -1069,19 +1071,28 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa QueryInterruptHandler::start(signals_before_stop); SCOPE_EXIT({ QueryInterruptHandler::stop(); }); - connection->sendQuery( - connection_parameters.timeouts, - query, - query_parameters, - global_context->getCurrentQueryId(), - query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), - true, - [&](const Progress & progress) { onProgress(progress); }); + try { + connection->sendQuery( + connection_parameters.timeouts, + query, + query_parameters, + global_context->getCurrentQueryId(), + query_processing_stage, + &global_context->getSettingsRef(), + &global_context->getClientInfo(), + true, + [&](const Progress & progress) { onProgress(progress); }); + + if (send_external_tables) + sendExternalTables(parsed_query); + } + catch (const NetException &) + { + // We still want to attempt to process whatever we already recieved or can recieve (socket receive buffer can be not empty) + receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); + throw; + } - if (send_external_tables) - sendExternalTables(parsed_query); receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); break; From 99bd796011aee169f3c4de25b07b330094c4a41a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 May 2024 16:58:50 +0000 Subject: [PATCH 0099/1722] fix spelling --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f3e53efd994..1b8fe83eb51 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1088,7 +1088,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } catch (const NetException &) { - // We still want to attempt to process whatever we already recieved or can recieve (socket receive buffer can be not empty) + // We still want to attempt to process whatever we already received or can receive (socket receive buffer can be not empty) receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); throw; } From a959663e977361af3896f010f9aeadaa8b64c323 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 23 May 2024 16:43:18 +0200 Subject: [PATCH 0100/1722] Update toStartOfInterval.cpp --- src/Functions/toStartOfInterval.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 749aa9f8800..a4f870613d3 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -241,14 +241,14 @@ private: if (isDate(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDateTime(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(&time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } From 94bc0a1e966d95b8a2180f9504ed93592d2026ed Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 23 May 2024 22:01:32 +0000 Subject: [PATCH 0101/1722] add config parameters and client arguments, make default notchunked_optional --- programs/benchmark/Benchmark.cpp | 52 +++++++++++++++++- src/Client/ClientBase.cpp | 39 ++++++++++++++ src/Client/Connection.cpp | 54 ++++++++++++++++++- src/Client/Connection.h | 5 ++ src/Client/ConnectionParameters.cpp | 3 ++ src/Client/ConnectionParameters.h | 2 + src/Client/ConnectionPool.cpp | 6 ++- src/Client/ConnectionPool.h | 15 +++++- .../ClickHouseDictionarySource.cpp | 8 ++- src/Dictionaries/ClickHouseDictionarySource.h | 2 + src/Interpreters/Cluster.cpp | 11 +++- src/Interpreters/Cluster.h | 2 + src/Server/TCPHandler.cpp | 44 ++++++++++++++- src/Server/TCPHandler.h | 2 + .../DistributedAsyncInsertDirectoryQueue.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 3 +- 16 files changed, 240 insertions(+), 10 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 48dca82eb2b..251761e0bad 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -75,6 +75,8 @@ public: const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, + const String & proto_recv_chunked_, const String & quota_key_, const String & stage, bool randomize_, @@ -128,7 +130,9 @@ public: connections.emplace_back(std::make_unique( concurrency, cur_host, cur_port, - default_database_, user_, password_, quota_key_, + default_database_, user_, password_, + proto_send_chunked_, proto_recv_chunked_, + quota_key_, /* cluster_= */ "", /* cluster_secret_= */ "", /* client_name_= */ std::string(DEFAULT_CLIENT_NAME), @@ -662,6 +666,50 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); + String proto_send_chunked {"notchunked_optional"}; + String proto_recv_chunked {"notchunked_optional"}; + + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + proto_send_chunked = cap_str; + proto_recv_chunked = cap_str; + } + else + { + if (direction == "send") + proto_send_chunked = cap_str; + else + proto_recv_chunked = cap_str; + } + } + } + + Benchmark benchmark( options["concurrency"].as(), options["delay"].as(), @@ -673,6 +721,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["database"].as(), options["user"].as(), options["password"].as(), + proto_send_chunked, + proto_recv_chunked, options["quota_key"].as(), options["stage"].as(), options.count("randomize"), diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 1b8fe83eb51..0bceee6ea4d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -75,9 +75,11 @@ #include #include #include +#include #include #include +#include #include "config.h" namespace fs = std::filesystem; @@ -2993,6 +2995,8 @@ void ClientBase::init(int argc, char ** argv) ("config-file,C", po::value(), "config-file path") + ("proto_caps", po::value(), "enable/disable chunked protocol: chunked_optional, notchunked, notchunked_optional, send_chunked, send_chunked_optional, send_notchunked, send_notchunked_optional, recv_chunked, recv_chunked_optional, recv_notchunked, recv_notchunked_optional") + ("query,q", po::value>()->multitoken(), R"(query; can be specified multiple times (--query "SELECT 1" --query "SELECT 2"...))") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") ("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.") @@ -3162,6 +3166,41 @@ void ClientBase::init(int argc, char ** argv) if (options.count("server_logs_file")) server_logs_file = options["server_logs_file"].as(); + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + config().setString("proto_caps.send", std::string(cap_str)); + config().setString("proto_caps.recv", std::string(cap_str)); + } + else + config().setString("proto_caps." + direction, std::string(cap_str)); + } + } + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); query_kind = parseQueryKind(options["query_kind"].as()); profile_events.print = options.count("print-profile-events"); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 082fe8d5098..9327b694d29 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -71,6 +71,7 @@ Connection::~Connection() = default; Connection::Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, const String & proto_recv_chunked_, [[maybe_unused]] const SSHKey & ssh_private_key_, const String & quota_key_, const String & cluster_, @@ -80,6 +81,7 @@ Connection::Connection(const String & host_, UInt16 port_, Protocol::Secure secure_) : host(host_), port(port_), default_database(default_database_) , user(user_), password(password_) + , proto_send_chunked(proto_send_chunked_), proto_recv_chunked(proto_recv_chunked_) #if USE_SSH , ssh_private_key(ssh_private_key_) #endif @@ -206,13 +208,46 @@ void Connection::connect(const ConnectionTimeouts & timeouts) sendHello(); receiveHello(timeouts.handshake_timeout); + bool out_chunked = false; + bool in_chunked = false; + + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) + { + bool chunked_srv = chunked_srv_str.starts_with("chunked"); + bool optional_srv = chunked_srv_str.ends_with("_optional"); + bool chunked_cl = chunked_cl_str.starts_with("chunked"); + bool optional_cl = chunked_cl_str.ends_with("_optional"); + + if (optional_srv) + return chunked_cl; + if (optional_cl) + return chunked_srv; + if (chunked_cl != chunked_srv) + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: {} set to {}, server requires {}", + direction, + chunked_cl ? "chunked" : "notchunked", + chunked_srv ? "chunked" : "notchunked"); + + return chunked_srv; + }; + + out_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send"); + in_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv"); + } + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - in->enableChunked(); - out->enableChunked(); + if (out_chunked) + out->enableChunked(); + if (in_chunked) + in->enableChunked(); } LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", @@ -359,6 +394,13 @@ void Connection::sendAddendum() { if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY) writeStringBinary(quota_key, *out); + + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + writeStringBinary(proto_send_chunked, *out); + writeStringBinary(proto_recv_chunked, *out); + } + out->next(); } @@ -438,6 +480,12 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) else server_version_patch = server_revision; + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + readStringBinary(proto_send_chunked_srv, *in); + readStringBinary(proto_recv_chunked_srv, *in); + } + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { UInt64 rules_size; @@ -1327,6 +1375,8 @@ ServerConnectionPtr Connection::createConnection(const ConnectionParameters & pa parameters.default_database, parameters.user, parameters.password, + parameters.proto_send_chunked, + parameters.proto_recv_chunked, parameters.ssh_private_key, parameters.quota_key, "", /* cluster */ diff --git a/src/Client/Connection.h b/src/Client/Connection.h index e7a6d948204..a04ccd44627 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -52,6 +52,7 @@ public: Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, const String & proto_recv_chunked_, const SSHKey & ssh_private_key_, const String & quota_key_, const String & cluster_, @@ -169,6 +170,10 @@ private: String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; + String proto_send_chunked_srv; + String proto_recv_chunked_srv; #if USE_SSH SSHKey ssh_private_key; #endif diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 774f3375f63..430c462084a 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -103,6 +103,9 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati #endif } + proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); + quota_key = config.getString("quota_key", ""); /// By default compression is disabled if address looks like localhost. diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index f23522d48b3..85174924016 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,6 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; + std::string proto_send_chunked; + std::string proto_recv_chunked; std::string quota_key; SSHKey ssh_private_key; Protocol::Secure security = Protocol::Secure::Disable; diff --git a/src/Client/ConnectionPool.cpp b/src/Client/ConnectionPool.cpp index 5cabb1465d1..05cb97cadc7 100644 --- a/src/Client/ConnectionPool.cpp +++ b/src/Client/ConnectionPool.cpp @@ -12,6 +12,8 @@ ConnectionPoolPtr ConnectionPoolFactory::get( String default_database, String user, String password, + String proto_send_chunked, + String proto_recv_chunked, String quota_key, String cluster, String cluster_secret, @@ -21,7 +23,7 @@ ConnectionPoolPtr ConnectionPoolFactory::get( Priority priority) { Key key{ - max_connections, host, port, default_database, user, password, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; + max_connections, host, port, default_database, user, password, proto_send_chunked, proto_recv_chunked, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; std::lock_guard lock(mutex); auto [it, inserted] = pools.emplace(key, ConnectionPoolPtr{}); @@ -38,6 +40,8 @@ ConnectionPoolPtr ConnectionPoolFactory::get( default_database, user, password, + proto_send_chunked, + proto_recv_chunked, quota_key, cluster, cluster_secret, diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index d35c2552461..2df97dfb454 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -72,6 +72,8 @@ public: const String & default_database_, const String & user_, const String & password_, + const String & proto_send_chunked_, + const String & proto_recv_chunked_, const String & quota_key_, const String & cluster_, const String & cluster_secret_, @@ -84,6 +86,8 @@ public: , default_database(default_database_) , user(user_) , password(password_) + , proto_send_chunked(proto_send_chunked_) + , proto_recv_chunked(proto_recv_chunked_) , quota_key(quota_key_) , cluster(cluster_) , cluster_secret(cluster_secret_) @@ -123,7 +127,9 @@ protected: { return std::make_shared( host, port, - default_database, user, password, SSHKey(), quota_key, + default_database, user, password, + proto_send_chunked, proto_recv_chunked, + SSHKey(), quota_key, cluster, cluster_secret, client_name, compression, secure); } @@ -132,6 +138,8 @@ private: String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; /// For inter-server authorization @@ -157,6 +165,8 @@ public: String default_database; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; String cluster; String cluster_secret; @@ -180,6 +190,8 @@ public: String default_database, String user, String password, + String proto_send_chunked, + String proto_recv_chunked, String quota_key, String cluster, String cluster_secret, @@ -197,6 +209,7 @@ inline bool operator==(const ConnectionPoolFactory::Key & lhs, const ConnectionP { return lhs.max_connections == rhs.max_connections && lhs.host == rhs.host && lhs.port == rhs.port && lhs.default_database == rhs.default_database && lhs.user == rhs.user && lhs.password == rhs.password + && lhs.proto_send_chunked == rhs.proto_send_chunked && lhs.proto_recv_chunked == rhs.proto_recv_chunked && lhs.quota_key == rhs.quota_key && lhs.cluster == rhs.cluster && lhs.cluster_secret == rhs.cluster_secret && lhs.client_name == rhs.client_name && lhs.compression == rhs.compression && lhs.secure == rhs.secure && lhs.priority == rhs.priority; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index bf16f315ddf..3b096da92c6 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -51,6 +51,8 @@ namespace configuration.db, configuration.user, configuration.password, + configuration.proto_send_chunked, + configuration.proto_recv_chunked, configuration.quota_key, "", /* cluster */ "", /* cluster_secret */ @@ -222,7 +224,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { validateNamedCollection( *named_collection, {}, ValidateKeysMultiset{ - "secure", "host", "hostname", "port", "user", "username", "password", "quota_key", "name", + "secure", "host", "hostname", "port", "user", "username", "password", "proto_send_chunked", "proto_recv_chunked", "quota_key", "name", "db", "database", "table","query", "where", "invalidate_query", "update_field", "update_lag"}); const auto secure = named_collection->getOrDefault("secure", false); @@ -234,6 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked_optional"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked_optional"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -258,6 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked_optional"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked_optional"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 3357514eab2..faf9e5f8009 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -23,6 +23,8 @@ public: const std::string host; const std::string user; const std::string password; + const std::string proto_send_chunked; + const std::string proto_recv_chunked; const std::string quota_key; const std::string db; const std::string table; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 59c98491c14..1d7ccd484d0 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,6 +113,9 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked_optional"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked_optional"); + const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); @@ -425,7 +428,9 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, auto pool = ConnectionPoolFactory::instance().get( static_cast(settings.distributed_connections_pool_size), address.host_name, address.port, - address.default_database, address.user, address.password, address.quota_key, + address.default_database, address.user, address.password, + address.proto_send_chunked, address.proto_recv_chunked, + address.quota_key, address.cluster, address.cluster_secret, "server", address.compression, address.secure, address.priority); @@ -589,6 +594,8 @@ void Cluster::addShard( replica.default_database, replica.user, replica.password, + replica.proto_send_chunked, + replica.proto_recv_chunked, replica.quota_key, replica.cluster, replica.cluster_secret, @@ -744,6 +751,8 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti address.default_database, address.user, address.password, + address.proto_send_chunked, + address.proto_recv_chunked, address.quota_key, address.cluster, address.cluster_secret, diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index dc5790ac339..c993af5fc5e 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,6 +114,8 @@ public: UInt16 port{0}; String user; String password; + String proto_send_chunked; + String proto_recv_chunked; String quota_key; /// For inter-server authorization diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 2071eac3a68..c7db25c4c3a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,6 +1,7 @@ #include "Interpreters/AsynchronousInsertQueue.h" #include "Interpreters/SquashingTransform.h" #include "Parsers/ASTInsertQuery.h" +#include #include #include #include @@ -99,6 +100,7 @@ namespace DB::ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; extern const int USER_EXPIRED; + extern const int NETWORK_ERROR; } namespace @@ -279,8 +281,35 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - in->enableChunked(); - out->enableChunked(); + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) + { + bool chunked_srv = chunked_srv_str.starts_with("chunked"); + bool optional_srv = chunked_srv_str.ends_with("_optional"); + bool chunked_cl = chunked_cl_str.starts_with("chunked"); + bool optional_cl = chunked_cl_str.ends_with("_optional"); + + if (optional_srv) + return chunked_cl; + if (optional_cl) + return chunked_srv; + if (chunked_cl != chunked_srv) + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: {} is {}, client requested {}", + direction, + chunked_srv ? "chunked" : "notchunked", + chunked_cl ? "chunked" : "notchunked"); + + return chunked_srv; + }; + + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked_optional"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked_optional"), proto_send_chunked_cl, "recv"); + + if (out_chunked) + out->enableChunked(); + if (in_chunked) + in->enableChunked(); } if (!is_interserver_mode) @@ -1575,6 +1604,12 @@ void TCPHandler::receiveAddendum() if (!is_interserver_mode) session->setQuotaClientKey(quota_key); + + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + readStringBinary(proto_send_chunked_cl, *in); + readStringBinary(proto_recv_chunked_cl, *in); + } } @@ -1608,6 +1643,11 @@ void TCPHandler::sendHello() writeStringBinary(server_display_name, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) writeVarUInt(VERSION_PATCH, *out); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) + { + writeStringBinary(server.config().getString("proto_caps.send", "notchunked_optional"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "notchunked_optional"), *out); + } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { auto rules = server.context()->getAccessControl().getPasswordComplexityRules(); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 67d77381167..baef92b9fa0 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -188,6 +188,8 @@ private: UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; UInt32 client_tcp_protocol_version = 0; + String proto_send_chunked_cl; + String proto_recv_chunked_cl; String quota_key; /// Connection settings, which are extracted from a context. diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d471c67553d..dd318f34148 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -273,6 +273,8 @@ ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(c address.default_database, address.user, address.password, + address.proto_send_chunked, + address.proto_recv_chunked, address.quota_key, address.cluster, address.cluster_secret, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 378b81c6d18..4475e265395 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5664,7 +5664,8 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu { auto connection = std::make_shared( node.host_name, node.port, query_context->getGlobalContext()->getCurrentDatabase(), - node.user, node.password, SSHKey(), node.quota_key, node.cluster, node.cluster_secret, + node.user, node.password, node.proto_send_chunked, node.proto_recv_chunked, + SSHKey(), node.quota_key, node.cluster, node.cluster_secret, "ParallelInsertSelectInititiator", node.compression, node.secure From 147ad42df09f374df971d6bed36ccf67c97d87a9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 24 May 2024 03:36:29 +0000 Subject: [PATCH 0102/1722] fix notchunked mode in ReadBufferFromPocoSocketChunked --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 7 +++++++ src/IO/ReadBufferFromPocoSocketChunked.h | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 4d40d8b4f14..a67a5bb41a9 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -44,6 +44,13 @@ void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callb buffer_socket.setAsyncCallback(async_callback_); } +bool ReadBufferFromPocoSocketChunked::hasBufferedData() const +{ + if (chunked) + return hasPendingData() || buffer_socket.hasPendingData(); + return hasPendingData(); +} + bool ReadBufferFromPocoSocketChunked::startChunk() { if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) < sizeof(chunk_left)) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index c70363cf7d8..b0f5dd7dc5f 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -16,7 +16,7 @@ public: bool poll(size_t timeout_microseconds); void setAsyncCallback(AsyncCallback async_callback_); - bool hasBufferedData() const { return hasPendingData() || buffer_socket.hasPendingData(); } + bool hasBufferedData() const; Poco::Net::SocketAddress peerAddress() { return peer_address; } Poco::Net::SocketAddress ourAddress() { return our_address; } From 04800f596c4471d10e15c40a533c539c6b549b06 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 24 May 2024 21:20:20 +0200 Subject: [PATCH 0103/1722] Incorporate review changes --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 60 +++++++++++++------ .../03150_infer_type_variant.reference | 29 +++++---- .../0_stateless/03150_infer_type_variant.sql | 3 +- 7 files changed, 67 insertions(+), 33 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ffc337b674f..be7564794e9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1066,7 +1066,7 @@ class IColumn; M(Bool, input_format_json_defaults_for_missing_elements_in_named_tuple, true, "Insert default value in named tuple element if it's missing in json object", 0) \ M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ - M(Bool, input_format_json_infer_variant_from_multi_type_array, false, "Try to infer variant type rather than tuple when column/array has multiple", 0) \ + M(Bool, input_format_try_infer_variants, false, "Try to infer the Variant type in text formats when there is more than one possible type for column/array elements", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 0665d1d6ca6..75fcb538b2b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,7 +93,7 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, - {"input_format_json_infer_variant_from_multi_type_array", 0, 0, "Allows inference of variant type if columns/arrays have multiple types."}, + {"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 792ac08a5df..2854802453e 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -137,7 +137,6 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.read_arrays_as_strings = settings.input_format_json_read_arrays_as_strings; format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings; format_settings.json.infer_incomplete_types_as_strings = settings.input_format_json_infer_incomplete_types_as_strings; - format_settings.json.infer_variant_from_multi_type_array = settings.input_format_json_infer_variant_from_multi_type_array; format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; @@ -266,6 +265,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth; format_settings.client_protocol_version = context->getClientProtocolVersion(); format_settings.date_time_overflow_behavior = settings.date_time_overflow_behavior; + format_settings.try_infer_variant = settings.input_format_try_infer_variants; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 927a7e691d8..a2a9e75bd44 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -36,6 +36,7 @@ struct FormatSettings bool decimal_trailing_zeros = false; bool defaults_for_omitted_fields = true; bool is_writing_to_terminal = false; + bool try_infer_variant = false; bool seekable_read = true; UInt64 max_rows_to_read_for_schema_inference = 25000; @@ -223,7 +224,6 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; - bool infer_variant_from_multi_type_array = false; bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; } json{}; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 2cbb680af97..298127cad68 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -239,6 +239,16 @@ namespace return true; } + bool checkIfTypesContainVariant(const DataTypes & types) + { + for (size_t i = 0; i < types.size(); ++i) + { + if (isVariant(types[i])) + return true; + } + return false; + } + void updateTypeIndexes(DataTypes & data_types, TypeIndexesSet & type_indexes) { type_indexes.clear(); @@ -308,20 +318,31 @@ namespace type_indexes.erase(TypeIndex::UInt64); } - /// if setting input_format_json_infer_variant_from_multi_type_array is true + /// if setting try_infer_variant is true /// and nested types are not equal then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { + auto typesAreEqual = checkIfTypesAreEqual(data_types); + auto typesContainVariant = checkIfTypesContainVariant(data_types); + if (typesAreEqual || typesContainVariant) + return; + + DataTypes new_data_types; + TypeIndexesSet new_type_indexes; + auto variant_type = std::make_shared(data_types); - /// replace separate types with a single variant type + size_t i = 0; + while (i != data_types.size()) + { + new_data_types.push_back(variant_type); + new_type_indexes.insert(TypeIndex::Variant); + i++; + } + data_types.clear(); type_indexes.clear(); - data_types.push_back(variant_type); - type_indexes.insert(TypeIndex::Variant); - - /// make the second type variant as well - data_types.push_back(variant_type); - type_indexes.insert(TypeIndex::Variant); + data_types = new_data_types; + type_indexes = new_type_indexes; } /// If we have only Date and DateTime types, convert Date to DateTime, @@ -661,16 +682,14 @@ namespace if (settings.try_infer_dates || settings.try_infer_datetimes) transformDatesAndDateTimes(data_types, type_indexes); + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); + if constexpr (!is_json) return; /// Check settings specific for JSON formats. - if (settings.json.infer_variant_from_multi_type_array) - { - transformVariant(data_types, type_indexes); - } - /// Convert numbers inferred from strings back to strings if needed. if (settings.json.try_infer_numbers_from_strings || settings.json.read_numbers_as_strings) transformJSONNumbersBackToString(data_types, settings, type_indexes, json_info); @@ -685,6 +704,10 @@ namespace if (settings.json.try_infer_objects_as_tuples) mergeJSONPaths(data_types, type_indexes, settings, json_info); + + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); + }; auto transform_complex_types = [&](DataTypes & data_types, TypeIndexesSet & type_indexes) @@ -696,14 +719,12 @@ namespace /// If there is at least one non Nothing type, change all Nothing types to it. transformNothingComplexTypes(data_types, type_indexes); + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); + if constexpr (!is_json) return; - if (settings.json.infer_variant_from_multi_type_array) - { - transformVariant(data_types, type_indexes); - } - /// Convert JSON tuples with same nested types to arrays. transformTuplesWithEqualNestedTypesToArrays(data_types, type_indexes); @@ -715,6 +736,9 @@ namespace if (json_info && json_info->allow_merging_named_tuples) mergeNamedTuples(data_types, type_indexes, settings, json_info); + + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); }; transformTypesRecursively(types, transform_simple_types, transform_complex_types); diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference index ffb4209eadb..f3f53057845 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.reference +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -2,7 +2,7 @@ ┃ arr ┃ toTypeName(arr) ┃ ┡â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ 1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple(…│ - │ │… a Int64))) │ + │ │… a Nullable(Int64)))) │ └──────────────────┴─────────────────────────────────────┘ â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ ┃ x ┃ toTypeName(x) ┃ @@ -11,12 +11,21 @@ ├───────┼────────────────────────┤ 2. │ Hello │ Variant(Int64, String) │ └───────┴────────────────────────┘ - â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ - ┃ x ┃ toTypeName(x) ┃ - ┡â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ -1. │ [1,2,3] │ Variant(Array(Int64), Tuple(…│ - │ │… a Int64)) │ - ├─────────┼──────────────────────────────┤ -2. │ (42) │ Variant(Array(Int64), Tuple(…│ - │ │… a Int64)) │ - └─────────┴──────────────────────────────┘ + â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ x ┃ toTypeName(x) ┃ + ┡â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ [1,2,3] │ Variant(Array(Nullable(Int64)), Tuple(…│ + │ │… a Nullable(Int64))) │ + ├─────────┼────────────────────────────────────────┤ +2. │ (42) │ Variant(Array(Nullable(Int64)), Tuple(…│ + │ │… a Nullable(Int64))) │ + └─────────┴────────────────────────────────────────┘ + â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ c1 ┃ toTypeName(c1) ┃ c2 ┃ toTypeName(c2) ┃ + ┡â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ 1 │ Nullable(Int64) │ Hello World! │ Nullable(String) │ + ├────┼─────────────────┼──────────────┼──────────────────┤ +2. │ 2 │ Nullable(Int64) │ [1,2,3] │ Nullable(String) │ + ├────┼─────────────────┼──────────────┼──────────────────┤ +3. │ 3 │ Nullable(Int64) │ 2020-01-01 │ Nullable(String) │ + └────┴─────────────────┴──────────────┴──────────────────┘ diff --git a/tests/queries/0_stateless/03150_infer_type_variant.sql b/tests/queries/0_stateless/03150_infer_type_variant.sql index 3253ddfe179..45126ccd471 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.sql +++ b/tests/queries/0_stateless/03150_infer_type_variant.sql @@ -1,4 +1,5 @@ -SET input_format_json_infer_variant_from_multi_type_array=1; +SET input_format_try_infer_variants=1; SELECT arr, toTypeName(arr) FROM format('JSONEachRow', '{"arr" : [1, "Hello", {"a" : 32}]}') FORMAT Pretty; SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : 42}, {"x" : "Hello"}') FORMAT Pretty; SELECT x, toTypeName(x) FROM format('JSONEachRow', '{"x" : [1, 2, 3]}, {"x" : {"a" : 42}}') FORMAT Pretty; +SELECT c1, toTypeName(c1), c2, toTypeName(c2) FROM format('CSV', '1,Hello World!\n2,"[1,2,3]"\n3,"2020-01-01"\n') FORMAT Pretty; \ No newline at end of file From 1f26281493d66f15408d8459c198170e71901f68 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 27 May 2024 15:43:14 +0200 Subject: [PATCH 0104/1722] build fix --- src/Functions/toStartOfInterval.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index a4f870613d3..c45501aa905 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -241,20 +241,20 @@ private: if (isDate(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDateTime(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(&time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone); } else if (isDateTime64(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column_col); + const auto * time_column_vec = checkAndGetColumn(&time_column_col); auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) From 89205d78a68879399129b64f78cd27f7602bf373 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 04:18:36 +0000 Subject: [PATCH 0105/1722] major refactoring --- src/Client/Connection.cpp | 2 +- src/IO/ReadBufferFromPocoSocket.cpp | 51 ++---- src/IO/ReadBufferFromPocoSocket.h | 20 ++- src/IO/ReadBufferFromPocoSocketChunked.cpp | 183 ++++++++++++--------- src/IO/ReadBufferFromPocoSocketChunked.h | 98 +++++++++-- 5 files changed, 222 insertions(+), 132 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 9327b694d29..c221124932a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1101,7 +1101,7 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || in->hasBufferedData(); + return last_input_packet_type.has_value() || in->hasPendingData(); } diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 5fb7ea0440c..5c338ef18bc 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -32,9 +32,16 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) +ssize_t ReadBufferFromPocoSocketBase::socketReceiveBytesImpl(char * ptr, size_t size) { ssize_t bytes_read = 0; + Stopwatch watch; + + SCOPE_EXIT({ + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); + }); /// Add more details to exceptions. try @@ -49,7 +56,7 @@ size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) socket.setBlocking(false); SCOPE_EXIT(socket.setBlocking(true)); bool secure = socket.secure(); - bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) @@ -61,12 +68,12 @@ size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); /// Try to read again. - bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); } } else { - bytes_read = socket.impl()->receiveBytes(begin, static_cast(size)); + bytes_read = socket.impl()->receiveBytes(ptr, static_cast(size)); } } catch (const Poco::Net::NetException & e) @@ -90,36 +97,12 @@ size_t ReadBufferFromPocoSocket::readSocket(Position begin, size_t size) return bytes_read; } -bool ReadBufferFromPocoSocket::readSocketExact(Position begin, size_t size) +bool ReadBufferFromPocoSocketBase::nextImpl() { - for (size_t bytes_left = size; bytes_left > 0;) - { - size_t ret = readSocket(begin + size - bytes_left, bytes_left); - if (ret == 0) - return false; - bytes_left -= ret; - } - - return true; -} - -bool ReadBufferFromPocoSocket::nextImpl() -{ - ssize_t bytes_read = 0; - Stopwatch watch; - - SCOPE_EXIT({ - /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one - ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); - }); - - CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - if (internal_buffer.size() > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - bytes_read = readSocket(internal_buffer.begin(), internal_buffer.size()); + ssize_t bytes_read = socketReceiveBytesImpl(internal_buffer.begin(), internal_buffer.size()); if (read_event != ProfileEvents::end()) ProfileEvents::increment(read_event, bytes_read); @@ -132,7 +115,7 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } -ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) +ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) , peer_address(socket.peerAddress()) @@ -141,13 +124,13 @@ ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, { } -ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBufferFromPocoSocket(socket_, buf_size) +ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) + : ReadBufferFromPocoSocketBase(socket_, buf_size) { read_event = read_event_; } -bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) const +bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { if (available()) return true; diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index c40a54ed7ae..a36bea6d679 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -9,7 +9,7 @@ namespace DB { /// Works with the ready Poco::Net::Socket. Blocking operations. -class ReadBufferFromPocoSocket : public BufferWithOwnMemory +class ReadBufferFromPocoSocketBase : public BufferWithOwnMemory { protected: Poco::Net::Socket & socket; @@ -25,19 +25,29 @@ protected: bool nextImpl() override; public: - explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit ReadBufferFromPocoSocketBase(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); bool poll(size_t timeout_microseconds) const; void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } - size_t readSocket(Position begin, size_t size); - bool readSocketExact(Position begin, size_t size); + ssize_t socketReceiveBytesImpl(char * ptr, size_t size); private: AsyncCallback async_callback; std::string socket_description; }; +class ReadBufferFromPocoSocket : public ReadBufferFromPocoSocketBase +{ +public: + explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, buf_size = DBMS_DEFAULT_BUFFER_SIZE) + {} + explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size) + {} +}; + } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index a67a5bb41a9..3cc8710407e 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -16,126 +16,149 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock {} ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBuffer(nullptr, 0), log(getLogger("Protocol")), peer_address(socket_.peerAddress()), our_address(socket_.address()), buffer_socket(socket_, read_event_, buf_size) + : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size), our_address(socket_.address()), log(getLogger("Protocol")) + { chassert(buf_size <= std::numeric_limits::max()); - - working_buffer = buffer_socket.buffer(); - pos = buffer_socket.position(); } void ReadBufferFromPocoSocketChunked::enableChunked() { - chunked = true; - buffer_socket.position() = pos; + if (chunked) + return; + chunked = 1; + data_end = buffer().end(); working_buffer.resize(offset()); + chunk_left = 0; + next_chunk = 0; } -bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) -{ - if (!chunked) - buffer_socket.position() = pos; - - return buffer_socket.poll(timeout_microseconds); -} - -void ReadBufferFromPocoSocketChunked::setAsyncCallback(AsyncCallback async_callback_) -{ - buffer_socket.setAsyncCallback(async_callback_); -} - -bool ReadBufferFromPocoSocketChunked::hasBufferedData() const +bool ReadBufferFromPocoSocketChunked::hasPendingData() const { if (chunked) - return hasPendingData() || buffer_socket.hasPendingData(); - return hasPendingData(); + return available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk); + + return ReadBufferFromPocoSocketBase::hasPendingData(); } -bool ReadBufferFromPocoSocketChunked::startChunk() +bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const { - if (buffer_socket.read(reinterpret_cast(&chunk_left), sizeof(chunk_left)) < sizeof(chunk_left)) - return false; - if (chunk_left == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + if (chunked) + if (available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk)) + return true; - chunk_left = fromLittleEndian(chunk_left); - - return nextChunk(); + return ReadBufferFromPocoSocketBase::poll(timeout_microseconds); } -bool ReadBufferFromPocoSocketChunked::nextChunk() -{ - if (chunk_left == 0) - { - started = true; - return startChunk(); - } - if (buffer_socket.available() == 0) - if (!buffer_socket.next()) +bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) +{ + auto buffered = std::min(static_cast(data_end - c_pos), sizeof(next_chunk)); + + if (buffered) + std::memcpy(&next_chunk, c_pos, buffered); + if (buffered < sizeof(next_chunk)) + if (socketReceiveBytesImpl(reinterpret_cast(&next_chunk) + buffered, sizeof(next_chunk) - buffered) < static_cast(sizeof(next_chunk) - buffered)) return false; - if (started) - LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*buffer_socket.position()), chunk_left); - else - LOG_TEST(log, "Packet receive continued. Size {}", chunk_left); + next_chunk = fromLittleEndian(next_chunk); - started = false; - - nextimpl_working_buffer_offset = buffer_socket.offset(); - - if (buffer_socket.available() < chunk_left) + if (next_chunk) { - working_buffer.resize(buffer_socket.offset() + buffer_socket.available()); - chunk_left -= buffer_socket.available(); - buffer_socket.position() += buffer_socket.available(); + if (cont) + LOG_TEST(log, "Packet receive continued. Size {}", next_chunk); + } + else + LOG_TEST(log, "Packet receive ended."); + + return true; +} + +bool ReadBufferFromPocoSocketChunked::process_chunk_left(Position c_pos) +{ + if (data_end - c_pos < chunk_left) + { + working_buffer.resize(data_end - buffer().begin()); + nextimpl_working_buffer_offset = c_pos - buffer().begin(); + chunk_left -= (data_end - c_pos); return true; } - working_buffer.resize(buffer_socket.offset() + chunk_left); - UInt8 buffered = std::min(static_cast(4), buffer_socket.available() - chunk_left); + nextimpl_working_buffer_offset = c_pos - buffer().begin(); + working_buffer.resize(nextimpl_working_buffer_offset + chunk_left); - buffer_socket.position() += chunk_left; - if (buffered > 0) - std::memcpy(&chunk_left, buffer_socket.position(), buffered); - buffer_socket.position() += buffered; + c_pos += chunk_left; - if (4 > buffered) - if (!buffer_socket.readSocketExact(reinterpret_cast(&chunk_left) + buffered, 4 - buffered)) - return false; - - chunk_left = fromLittleEndian(chunk_left); - - if (chunk_left == 0) - LOG_TEST(log, "Packet receive ended."); + if (!load_next_chunk(c_pos, true)) + return false; + chunk_left = 0; return true; } bool ReadBufferFromPocoSocketChunked::nextImpl() { - if (chunked) + if (!chunked) + return ReadBufferFromPocoSocketBase::nextImpl(); + + auto c_pos = pos; + + if (chunk_left == 0) { - if (!nextChunk()) + if (next_chunk == 0) { - pos = buffer_socket.position(); - return false; + if (chunked == 1) + chunked = 2; // first chunked block - no end marker + else + c_pos = pos + sizeof(next_chunk); // bypass chunk end marker + + if (c_pos > data_end) + c_pos = data_end; + + if (!load_next_chunk(c_pos)) + return false; + + chunk_left = next_chunk; + next_chunk = 0; + + c_pos += sizeof(next_chunk); + + if (c_pos >= data_end) + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*c_pos), chunk_left); + } + else + { + c_pos += sizeof(next_chunk); + if (c_pos >= data_end) + { + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); + } + + chunk_left = next_chunk; + next_chunk = 0; } - return true; } - - buffer_socket.position() = pos; - - if (!buffer_socket.next()) + else { - pos = buffer_socket.position(); - return false; + chassert(c_pos == data_end); + + if (!ReadBufferFromPocoSocketBase::nextImpl()) + return false; + data_end = buffer().end(); + c_pos = buffer().begin(); } - pos = buffer_socket.position(); - working_buffer.resize(offset() + buffer_socket.available()); - - return true; + return process_chunk_left(c_pos); } } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index b0f5dd7dc5f..851a90042ac 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -3,37 +3,111 @@ #include #include +/* + +Handshake +============= + | 'Hello' type + | handshake exchange + | chunked protocol negotiation + +============= + + +Basic chunk: + +============= +Chunk begins | 0x12345678 chunk size, 4 bytes little endian + +------------- + | Packet type always follows beginning of the chunk + | packet data + +------------- +Chunk ends | 0x00000000 4 zero bytes + +============= + + + + +Datastream chunk: + +============= +Chunk begins | 0x12345678 + +------------- + | Packet type + | packet data + +------------- + | Packet type + | packet data + +------------- +...arbitrary number ..... +of packets... ..... + +------------- + | Packet type + | packet data + +------------- +Chunk ends | 0x00000000 + +============= + + + +Multipart chunk: + +============= +Chunk begins | 0x12345678 chunk part size, 4 bytes little endian + +------------- + | Packet type + | packet data + +------------- + | Packet type + | (partial) packet data + +============= +Chunk continues | 0x12345678 chunk next part size, 4 bytes little endian + +============= + | possibly previous packet's data + +------------- + | Packet type + | packet data + +------------- +...arbitrary number ..... +of chunk parts... ..... + +------------- + | Packet type + | packet data + +------------- +Chunk ends | 0x00000000 + +============= + +*/ + namespace DB { -class ReadBufferFromPocoSocketChunked: public ReadBuffer +class ReadBufferFromPocoSocketChunked: public ReadBufferFromPocoSocketBase { public: + using ReadBufferFromPocoSocketBase::setAsyncCallback; + explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); explicit ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); void enableChunked(); - bool poll(size_t timeout_microseconds); - void setAsyncCallback(AsyncCallback async_callback_); - bool hasBufferedData() const; + bool hasPendingData() const; + + bool poll(size_t timeout_microseconds) const; Poco::Net::SocketAddress peerAddress() { return peer_address; } Poco::Net::SocketAddress ourAddress() { return our_address; } protected: - bool startChunk(); - bool nextChunk(); + bool load_next_chunk(Position c_pos, bool cont = false); + bool process_chunk_left(Position c_pos); bool nextImpl() override; +protected: + Poco::Net::SocketAddress our_address; + private: LoggerPtr log; - Poco::Net::SocketAddress peer_address; - Poco::Net::SocketAddress our_address; - ReadBufferFromPocoSocket buffer_socket; - bool chunked = false; - UInt32 chunk_left = 0; // chunk left to read from socket - bool started = false; + Position data_end = nullptr; // end position of data in the internal_buffer + UInt32 chunk_left = 0; // chunk left to read from socket + UInt32 next_chunk = 0; // size of the next cnunk + UInt8 chunked = 0; // 0 - disabled; 1 - started; 2 - enabled; }; } From 4545f3af52d8046cd2a1b54fc22fd0d592a48a31 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 04:35:01 +0000 Subject: [PATCH 0106/1722] fix --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 3cc8710407e..59c56b9d008 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -121,6 +121,9 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() chunk_left = next_chunk; next_chunk = 0; + if (chunk_left == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: empty chunk received"); + c_pos += sizeof(next_chunk); if (c_pos >= data_end) From d1bc58f23254ca781b6645bafb9c7cdf00326a04 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 05:05:18 +0000 Subject: [PATCH 0107/1722] fix --- src/IO/ReadBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 5c338ef18bc..af58efc7e10 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -43,6 +43,8 @@ ssize_t ReadBufferFromPocoSocketBase::socketReceiveBytesImpl(char * ptr, size_t ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); }); + CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); + /// Add more details to exceptions. try { From 1dc381dbc1f0b7b53d8707b9515a0d3f6ad3f442 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 05:07:40 +0000 Subject: [PATCH 0108/1722] fix --- src/IO/ReadBufferFromPocoSocketChunked.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 851a90042ac..749ee042a7c 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -23,8 +23,6 @@ Chunk ends | 0x00000000 4 zero bytes +============= - - Datastream chunk: +============= Chunk begins | 0x12345678 @@ -45,7 +43,6 @@ Chunk ends | 0x00000000 +============= - Multipart chunk: +============= Chunk begins | 0x12345678 chunk part size, 4 bytes little endian From fdccba97a3c7d1097034bc6b0994b7f37bc5721e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 06:35:04 +0000 Subject: [PATCH 0109/1722] set chunked for testing --- src/Client/ConnectionParameters.cpp | 4 ++-- src/Server/TCPHandler.cpp | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 430c462084a..b6ed242acd4 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -103,8 +103,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati #endif } - proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); - proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); + proto_send_chunked = config.getString("proto_caps.send", "chunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); quota_key = config.getString("quota_key", ""); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c7db25c4c3a..47e5f982a93 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -303,8 +303,8 @@ void TCPHandler::runImpl() return chunked_srv; }; - bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked_optional"), proto_recv_chunked_cl, "send"); - bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked_optional"), proto_send_chunked_cl, "recv"); + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked"), proto_send_chunked_cl, "recv"); if (out_chunked) out->enableChunked(); @@ -1645,8 +1645,8 @@ void TCPHandler::sendHello() writeVarUInt(VERSION_PATCH, *out); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - writeStringBinary(server.config().getString("proto_caps.send", "notchunked_optional"), *out); - writeStringBinary(server.config().getString("proto_caps.recv", "notchunked_optional"), *out); + writeStringBinary(server.config().getString("proto_caps.send", "chunked"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "chunked"), *out); } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { From e3d57ab117391c3b99a8937783320a8c59e0b196 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 31 May 2024 16:05:42 +0000 Subject: [PATCH 0110/1722] set default protocol to notchunked_optional for cluster clients --- src/Interpreters/Cluster.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index c993af5fc5e..f3146ac0134 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,8 +114,8 @@ public: UInt16 port{0}; String user; String password; - String proto_send_chunked; - String proto_recv_chunked; + String proto_send_chunked = "notchunked_optional"; + String proto_recv_chunked = "notchunked_optional"; String quota_key; /// For inter-server authorization From f11f41491087099c63ee9f98b6bf8a27a8e87ed9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 2 Jun 2024 07:25:48 +0000 Subject: [PATCH 0111/1722] fix special case of testing feature for chunked protocol --- src/Server/TCPHandler.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 47e5f982a93..da276e1c404 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2268,16 +2268,26 @@ void TCPHandler::sendData(const Block & block) } writeVarUInt(Protocol::Server::Data, *out); - /// Send external table name (empty name is the main table) - writeStringBinary("", *out); /// For testing hedged requests if (block.rows() > 0 && query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()) { + /// This strange sequence is needed in case of chunked protocol is enabled, in order for client not to + /// hang on recieving of at least packet type - chunk will not be processed unless either chunk footer + /// or chunk continuation header is recieved - first 'next' is sending starting chunk containing packet type + /// and second 'next' is sending chunk continuation header. + out->next(); + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); out->next(); std::chrono::milliseconds ms(query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()); std::this_thread::sleep_for(ms); } + else + { + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); + } state.block_out->write(block); state.maybe_compressed_out->next(); From eaeabd8d374e2e28a6208fb9ea1ea7835676c7e5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 2 Jun 2024 13:03:48 +0000 Subject: [PATCH 0112/1722] fix typos --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index da276e1c404..1a64ec1dd10 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2273,8 +2273,8 @@ void TCPHandler::sendData(const Block & block) if (block.rows() > 0 && query_context->getSettingsRef().sleep_in_send_data_ms.totalMilliseconds()) { /// This strange sequence is needed in case of chunked protocol is enabled, in order for client not to - /// hang on recieving of at least packet type - chunk will not be processed unless either chunk footer - /// or chunk continuation header is recieved - first 'next' is sending starting chunk containing packet type + /// hang on receiving of at least packet type - chunk will not be processed unless either chunk footer + /// or chunk continuation header is received - first 'next' is sending starting chunk containing packet type /// and second 'next' is sending chunk continuation header. out->next(); /// Send external table name (empty name is the main table) From e0be652f4de803198b406dcbda5b1f1ac6938a9c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 3 Jun 2024 07:24:28 +0000 Subject: [PATCH 0113/1722] fix test, better log, fix defaults for client --- src/Client/ConnectionParameters.cpp | 4 ++-- src/Client/ConnectionParameters.h | 4 ++-- src/IO/ReadBufferFromPocoSocketChunked.cpp | 6 +++--- src/IO/WriteBufferFromPocoSocketChunked.h | 6 +++--- tests/integration/test_hedged_requests/test.py | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index b6ed242acd4..430c462084a 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -103,8 +103,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati #endif } - proto_send_chunked = config.getString("proto_caps.send", "chunked"); - proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); + proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); quota_key = config.getString("quota_key", ""); diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index 85174924016..52fe7bd9b2b 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,8 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; - std::string proto_send_chunked; - std::string proto_recv_chunked; + std::string proto_send_chunked = "notchunked_optional"; + std::string proto_recv_chunked = "notchunked_optional"; std::string quota_key; SSHKey ssh_private_key; Protocol::Secure security = Protocol::Secure::Disable; diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 59c56b9d008..328b70bdb9b 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -65,10 +65,10 @@ bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) if (next_chunk) { if (cont) - LOG_TEST(log, "Packet receive continued. Size {}", next_chunk); + LOG_TEST(log, "{} <- {} Chunk receive continued. Size {}", ourAddress().toString(), peerAddress().toString(), next_chunk); } else - LOG_TEST(log, "Packet receive ended."); + LOG_TEST(log, "{} <- {} Chunk receive ended.", ourAddress().toString(), peerAddress().toString()); return true; } @@ -134,7 +134,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() c_pos = buffer().begin(); } - LOG_TEST(log, "Packet receive started. Message {}, size {}", static_cast(*c_pos), chunk_left); + LOG_TEST(log, "{} <- {} Chunk receive started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*c_pos), chunk_left); } else { diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 6c35db62c0c..7c6ab53dc91 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -30,7 +30,7 @@ public: if (finished) throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); - LOG_TEST(log, "Packet send ended."); + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); finished = true; UInt32 s = 0; @@ -43,9 +43,9 @@ protected: { UInt32 s = static_cast(offset()); if (finished) - LOG_TEST(log, "Packet send started. Message {}, size {}", static_cast(*buffer().begin()), s); + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*buffer().begin()), s); else - LOG_TEST(log, "Packet send continued. Size {}", s); + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), s); finished = false; s = toLittleEndian(s); diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 02ecf3c1367..0d72f7c45b1 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -333,7 +333,7 @@ def test_receive_timeout2(started_cluster): # in packet receiving but there are replicas in process of # connection establishing. update_configs( - node_1_sleep_in_send_data=4000, + node_1_sleep_in_send_data=5000, node_2_sleep_in_send_tables_status=2000, node_3_sleep_in_send_tables_status=2000, ) From 66e387562659e9712088e09427d4c050e9f22c1f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 3 Jun 2024 09:55:51 +0000 Subject: [PATCH 0114/1722] fix tidy build --- src/IO/ReadBufferFromPocoSocket.h | 2 +- src/IO/ReadBufferFromPocoSocketChunked.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index a36bea6d679..912388adaac 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -43,7 +43,7 @@ class ReadBufferFromPocoSocket : public ReadBufferFromPocoSocketBase { public: explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) - : ReadBufferFromPocoSocketBase(socket_, buf_size = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBufferFromPocoSocketBase(socket_, buf_size) {} explicit ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 749ee042a7c..acf0edafe0a 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -96,7 +96,6 @@ protected: bool process_chunk_left(Position c_pos); bool nextImpl() override; -protected: Poco::Net::SocketAddress our_address; private: From 1cda4596adfc9ca384a28da80a91159641952e36 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 3 Jun 2024 11:51:01 +0000 Subject: [PATCH 0115/1722] fix tidy build --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 328b70bdb9b..6ed6b63289c 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -101,7 +101,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() if (!chunked) return ReadBufferFromPocoSocketBase::nextImpl(); - auto c_pos = pos; + auto * c_pos = pos; if (chunk_left == 0) { From 655262d1a1b21d85f4fbe284e0835065bcca379b Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 5 Jun 2024 14:34:17 +0200 Subject: [PATCH 0116/1722] Fix issue with nullables --- src/Formats/SchemaInferenceUtils.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 298127cad68..43120cb7b22 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1547,11 +1547,13 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) DataTypes nested_types; for (const auto & nested_type: variant_type->getVariants()) { - /// unlike tuple or array, here we do not want to make any of the variants nullable - /// so we do not call makeNullableRecursively - nested_types.push_back(nested_type); + auto is_low_cardinality = nested_type->lowCardinality(); + auto has_sub_types = nested_type->haveSubtypes(); + if (!is_low_cardinality && has_sub_types) + nested_types.push_back(makeNullableRecursively(nested_type)); + else + nested_types.push_back(nested_type); } - return std::make_shared(nested_types); } From 9a7888ba516420c00978901bbe5b85cafe6e7bdf Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 5 Jun 2024 16:39:54 +0200 Subject: [PATCH 0117/1722] fix tests --- src/Functions/toStartOfInterval.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index c45501aa905..b98d78171ae 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -376,7 +376,7 @@ private: /// If we have a time argument that has bigger scale than the interval can contain and interval is not default, we need /// to return a value with bigger precision and thus we should multiply result on the scale difference. result_data[i] = 0; - result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_endtime)) * scale_diff; + result_data[i] += static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_endtime)); } } else From 418fc7f4438abd25eae4928f36ff0c3fef2395f8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 6 Jun 2024 10:02:19 +0200 Subject: [PATCH 0118/1722] Fix incorrect inference for other formats --- src/Formats/SchemaInferenceUtils.cpp | 29 +++++++++-- .../03150_infer_type_variant.reference | 48 +++++++++---------- 2 files changed, 49 insertions(+), 28 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 43120cb7b22..a8b5d4343f5 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -318,19 +318,40 @@ namespace type_indexes.erase(TypeIndex::UInt64); } - /// if setting try_infer_variant is true - /// and nested types are not equal then we convert to type variant. + /// if setting 'try_infer_variant' is true then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { auto typesAreEqual = checkIfTypesAreEqual(data_types); auto typesContainVariant = checkIfTypesContainVariant(data_types); - if (typesAreEqual || typesContainVariant) + if (typesAreEqual) return; DataTypes new_data_types; TypeIndexesSet new_type_indexes; + std::shared_ptr variant_type; + + /// extract the nested types of variant and make a new variant with the nested types and the other type. + /// eg. Type 1: variant, Type 2: Date -> variant. + if (typesContainVariant) + { + DataTypes extracted_types; + for (size_t i=0; i(data_types[i].get())) + extracted_types = variant->getVariants(); + } + else + extracted_types.push_back(data_types[i]); + } + variant_type = std::make_shared(extracted_types); + } + else + { + variant_type = std::make_shared(data_types); + } - auto variant_type = std::make_shared(data_types); size_t i = 0; while (i != data_types.size()) { diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference index f3f53057845..a5f56cb3618 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.reference +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -1,9 +1,9 @@ - â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ - ┃ arr ┃ toTypeName(arr) ┃ - ┡â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ -1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple(…│ - │ │… a Nullable(Int64)))) │ - └──────────────────┴─────────────────────────────────────┘ + â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ arr ┃ toTypeName(arr) ┃ + ┡â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple( + a Nullable(Int64)))) │ + └──────────────────┴─────────────────────────────────────────────────────────────┘ â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ ┃ x ┃ toTypeName(x) ┃ ┡â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ @@ -11,21 +11,21 @@ ├───────┼────────────────────────┤ 2. │ Hello │ Variant(Int64, String) │ └───────┴────────────────────────┘ - â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ - ┃ x ┃ toTypeName(x) ┃ - ┡â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ -1. │ [1,2,3] │ Variant(Array(Nullable(Int64)), Tuple(…│ - │ │… a Nullable(Int64))) │ - ├─────────┼────────────────────────────────────────┤ -2. │ (42) │ Variant(Array(Nullable(Int64)), Tuple(…│ - │ │… a Nullable(Int64))) │ - └─────────┴────────────────────────────────────────┘ - â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ - ┃ c1 ┃ toTypeName(c1) ┃ c2 ┃ toTypeName(c2) ┃ - ┡â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ -1. │ 1 │ Nullable(Int64) │ Hello World! │ Nullable(String) │ - ├────┼─────────────────┼──────────────┼──────────────────┤ -2. │ 2 │ Nullable(Int64) │ [1,2,3] │ Nullable(String) │ - ├────┼─────────────────┼──────────────┼──────────────────┤ -3. │ 3 │ Nullable(Int64) │ 2020-01-01 │ Nullable(String) │ - └────┴─────────────────┴──────────────┴──────────────────┘ + â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ x ┃ toTypeName(x) ┃ + ┡â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ [1,2,3] │ Variant(Array(Nullable(Int64)), Tuple( + a Nullable(Int64))) │ + ├─────────┼───────────────────────────────────────────────────────────────┤ +2. │ (42) │ Variant(Array(Nullable(Int64)), Tuple( + a Nullable(Int64))) │ + └─────────┴───────────────────────────────────────────────────────────────┘ + â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ c1 ┃ toTypeName(c1) ┃ c2 ┃ toTypeName(c2) ┃ + ┡â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ 1 │ Nullable(Int64) │ Hello World! │ Variant(Array(Nullable(Int64)), String) │ + ├────┼─────────────────┼──────────────┼─────────────────────────────────────────┤ +2. │ 2 │ Nullable(Int64) │ [1,2,3] │ Variant(Array(Nullable(Int64)), String) │ + ├────┼─────────────────┼──────────────┼─────────────────────────────────────────┤ +3. │ 3 │ Nullable(Int64) │ 2020-01-01 │ Variant(Array(Nullable(Int64)), String) │ + └────┴─────────────────┴──────────────┴─────────────────────────────────────────┘ From 63b0d13a62c84245a3efe6e3cd9cf08cd2a588a8 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 6 Jun 2024 16:15:35 +0000 Subject: [PATCH 0119/1722] trying to fix tests --- src/Functions/toStartOfInterval.cpp | 2 -- .../02207_subseconds_intervals.reference | 16 ++++++++-------- ...6_fix_to_start_of_milli_microsecond.reference | 12 ++++++------ 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index b98d78171ae..e358addf972 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -192,8 +192,6 @@ public: case ResultType::DateTime64: { UInt32 scale = 0; - if (isDateTime64(arguments[0].type)) - scale = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Kind::Nanosecond) scale = (9 > scale) ? 9 : scale; else if (interval_type->getKind() == IntervalKind::Kind::Microsecond) diff --git a/tests/queries/0_stateless/02207_subseconds_intervals.reference b/tests/queries/0_stateless/02207_subseconds_intervals.reference index 6cde773c3c4..7e6d64b6b9f 100644 --- a/tests/queries/0_stateless/02207_subseconds_intervals.reference +++ b/tests/queries/0_stateless/02207_subseconds_intervals.reference @@ -10,25 +10,25 @@ test intervals - test microseconds 1980-12-12 12:12:12.123456 1980-12-12 12:12:12.123400 -1980-12-12 12:12:12.12345700 -1980-12-12 12:12:12.12345700 +1980-12-12 12:12:12.123457 +1980-12-12 12:12:12.123457 1930-12-12 12:12:12.123456 1930-12-12 12:12:12.123400 -1930-12-12 12:12:12.12345600 +1930-12-12 12:12:12.123456 2220-12-12 12:12:12.123456 2220-12-12 12:12:12.123400 -2220-12-12 12:12:12.12345700 +2220-12-12 12:12:12.123457 - test milliseconds 1980-12-12 12:12:12.123 1980-12-12 12:12:12.120 -1980-12-12 12:12:12.123000 -1980-12-12 12:12:12.123000 +1980-12-12 12:12:12.123 +1980-12-12 12:12:12.123 1930-12-12 12:12:12.123 1930-12-12 12:12:12.120 -1930-12-12 12:12:12.123000 +1930-12-12 12:12:12.123 2220-12-12 12:12:12.123 2220-12-12 12:12:12.120 -2220-12-12 12:12:12.123000 +2220-12-12 12:12:12.123 test add[...]seconds() - test nanoseconds 1980-12-12 12:12:12.123456790 diff --git a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference index 95a05a24981..413c79828c7 100644 --- a/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference +++ b/tests/queries/0_stateless/02956_fix_to_start_of_milli_microsecond.reference @@ -1,7 +1,7 @@ -2023-10-09 10:11:12.001000 -2023-10-09 10:11:12.001000 -2023-10-09 10:11:12.000000 -2023-10-09 10:11:12.000000 -2023-10-09 00:00:00.000000000 -2023-10-09 00:00:00.000000000 +2023-10-09 10:11:12.001 +2023-10-09 10:11:12.001 +2023-10-09 10:11:12.000 +2023-10-09 10:11:12.000 +2023-10-09 00:00:00.000000 +2023-10-09 00:00:00.000 2023-10-09 00:00:00 From c5ba8eb477fb168d38dd1493b2b20a121a3ed1fd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 6 Jun 2024 16:46:38 +0000 Subject: [PATCH 0120/1722] fix for origin --- src/Functions/toStartOfInterval.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index e358addf972..cc5ffd56976 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -192,6 +192,8 @@ public: case ResultType::DateTime64: { UInt32 scale = 0; + if (isDateTime64(arguments[0].type) && overload == Overload::Origin) + scale = assert_cast(*arguments[0].type.get()).getScale(); if (interval_type->getKind() == IntervalKind::Kind::Nanosecond) scale = (9 > scale) ? 9 : scale; else if (interval_type->getKind() == IntervalKind::Kind::Microsecond) From a562118d2a5b66955f44d393949eccb0e8c3b8b7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 01:45:56 +0000 Subject: [PATCH 0121/1722] major refactoring of chunked write buffer - more buffering, some bugs fixed --- src/Client/Connection.cpp | 23 +++-- src/IO/ReadBufferFromPocoSocketChunked.cpp | 2 - src/IO/WriteBuffer.h | 8 +- src/IO/WriteBufferFromPocoSocketChunked.h | 114 +++++++++++++++++---- src/Server/TCPHandler.cpp | 38 +++---- 5 files changed, 134 insertions(+), 51 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index c221124932a..9f727b974ee 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -625,7 +625,7 @@ bool Connection::ping(const ConnectionTimeouts & timeouts) UInt64 pong = 0; writeVarUInt(Protocol::Client::Ping, *out); - out->finishPacket(); + out->finishChunk(); out->next(); if (in->eof()) @@ -675,7 +675,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); - out->finishPacket(); + out->finishChunk(); out->next(); UInt64 response_type = 0; @@ -827,7 +827,7 @@ void Connection::sendQuery( block_profile_events_in.reset(); block_out.reset(); - out->finishPacket(); + out->finishChunk(); /// Send empty block which means end of data. if (!with_pending_data) @@ -845,7 +845,7 @@ void Connection::sendCancel() return; writeVarUInt(Protocol::Client::Cancel, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -871,9 +871,10 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) size_t prev_bytes = out->count(); block_out->write(block); - maybe_compressed_out->next(); + if (maybe_compressed_out != out) + maybe_compressed_out->next(); if (!block) - out->finishPacket(); + out->finishChunk(); out->next(); if (throttler) @@ -884,7 +885,7 @@ void Connection::sendIgnoredPartUUIDs(const std::vector & uuids) { writeVarUInt(Protocol::Client::IgnoredPartUUIDs, *out); writeVectorBinary(uuids, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -894,7 +895,7 @@ void Connection::sendReadTaskResponse(const String & response) writeVarUInt(Protocol::Client::ReadTaskResponse, *out); writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out); writeStringBinary(response, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -903,7 +904,7 @@ void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & resp { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); response.serialize(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -922,7 +923,7 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String else copyData(input, *out, size); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -951,7 +952,7 @@ void Connection::sendScalarsData(Scalars & data) sendData(elem.second, elem.first, true /* scalar */); } - out->finishPacket(); + out->finishChunk(); out_bytes = out->count() - out_bytes; maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes; diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 6ed6b63289c..798be547e99 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -153,8 +153,6 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() } else { - chassert(c_pos == data_end); - if (!ReadBufferFromPocoSocketBase::nextImpl()) return false; data_end = buffer().end(); diff --git a/src/IO/WriteBuffer.h b/src/IO/WriteBuffer.h index 1ceb938e454..bb3200d2e54 100644 --- a/src/IO/WriteBuffer.h +++ b/src/IO/WriteBuffer.h @@ -63,7 +63,8 @@ public: } bytes += bytes_in_buffer; - pos = working_buffer.begin(); + pos = working_buffer.begin() + nextimpl_working_buffer_offset; + nextimpl_working_buffer_offset = 0; } /// Calling finalize() in the destructor of derived classes is a bad practice. @@ -152,6 +153,11 @@ protected: bool finalized = false; + /// The number of bytes to preserve from the initial position of `working_buffer` + /// buffer. Apparently this is an additional out-parameter for nextImpl(), + /// not a real field. + size_t nextimpl_working_buffer_offset = 0; + private: /** Write the data in the buffer (from the beginning of the buffer to the current position). * Throw an exception if something is wrong. diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 7c6ab53dc91..3fe39487923 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -19,40 +19,114 @@ public: explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} - void enableChunked() { chunked = true; } - void finishPacket() + void enableChunked() + { + chunked = true; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + } + + void finishChunk() { if (!chunked) return; - next(); - - if (finished) + if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); - LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); - finished = true; + /// Fill up current chunk size + *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); - UInt32 s = 0; - socketSendBytes(reinterpret_cast(&s), sizeof(s)); + if (!chunk_started) + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + else + chunk_started = false; + + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + + if (available() < sizeof(*chunk_size_ptr)) + { + finishing = available(); + pos += available(); + chunk_size_ptr = reinterpret_cast(pos); + return; + } + + /// Buffer end-of-chunk + *reinterpret_cast(pos) = 0; + pos += sizeof(*chunk_size_ptr); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); } + protected: void nextImpl() override { - if (chunked) - { - UInt32 s = static_cast(offset()); - if (finished) - LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", ourAddress().toString(), peerAddress().toString(), static_cast(*buffer().begin()), s); - else - LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), s); + if (!chunked) + return WriteBufferFromPocoSocket::nextImpl(); - finished = false; - s = toLittleEndian(s); + if (finishing < sizeof(*chunk_size_ptr)) + { + pos -= finishing; + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Send end-of-chunk directly + UInt32 s = 0; socketSendBytes(reinterpret_cast(&s), sizeof(s)); + + finishing = sizeof(*chunk_size_ptr); + + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + return; } + if (offset() == sizeof(*chunk_size_ptr)) // prevent sending empty chunk + { + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + return; + } + + if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) + { + pos = reinterpret_cast(chunk_size_ptr); + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + return; + } + + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + else /// Fill up current chunk size + { + *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); + if (!chunk_started) + { + chunk_started = true; + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + } + else + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + /// Send current chunk WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); } Poco::Net::SocketAddress peerAddress() @@ -67,7 +141,9 @@ protected: private: LoggerPtr log; bool chunked = false; - bool finished = true; + bool chunk_started = false; // chunk started flag + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker }; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1a64ec1dd10..89ad8e856d5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1188,7 +1188,7 @@ void TCPHandler::processTablesStatusRequest() response.write(*out, client_tcp_protocol_version); - out->finishPacket(); + out->finishChunk(); } void TCPHandler::receiveUnexpectedTablesStatusRequest() @@ -1210,7 +1210,7 @@ void TCPHandler::sendPartUUIDs() writeVarUInt(Protocol::Server::PartUUIDs, *out); writeVectorBinary(uuids, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } } @@ -1220,7 +1220,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() { writeVarUInt(Protocol::Server::ReadTaskRequest, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1230,7 +1230,7 @@ void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRanges writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); announcement.serialize(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1240,7 +1240,7 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); request.serialize(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1250,7 +1250,7 @@ void TCPHandler::sendProfileInfo(const ProfileInfo & info) writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1267,7 +1267,7 @@ void TCPHandler::sendTotals(const Block & totals) state.block_out->write(totals); state.maybe_compressed_out->next(); - out->finishPacket(); + out->finishChunk(); out->next(); } } @@ -1285,7 +1285,7 @@ void TCPHandler::sendExtremes(const Block & extremes) state.block_out->write(extremes); state.maybe_compressed_out->next(); - out->finishPacket(); + out->finishChunk(); out->next(); } } @@ -1304,7 +1304,7 @@ void TCPHandler::sendProfileEvents() state.profile_events_block_out->write(block); - out->finishPacket(); + out->finishChunk(); out->next(); auto elapsed_milliseconds = stopwatch.elapsedMilliseconds(); @@ -1343,7 +1343,7 @@ void TCPHandler::sendTimezone() writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -1700,7 +1700,7 @@ bool TCPHandler::receivePacket() case Protocol::Client::Ping: writeVarUInt(Protocol::Server::Pong, *out); - out->finishPacket(); + out->finishChunk(); out->next(); return false; @@ -2290,9 +2290,11 @@ void TCPHandler::sendData(const Block & block) } state.block_out->write(block); - state.maybe_compressed_out->next(); - out->finishPacket(); + if (state.maybe_compressed_out != out) + state.maybe_compressed_out->next(); + + out->finishChunk(); out->next(); } catch (...) @@ -2329,7 +2331,7 @@ void TCPHandler::sendLogData(const Block & block) state.logs_block_out->write(block); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2341,7 +2343,7 @@ void TCPHandler::sendTableColumns(const ColumnsDescription & columns) writeStringBinary("", *out); writeStringBinary(columns.toString(), *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2352,7 +2354,7 @@ void TCPHandler::sendException(const Exception & e, bool with_stack_trace) writeVarUInt(Protocol::Server::Exception, *out); writeException(e, *out, with_stack_trace); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2364,7 +2366,7 @@ void TCPHandler::sendEndOfStream() writeVarUInt(Protocol::Server::EndOfStream, *out); - out->finishPacket(); + out->finishChunk(); out->next(); } @@ -2384,7 +2386,7 @@ void TCPHandler::sendProgress() state.prev_elapsed_ns = current_elapsed_ns; increment.write(*out, client_tcp_protocol_version); - out->finishPacket(); + out->finishChunk(); out->next(); } From 390a2a2488bdd20a87400ec3f5851dfde0f1bac0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 02:06:26 +0000 Subject: [PATCH 0122/1722] fix style --- src/IO/WriteBufferFromPocoSocketChunked.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 3fe39487923..9a9d53a1f30 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -142,7 +142,7 @@ private: LoggerPtr log; bool chunked = false; bool chunk_started = false; // chunk started flag - UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker }; From 11d9f7d51b2cd658c495adb11c3b32f6fc5a8cc6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 12:07:35 +0000 Subject: [PATCH 0123/1722] allow to set end-of-chunk marker on sent chunk, ignore duplicate finish chunk --- src/IO/WriteBufferFromPocoSocketChunked.h | 51 +++++++++++++++++++++-- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 9a9d53a1f30..40a89416f84 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -1,5 +1,6 @@ #pragma once +#include "base/defines.h" #include #include #include @@ -33,7 +34,26 @@ public: return; if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Native protocol: attempt to send empty chunk"); + { + if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk + return; + + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), + /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. + /// We don't need to wary about if it's the end of the buffer because next() always sends the whole buffer + /// so it should be a beginning of the buffer. + + chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); + + *chunk_size_ptr = 0; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; + + return; + } /// Fill up current chunk size *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); @@ -62,6 +82,8 @@ public: /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; } protected: @@ -70,6 +92,7 @@ protected: if (!chunked) return WriteBufferFromPocoSocket::nextImpl(); + /// next() after finishChunk ar the end of the buffer if (finishing < sizeof(*chunk_size_ptr)) { pos -= finishing; @@ -85,15 +108,34 @@ protected: chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + last_finish_chunk = chunk_size_ptr; + return; } - if (offset() == sizeof(*chunk_size_ptr)) // prevent sending empty chunk + /// Send end-of-chunk buffered by finishChunk + if (offset() == 2 * sizeof(*chunk_size_ptr)) + { + pos -= sizeof(*chunk_size_ptr); + /// Send end-of-chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Prevent sending empty chunk + if (offset() == sizeof(*chunk_size_ptr)) { nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); return; } + /// Finish chunk at the end of the buffer if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) { pos = reinterpret_cast(chunk_size_ptr); @@ -106,9 +148,9 @@ protected: return; } - if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk pos -= sizeof(*chunk_size_ptr); - else /// Fill up current chunk size + else // fill up current chunk size { *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); if (!chunk_started) @@ -141,6 +183,7 @@ protected: private: LoggerPtr log; bool chunked = false; + UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk bool chunk_started = false; // chunk started flag UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker From d2dd640beb3ff917352135477e349fd1d379f38e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 12:25:46 +0000 Subject: [PATCH 0124/1722] fix style --- src/IO/WriteBufferFromPocoSocketChunked.h | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 40a89416f84..d1ba492738e 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -9,11 +9,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: @@ -37,7 +32,7 @@ public: { if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk return; - + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. /// We don't need to wary about if it's the end of the buffer because next() always sends the whole buffer From 740501b36e58c08d3a6a52348c9b0411d0f5dd90 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 7 Jun 2024 18:23:37 +0000 Subject: [PATCH 0125/1722] some potential bug fixes --- src/IO/WriteBufferFromPocoSocketChunked.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index d1ba492738e..689389ba2ea 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -109,7 +109,7 @@ protected: } /// Send end-of-chunk buffered by finishChunk - if (offset() == 2 * sizeof(*chunk_size_ptr)) + if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) { pos -= sizeof(*chunk_size_ptr); /// Send end-of-chunk @@ -140,6 +140,8 @@ protected: chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + last_finish_chunk = nullptr; + return; } @@ -164,6 +166,8 @@ protected: /// Initialize next chunk chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; } Poco::Net::SocketAddress peerAddress() From 90b5ad3613ea7e3b4dea202975407569d0aaee84 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 9 Jun 2024 19:31:20 +0000 Subject: [PATCH 0126/1722] fix tidy build --- src/IO/WriteBufferFromPocoSocketChunked.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 689389ba2ea..ecc33180140 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -85,7 +85,10 @@ protected: void nextImpl() override { if (!chunked) - return WriteBufferFromPocoSocket::nextImpl(); + { + WriteBufferFromPocoSocket::nextImpl(); + return; + } /// next() after finishChunk ar the end of the buffer if (finishing < sizeof(*chunk_size_ptr)) From 9cf11a210f07110676b373b864ea098583d87ff6 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 11 Jun 2024 11:11:06 +0200 Subject: [PATCH 0127/1722] Review changes --- src/Core/SettingsChangesHistory.h | 3 +- src/Formats/SchemaInferenceUtils.cpp | 73 ++++++------------- .../03150_infer_type_variant.reference | 24 +++--- 3 files changed, 35 insertions(+), 65 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3f743ef42bf..661ecc607ba 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,6 +92,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, @@ -103,8 +104,6 @@ static std::map sett {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, - {"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index a8b5d4343f5..b7c71a95b29 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -239,16 +239,6 @@ namespace return true; } - bool checkIfTypesContainVariant(const DataTypes & types) - { - for (size_t i = 0; i < types.size(); ++i) - { - if (isVariant(types[i])) - return true; - } - return false; - } - void updateTypeIndexes(DataTypes & data_types, TypeIndexesSet & type_indexes) { type_indexes.clear(); @@ -321,49 +311,28 @@ namespace /// if setting 'try_infer_variant' is true then we convert to type variant. void transformVariant(DataTypes & data_types, TypeIndexesSet & type_indexes) { - auto typesAreEqual = checkIfTypesAreEqual(data_types); - auto typesContainVariant = checkIfTypesContainVariant(data_types); - if (typesAreEqual) + if (checkIfTypesAreEqual(data_types)) return; - DataTypes new_data_types; - TypeIndexesSet new_type_indexes; - std::shared_ptr variant_type; - - /// extract the nested types of variant and make a new variant with the nested types and the other type. - /// eg. Type 1: variant, Type 2: Date -> variant. - if (typesContainVariant) + DataTypes variant_types; + for (const auto & type : data_types) { - DataTypes extracted_types; - for (size_t i=0; i(type.get())) { - if (isVariant(data_types[i])) - { - if (const auto * variant = typeid_cast(data_types[i].get())) - extracted_types = variant->getVariants(); - } - else - extracted_types.push_back(data_types[i]); + const auto & current_variants = variant_type->getVariants(); + variant_types.insert(variant_types.end(), current_variants.begin(), current_variants.end()); + } + else + { + variant_types.push_back(type); } - variant_type = std::make_shared(extracted_types); - } - else - { - variant_type = std::make_shared(data_types); } - size_t i = 0; - while (i != data_types.size()) - { - new_data_types.push_back(variant_type); - new_type_indexes.insert(TypeIndex::Variant); - i++; - } + auto variant_type = std::make_shared(variant_types); - data_types.clear(); - type_indexes.clear(); - data_types = new_data_types; - type_indexes = new_type_indexes; + for (auto & type : data_types) + type = variant_type; + type_indexes = {TypeIndex::Variant}; } /// If we have only Date and DateTime types, convert Date to DateTime, @@ -703,11 +672,12 @@ namespace if (settings.try_infer_dates || settings.try_infer_datetimes) transformDatesAndDateTimes(data_types, type_indexes); - if (settings.try_infer_variant) - transformVariant(data_types, type_indexes); - if constexpr (!is_json) + { + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); return; + } /// Check settings specific for JSON formats. @@ -740,11 +710,12 @@ namespace /// If there is at least one non Nothing type, change all Nothing types to it. transformNothingComplexTypes(data_types, type_indexes); - if (settings.try_infer_variant) - transformVariant(data_types, type_indexes); - if constexpr (!is_json) + { + if (settings.try_infer_variant) + transformVariant(data_types, type_indexes); return; + } /// Convert JSON tuples with same nested types to arrays. transformTuplesWithEqualNestedTypesToArrays(data_types, type_indexes); diff --git a/tests/queries/0_stateless/03150_infer_type_variant.reference b/tests/queries/0_stateless/03150_infer_type_variant.reference index a5f56cb3618..a43fa1e1227 100644 --- a/tests/queries/0_stateless/03150_infer_type_variant.reference +++ b/tests/queries/0_stateless/03150_infer_type_variant.reference @@ -1,16 +1,16 @@ - â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ - ┃ arr ┃ toTypeName(arr) ┃ - ┡â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ -1. │ [1,'Hello',(32)] │ Array(Variant(Int64, String, Tuple( + â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ arr ┃ toTypeName(arr) ┃ + ┡â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ ['1','Hello',(32)] │ Array(Variant(String, Tuple( a Nullable(Int64)))) │ - └──────────────────┴─────────────────────────────────────────────────────────────┘ - â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ - ┃ x ┃ toTypeName(x) ┃ - ┡â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ -1. │ 42 │ Variant(Int64, String) │ - ├───────┼────────────────────────┤ -2. │ Hello │ Variant(Int64, String) │ - └───────┴────────────────────────┘ + └────────────────────┴──────────────────────────────────────────────────────┘ + â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ + ┃ x ┃ toTypeName(x) ┃ + ┡â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ +1. │ 42 │ Nullable(String) │ + ├───────┼──────────────────┤ +2. │ Hello │ Nullable(String) │ + └───────┴──────────────────┘ â”â”â”â”â”â”â”â”â”â”┳â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┓ ┃ x ┃ toTypeName(x) ┃ ┡â”â”â”â”â”â”â”â”â”╇â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”â”┩ From 548c90901020317669a10d191a0b6f8a7d0a0511 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 11 Jun 2024 12:14:36 +0200 Subject: [PATCH 0128/1722] Add documentation --- docs/en/operations/settings/settings-formats.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 6aae1ea62e5..8bbb469547b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -194,6 +194,17 @@ If enabled, ClickHouse will try to infer type `DateTime64` from string fields in Enabled by default. +## input_format_try_infer_variants {#input_format_try_infer_variants} + +If enabled, ClickHouse will try to infer type [`Variant`](../../sql-reference/data-types/variant.md) in schema inference for text formats when there is more than one possible type for column/array elements. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `0`. + ## date_time_input_format {#date_time_input_format} Allows choosing a parser of the text representation of date and time. From bad5e27bbffa9c1f6727a0416edcb135dadcc1fe Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 11 Jun 2024 13:32:34 +0200 Subject: [PATCH 0129/1722] Update src/Formats/SchemaInferenceUtils.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Formats/SchemaInferenceUtils.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 0ac8b32f8aa..240830013c6 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1539,9 +1539,7 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) DataTypes nested_types; for (const auto & nested_type: variant_type->getVariants()) { - auto is_low_cardinality = nested_type->lowCardinality(); - auto has_sub_types = nested_type->haveSubtypes(); - if (!is_low_cardinality && has_sub_types) + if (!nested_type->lowCardinality() && nested_type->haveSubtypes()) nested_types.push_back(makeNullableRecursively(nested_type)); else nested_types.push_back(nested_type); From fb49cf503e4159549348c76ebf9c3ca686b9f02f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 11 Jun 2024 16:47:05 +0000 Subject: [PATCH 0130/1722] some fixes --- src/IO/WriteBufferFromPocoSocketChunked.h | 31 ++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index ecc33180140..4325ab2bd4b 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -35,7 +35,7 @@ public: /// If current chunk is empty it means we are finishing a chunk previously sent by next(), /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. - /// We don't need to wary about if it's the end of the buffer because next() always sends the whole buffer + /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer /// so it should be a beginning of the buffer. chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); @@ -50,6 +50,13 @@ public: return; } + /// Previously finished chunk wasn't sent yet + if (last_finish_chunk == chunk_size_ptr) + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + } + /// Fill up current chunk size *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); @@ -59,7 +66,10 @@ public: static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), *chunk_size_ptr); else + { chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); @@ -81,6 +91,18 @@ public: last_finish_chunk = chunk_size_ptr; } + ~WriteBufferFromPocoSocketChunked() override + { + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + protected: void nextImpl() override { @@ -173,6 +195,13 @@ protected: last_finish_chunk = nullptr; } + void finalizeImpl() override + { + if (offset() == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + WriteBufferFromPocoSocket::finalizeImpl(); + } + Poco::Net::SocketAddress peerAddress() { return peer_address; From aad55ab55faa9ed64d802c73080db77503c6363d Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 12 Jun 2024 14:49:06 +0000 Subject: [PATCH 0131/1722] IO scheduling on HTTP session level --- base/poco/Net/include/Poco/Net/HTTPSession.h | 43 ++++++++++ base/poco/Net/src/HTTPSession.cpp | 24 ++++-- base/poco/Net/src/SocketImpl.cpp | 36 ++++----- src/Common/CurrentMetrics.cpp | 3 + src/Common/CurrentThread.cpp | 50 ++++++++++++ src/Common/CurrentThread.h | 43 +++++++++- src/Common/HTTPConnectionPool.cpp | 64 ++++++++++++++- src/Common/ProfileEvents.cpp | 7 ++ .../Scheduler/Nodes/tests/ResourceTest.h | 6 +- .../tests/gtest_dynamic_resource_manager.cpp | 4 +- .../Nodes/tests/gtest_resource_scheduler.cpp | 8 +- src/Common/Scheduler/ResourceGuard.h | 79 +++++++++++++++++-- src/Common/Scheduler/ResourceLink.h | 23 +++++- src/Common/ThreadStatus.h | 6 +- .../IO/ReadBufferFromAzureBlobStorage.cpp | 5 ++ .../IO/WriteBufferFromAzureBlobStorage.cpp | 6 +- .../ObjectStorages/DiskObjectStorage.cpp | 13 +-- src/IO/ReadBufferFromS3.cpp | 12 +-- src/IO/ReadSettings.h | 3 +- src/IO/WriteBufferFromS3.cpp | 13 +-- src/IO/WriteSettings.h | 3 +- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 8 +- .../HDFS/WriteBufferFromHDFS.cpp | 10 +-- 23 files changed, 379 insertions(+), 90 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index cac14f479db..b0e59443f9b 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -19,6 +19,8 @@ #include +#include +#include #include "Poco/Any.h" #include "Poco/Buffer.h" #include "Poco/Exception.h" @@ -33,6 +35,27 @@ namespace Net { + class IHTTPSessionDataHooks + /// Interface to control stream of data bytes being sent or received though socket by HTTPSession + /// It allows to monitor, throttle and schedule data streams with syscall granulatrity + { + public: + virtual ~IHTTPSessionDataHooks() = default; + + virtual void start(int bytes) = 0; + /// Called before sending/receiving data `bytes` to/from socket. + + virtual void finish(int bytes) = 0; + /// Called when sending/receiving of data `bytes` is successfully finished. + + virtual void fail() = 0; + /// If an error occured during send/receive `fail()` is called instead of `finish()`. + }; + + + using HTTPSessionDataHooksPtr = std::shared_ptr; + + class Net_API HTTPSession /// HTTPSession implements basic HTTP session management /// for both HTTP clients and HTTP servers. @@ -73,6 +96,12 @@ namespace Net Poco::Timespan getReceiveTimeout() const; /// Returns receive timeout for the HTTP session. + void setSendDataHooks(const HTTPSessionDataHooksPtr & sendDataHooks = {}); + /// Sets data hooks that will be called on every sent to the socket. + + void setReceiveDataHooks(const HTTPSessionDataHooksPtr & receiveDataHooks = {}); + /// Sets data hooks that will be called on every receive from the socket. + bool connected() const; /// Returns true if the underlying socket is connected. @@ -211,6 +240,10 @@ namespace Net Poco::Exception * _pException; Poco::Any _data; + // Data hooks + HTTPSessionDataHooksPtr _sendDataHooks; + HTTPSessionDataHooksPtr _receiveDataHooks; + friend class HTTPStreamBuf; friend class HTTPHeaderStreamBuf; friend class HTTPFixedLengthStreamBuf; @@ -246,6 +279,16 @@ namespace Net return _receiveTimeout; } + inline void HTTPSession::setSendDataHooks(const HTTPSessionDataHooksPtr & sendDataHooks) + { + _sendDataHooks = sendDataHooks; + } + + inline void HTTPSession::setReceiveDataHooks(const HTTPSessionDataHooksPtr & receiveDataHooks) + { + _receiveDataHooks = receiveDataHooks; + } + inline StreamSocket & HTTPSession::socket() { return _socket; diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 8f951b3102c..596185703fa 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -128,14 +128,14 @@ int HTTPSession::get() { if (_pCurrent == _pEnd) refill(); - + if (_pCurrent < _pEnd) return *_pCurrent++; else return std::char_traits::eof(); } - + int HTTPSession::peek() { if (_pCurrent == _pEnd) @@ -147,7 +147,7 @@ int HTTPSession::peek() return std::char_traits::eof(); } - + int HTTPSession::read(char* buffer, std::streamsize length) { if (_pCurrent < _pEnd) @@ -166,10 +166,17 @@ int HTTPSession::write(const char* buffer, std::streamsize length) { try { - return _socket.sendBytes(buffer, (int) length); + if (_sendDataHooks) + _sendDataHooks->start((int) length); + int result = _socket.sendBytes(buffer, (int) length); + if (_sendDataHooks) + _sendDataHooks->finish(result); + return result; } catch (Poco::Exception& exc) { + if (_sendDataHooks) + _sendDataHooks->fail(); setException(exc); throw; } @@ -180,10 +187,17 @@ int HTTPSession::receive(char* buffer, int length) { try { - return _socket.receiveBytes(buffer, length); + if (_receiveDataHooks) + _receiveDataHooks->start(length); + int result = _socket.receiveBytes(buffer, length); + if (_receiveDataHooks) + _receiveDataHooks->finish(result); + return result; } catch (Poco::Exception& exc) { + if (_receiveDataHooks) + _receiveDataHooks->fail(); setException(exc); throw; } diff --git a/base/poco/Net/src/SocketImpl.cpp b/base/poco/Net/src/SocketImpl.cpp index 484b8cfeec3..65456a287fb 100644 --- a/base/poco/Net/src/SocketImpl.cpp +++ b/base/poco/Net/src/SocketImpl.cpp @@ -62,7 +62,7 @@ bool checkIsBrokenTimeout() SocketImpl::SocketImpl(): _sockfd(POCO_INVALID_SOCKET), - _blocking(true), + _blocking(true), _isBrokenTimeout(checkIsBrokenTimeout()) { } @@ -81,7 +81,7 @@ SocketImpl::~SocketImpl() close(); } - + SocketImpl* SocketImpl::acceptConnection(SocketAddress& clientAddr) { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -117,7 +117,7 @@ void SocketImpl::connect(const SocketAddress& address) rc = ::connect(_sockfd, address.addr(), address.length()); } while (rc != 0 && lastError() == POCO_EINTR); - if (rc != 0) + if (rc != 0) { int err = lastError(); error(err, address.toString()); @@ -204,7 +204,7 @@ void SocketImpl::bind6(const SocketAddress& address, bool reuseAddress, bool reu #if defined(POCO_HAVE_IPv6) if (address.family() != SocketAddress::IPv6) throw Poco::InvalidArgumentException("SocketAddress must be an IPv6 address"); - + if (_sockfd == POCO_INVALID_SOCKET) { init(address.af()); @@ -225,11 +225,11 @@ void SocketImpl::bind6(const SocketAddress& address, bool reuseAddress, bool reu #endif } - + void SocketImpl::listen(int backlog) { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); - + int rc = ::listen(_sockfd, backlog); if (rc != 0) error(); } @@ -253,7 +253,7 @@ void SocketImpl::shutdownReceive() if (rc != 0) error(); } - + void SocketImpl::shutdownSend() { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -262,7 +262,7 @@ void SocketImpl::shutdownSend() if (rc != 0) error(); } - + void SocketImpl::shutdown() { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -317,7 +317,7 @@ int SocketImpl::receiveBytes(void* buffer, int length, int flags) throw TimeoutException(); } } - + int rc; do { @@ -325,7 +325,7 @@ int SocketImpl::receiveBytes(void* buffer, int length, int flags) rc = ::recv(_sockfd, reinterpret_cast(buffer), length, flags); } while (blocking && rc < 0 && lastError() == POCO_EINTR); - if (rc < 0) + if (rc < 0) { int err = lastError(); if ((err == POCO_EAGAIN || err == POCO_EWOULDBLOCK) && !blocking) @@ -363,7 +363,7 @@ int SocketImpl::receiveFrom(void* buffer, int length, SocketAddress& address, in throw TimeoutException(); } } - + sockaddr_storage abuffer; struct sockaddr* pSA = reinterpret_cast(&abuffer); poco_socklen_t saLen = sizeof(abuffer); @@ -450,7 +450,7 @@ bool SocketImpl::pollImpl(Poco::Timespan& remainingTime, int mode) } while (rc < 0 && lastError() == POCO_EINTR); if (rc < 0) error(); - return rc > 0; + return rc > 0; #else @@ -493,7 +493,7 @@ bool SocketImpl::pollImpl(Poco::Timespan& remainingTime, int mode) } while (rc < 0 && errorCode == POCO_EINTR); if (rc < 0) error(errorCode); - return rc > 0; + return rc > 0; #endif // POCO_HAVE_FD_POLL } @@ -503,13 +503,13 @@ bool SocketImpl::poll(const Poco::Timespan& timeout, int mode) Poco::Timespan remainingTime(timeout); return pollImpl(remainingTime, mode); } - + void SocketImpl::setSendBufferSize(int size) { setOption(SOL_SOCKET, SO_SNDBUF, size); } - + int SocketImpl::getSendBufferSize() { int result; @@ -523,7 +523,7 @@ void SocketImpl::setReceiveBufferSize(int size) setOption(SOL_SOCKET, SO_RCVBUF, size); } - + int SocketImpl::getReceiveBufferSize() { int result; @@ -569,7 +569,7 @@ Poco::Timespan SocketImpl::getReceiveTimeout() return result; } - + SocketAddress SocketImpl::address() { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -580,7 +580,7 @@ SocketAddress SocketImpl::address() int rc = ::getsockname(_sockfd, pSA, &saLen); if (rc == 0) return SocketAddress(pSA, saLen); - else + else error(); return SocketAddress(); } diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 731c72d65f2..63156f5291d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -291,6 +291,9 @@ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") \ \ + M(SchedulerIOReadScheduled, "Number of IO reads are being scheduled currently") \ + M(SchedulerIOWriteScheduled, "Number of IO writes are being scheduled currently") \ + \ M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \ M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \ \ diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 70b69d6bcc7..ba7087ca7f1 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -113,6 +113,56 @@ std::string_view CurrentThread::getQueryId() return current_thread->getQueryId(); } +void CurrentThread::attachReadResource(ResourceLink link) +{ + if (unlikely(!current_thread)) + return; + if (current_thread->read_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has been already attached to read resource", std::to_string(getThreadId())); + current_thread->read_resource_link = link; +} + +void CurrentThread::detachReadResource() +{ + if (unlikely(!current_thread)) + return; + if (!current_thread->read_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has not been attached to read resource", std::to_string(getThreadId())); + current_thread->read_resource_link.reset(); +} + +ResourceLink CurrentThread::getReadResourceLink() +{ + if (unlikely(!current_thread)) + return {}; + return current_thread->read_resource_link; +} + +void CurrentThread::attachWriteResource(ResourceLink link) +{ + if (unlikely(!current_thread)) + return; + if (current_thread->write_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has been already attached to write resource", std::to_string(getThreadId())); + current_thread->write_resource_link = link; +} + +void CurrentThread::detachWriteResource() +{ + if (unlikely(!current_thread)) + return; + if (!current_thread->write_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has not been attached to write resource", std::to_string(getThreadId())); + current_thread->write_resource_link.reset(); +} + +ResourceLink CurrentThread::getWriteResourceLink() +{ + if (unlikely(!current_thread)) + return {}; + return current_thread->write_resource_link; +} + MemoryTracker * CurrentThread::getUserMemoryTracker() { if (unlikely(!current_thread)) diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 53b61ba315f..787e8369a83 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -23,7 +24,6 @@ class QueryStatus; struct Progress; class InternalTextLogsQueue; - /** Collection of static methods to work with thread-local objects. * Allows to attach and detach query/process (thread group) to a thread * (to calculate query-related metrics and to allow to obtain query-related data from a thread). @@ -92,6 +92,14 @@ public: static std::string_view getQueryId(); + // For IO Scheduling + static void attachReadResource(ResourceLink link); + static void detachReadResource(); + static ResourceLink getReadResourceLink(); + static void attachWriteResource(ResourceLink link); + static void detachWriteResource(); + static ResourceLink getWriteResourceLink(); + /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope : private boost::noncopyable { @@ -102,6 +110,39 @@ public: void logPeakMemoryUsage(); bool log_peak_memory_usage_in_destructor = true; }; + + /// Scoped attach/detach of IO resource links + struct IOScope : private boost::noncopyable + { + explicit IOScope(ResourceLink read_resource_link, ResourceLink write_resource_link) + { + if (read_resource_link) + { + attachReadResource(read_resource_link); + read_attached = true; + } + if (write_resource_link) + { + attachWriteResource(write_resource_link); + write_attached = true; + } + } + + explicit IOScope(const IOSchedulingSettings & settings) + : IOScope(settings.read_resource_link, settings.write_resource_link) + {} + + ~IOScope() + { + if (read_attached) + detachReadResource(); + if (write_attached) + detachWriteResource(); + } + + bool read_attached = false; + bool write_attached = false; + }; }; } diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 167aeee68f3..de7e10d044a 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -249,6 +250,54 @@ public: }; +// Session data hooks implementation for integration with resource scheduler. +// Hooks are created per every request-response pair and are registered/unregistered in HTTP session. +// * `start()` send resource request to the scheduler every time HTTP session is going to send or receive +// data to/from socket. `start()` waits for the scheduler confirmation. This way scheduler might +// throttle and/or schedule socket data streams. +// * `finish()` hook is called on successful socket read/write operation. +// It informs the scheduler that operation is complete, which allows the scheduler to control the total +// amount of in-flight bytes and/or operations. +// * `fail()` hook is called on failure of socket operation. The purpose is to correct the amount of bytes +// passed through the scheduler queue to ensure fair bandwidth allocation even in presence of errors. +struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks +{ + explicit ResourceGuardSessionDataHooks(const ResourceGuard::Metrics * metrics, ResourceLink link_) + : link(link_) + { + request.metrics = metrics; + chassert(link); + } + + ~ResourceGuardSessionDataHooks() override + { + request.assertFinished(); // Never destruct with an active request + } + + void start(int bytes) override + { + // TODO(serxa): add metrics here or better in scheduler code (e.g. during enqueue, or better in REsourceGuard::REquest)? + request.enqueue(bytes, link); + request.wait(); + } + + void finish(int bytes) override + { + request.finish(); + link.adjust(request.cost, bytes); // success + } + + void fail() override + { + request.finish(); + link.accumulate(request.cost); // We assume no resource was used in case of failure + } + + ResourceLink link; + ResourceGuard::Request request; +}; + + // EndpointConnectionPool manage connections to the endpoint // Features: // - it uses HostResolver for address selecting. See Common/HostResolver.h for more info. @@ -259,8 +308,6 @@ public: // - `Session::reconnect()` uses the pool as well // - comprehensive sensors // - session is reused according its inner state, automatically - - template class EndpointConnectionPool : public std::enable_shared_from_this>, public IExtendedPool { @@ -350,6 +397,19 @@ private: std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override { auto idle = idleTime(); + + // Reset data hooks for IO scheduling + if (ResourceLink link = CurrentThread::getReadResourceLink()) { + Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); + } else { + Session::setSendDataHooks(); + } + if (ResourceLink link = CurrentThread::getWriteResourceLink()) { + Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + } else { + Session::setReceiveDataHooks(); + } + std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f73e16c517d..b305614e54c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -104,6 +104,13 @@ M(PartsWithAppliedMutationsOnFly, "Total number of parts for which there was any mutation applied on fly") \ M(MutationsAppliedOnFlyInAllParts, "The sum of number of applied mutations on-fly for part among all read parts") \ \ + M(SchedulerIOReadRequests, "Resource requests passed through scheduler for IO reads.") \ + M(SchedulerIOReadBytes, "Bytes passed through scheduler for IO reads.") \ + M(SchedulerIOReadWaitMicroseconds, "Total time a query was waiting on resource requests for IO reads.") \ + M(SchedulerIOWriteRequests, "Resource requests passed through scheduler for IO writes.") \ + M(SchedulerIOWriteBytes, "Bytes passed through scheduler for IO writes.") \ + M(SchedulerIOWriteWaitMicroseconds, "Total time a query was waiting on resource requests for IO writes.") \ + \ M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \ \ M(ReplicatedPartFetches, "Number of times a data part was downloaded from replica of a ReplicatedMergeTree table.") \ diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index ea3f9edf765..a5eb98f2a2f 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -232,7 +232,7 @@ struct ResourceTestManager : public ResourceTestBase ResourceTestManager & t; Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(link_, cost, PostponeLocking) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, PostponeLocking) , t(t_) { t.onEnqueue(link); @@ -310,7 +310,7 @@ struct ResourceTestManager : public ResourceTestBase // NOTE: actually leader's request(s) make their own small busy period. void blockResource(ResourceLink link) { - ResourceGuard g(link, 1, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::PostponeLocking); g.lock(); // NOTE: at this point we assume resource to be blocked by single request (1) busy_period.arrive_and_wait(); // (1) notify all followers that resource is blocked @@ -320,7 +320,7 @@ struct ResourceTestManager : public ResourceTestBase { getLinkData(link).left += total_requests + 1; busy_period.arrive_and_wait(); // (1) wait leader to block resource - ResourceGuard g(link, cost, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::PostponeLocking); onEnqueue(link); busy_period.arrive_and_wait(); // (2) notify leader to unblock g.lock(); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 1901a4fd120..4ac79977663 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -36,11 +36,11 @@ TEST(SchedulerDynamicResourceManager, Smoke) for (int i = 0; i < 10; i++) { - ResourceGuard gA(cA->get("res1"), ResourceGuard::PostponeLocking); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), ResourceGuard::PostponeLocking); gA.lock(); gA.unlock(); - ResourceGuard gB(cB->get("res1")); + ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index f8196d15819..ba573bf0c85 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -109,22 +109,22 @@ TEST(SchedulerRoot, Smoke) r2.registerResource(); { - ResourceGuard rg(a); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); EXPECT_TRUE(fc1->requests.contains(&rg.request)); } { - ResourceGuard rg(b); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); EXPECT_TRUE(fc1->requests.contains(&rg.request)); } { - ResourceGuard rg(c); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); EXPECT_TRUE(fc2->requests.contains(&rg.request)); } { - ResourceGuard rg(d); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); EXPECT_TRUE(fc2->requests.contains(&rg.request)); } } diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 3c29f588fba..73aea4afdb6 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -7,10 +7,30 @@ #include #include +#include +#include +#include + #include #include +namespace ProfileEvents +{ + extern const Event SchedulerIOReadRequests; + extern const Event SchedulerIOReadBytes; + extern const Event SchedulerIOReadWaitMicroseconds; + extern const Event SchedulerIOWriteRequests; + extern const Event SchedulerIOWriteBytes; + extern const Event SchedulerIOWriteWaitMicroseconds; +} + +namespace CurrentMetrics +{ + extern const Metric SchedulerIOReadScheduled; + extern const Metric SchedulerIOWriteScheduled; +} + namespace DB { @@ -30,6 +50,36 @@ public: PostponeLocking /// Don't lock in constructor, but send request }; + struct Metrics + { + const ProfileEvents::Event requests = ProfileEvents::end(); + const ProfileEvents::Event cost = ProfileEvents::end(); + const ProfileEvents::Event wait_microseconds = ProfileEvents::end(); + const CurrentMetrics::Metric scheduled_count = CurrentMetrics::end(); + + static const Metrics * getIORead() + { + static Metrics metrics{ + .requests = ProfileEvents::SchedulerIOReadRequests, + .cost = ProfileEvents::SchedulerIOReadBytes, + .wait_microseconds = ProfileEvents::SchedulerIOReadWaitMicroseconds, + .scheduled_count = CurrentMetrics::SchedulerIOReadScheduled + }; + return &metrics; + } + + static const Metrics * getIOWrite() + { + static Metrics metrics{ + .requests = ProfileEvents::SchedulerIOWriteRequests, + .cost = ProfileEvents::SchedulerIOWriteBytes, + .wait_microseconds = ProfileEvents::SchedulerIOWriteWaitMicroseconds, + .scheduled_count = CurrentMetrics::SchedulerIOWriteScheduled + }; + return &metrics; + } + }; + enum RequestState { Finished, // Last request has already finished; no concurrent access is possible @@ -46,6 +96,8 @@ public: chassert(state == Finished); state = Enqueued; ResourceRequest::reset(cost_); + ProfileEvents::increment(metrics->requests); + ProfileEvents::increment(metrics->cost, cost_); link_.queue->enqueueRequestUsingBudget(this); } @@ -63,6 +115,8 @@ public: void wait() { + CurrentMetrics::Increment scheduled(metrics->scheduled_count); + auto timer = CurrentThread::getProfileEvents().timer(metrics->wait_microseconds); std::unique_lock lock(mutex); dequeued_cv.wait(lock, [this] { return state == Dequeued; }); } @@ -75,14 +129,23 @@ public: ResourceRequest::finish(); } - static Request & local() + void assertFinished() + { + // lock(mutex) is not required because `Finished` request cannot be used by the scheduler thread + chassert(state == Finished); + } + + static Request & local(const Metrics * metrics) { // Since single thread cannot use more than one resource request simultaneously, // we can reuse thread-local request to avoid allocations static thread_local Request instance; + instance.metrics = metrics; return instance; } + const Metrics * metrics = nullptr; // Must be initialized before use + private: std::mutex mutex; std::condition_variable dequeued_cv; @@ -90,13 +153,13 @@ public: }; /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) - explicit ResourceGuard(ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) + explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) : link(link_) - , request(Request::local()) + , request(Request::local(metrics)) { if (cost == 0) - link.queue = nullptr; // Ignore zero-cost requests - else if (link.queue) + link.reset(); // Ignore zero-cost requests + else if (link) { request.enqueue(cost, link); if (ctor == LockStraightAway) @@ -112,17 +175,17 @@ public: /// Blocks until resource is available void lock() { - if (link.queue) + if (link) request.wait(); } /// Report resource consumption has finished void unlock() { - if (link.queue) + if (link) { request.finish(); - link.queue = nullptr; + link.reset(); } } diff --git a/src/Common/Scheduler/ResourceLink.h b/src/Common/Scheduler/ResourceLink.h index 450d9bc1efa..6dd3be930ca 100644 --- a/src/Common/Scheduler/ResourceLink.h +++ b/src/Common/Scheduler/ResourceLink.h @@ -13,13 +13,32 @@ using ResourceCost = Int64; struct ResourceLink { ISchedulerQueue * queue = nullptr; + bool operator==(const ResourceLink &) const = default; + explicit operator bool() const { return queue != nullptr; } void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const; - void consumed(ResourceCost cost) const; - void accumulate(ResourceCost cost) const; + + void reset() + { + queue = nullptr; + } +}; + +/* + * Everything required for IO scheduling. + * Note that raw pointer are stored inside, so make sure that `ClassifierPtr` that produced + * resource links will outlive them. Usually classifier is stored in query `Context`. + */ +struct IOSchedulingSettings +{ + ResourceLink read_resource_link; + ResourceLink write_resource_link; + + bool operator==(const IOSchedulingSettings &) const = default; + explicit operator bool() const { return read_resource_link && write_resource_link; } }; } diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 0c02ab8fdb0..6ea0a9a848c 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -7,11 +7,11 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -188,6 +188,10 @@ public: Progress progress_in; Progress progress_out; + /// IO scheduling + ResourceLink read_resource_link; + ResourceLink write_resource_link; + private: /// Group of threads, to which this thread attached ThreadGroupPtr thread_group; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index da1ea65f2ea..ecc4168c729 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -113,13 +114,17 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() { try { + ResourceGuard rlock(ResourceGuard::Metrics::getIORead(), read_settings.io_scheduling.read_resource_link, to_read_bytes); bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); + read_settings.io_scheduling.read_resource_link.adjust(to_read_bytes, bytes_read); + rlock.unlock(); // Do not hold resource under bandwidth throttler if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); break; } catch (const Azure::Core::RequestFailedException & e) { + read_settings.io_scheduling.read_resource_link.accumulate(to_read_bytes); // We assume no resource was used in case of failure ProfileEvents::increment(ProfileEvents::ReadBufferFromAzureRequestsErrors); LOG_DEBUG(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..355f70b5e33 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -88,14 +88,14 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { try { - ResourceGuard rlock(write_settings.resource_link, cost); // Note that zero-cost requests are ignored + ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, cost); // Note that zero-cost requests are ignored func(); break; } catch (const Azure::Core::RequestFailedException & e) { if (cost) - write_settings.resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it + write_settings.io_scheduling.write_resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it if (i == num_tries - 1 || !isRetryableAzureException(e)) throw; @@ -105,7 +105,7 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, catch (...) { if (cost) - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure + write_settings.io_scheduling.write_resource_link.accumulate(cost); // We assume no resource was used in case of failure throw; } } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 5803a985000..ccdb321c904 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -461,14 +461,17 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() } template -static inline Settings updateResourceLink(const Settings & settings, const String & resource_name) +static inline Settings updateIOSchedulingSettings(const Settings & settings, const String & read_resource_name, const String & write_resource_name) { - if (resource_name.empty()) + if (read_resource_name.empty() && write_resource_name.empty()) return settings; if (auto query_context = CurrentThread::getQueryContext()) { Settings result(settings); - result.resource_link = query_context->getWorkloadClassifier()->get(resource_name); + if (!read_resource_name.empty()) + result.io_scheduling.read_resource_link = query_context->getWorkloadClassifier()->get(read_resource_name); + if (!write_resource_name.empty()) + result.io_scheduling.write_resource_link = query_context->getWorkloadClassifier()->get(write_resource_name); return result; } return settings; @@ -500,7 +503,7 @@ std::unique_ptr DiskObjectStorage::readFile( return object_storage->readObjects( storage_objects, - updateResourceLink(settings, getReadResourceName()), + updateIOSchedulingSettings(settings, getReadResourceName(), getWriteResourceName()), read_hint, file_size); } @@ -513,7 +516,7 @@ std::unique_ptr DiskObjectStorage::writeFile( { LOG_TEST(log, "Write file: {}", path); - WriteSettings write_settings = updateResourceLink(settings, getWriteResourceName()); + WriteSettings write_settings = updateIOSchedulingSettings(settings, getReadResourceName(), getWriteResourceName()); auto transaction = createObjectStorageTransaction(); return transaction->writeFile(path, buf_size, mode, write_settings); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 8823af55936..0c90ae25626 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -425,22 +424,13 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3InitMicroseconds); // We do not know in advance how many bytes we are going to consume, to avoid blocking estimated it from below - constexpr ResourceCost estimated_cost = 1; - ResourceGuard rlock(read_settings.resource_link, estimated_cost); - + CurrentThread::IOScope io_scope(read_settings.io_scheduling); Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); - rlock.unlock(); - if (outcome.IsSuccess()) - { - ResourceCost bytes_read = outcome.GetResult().GetContentLength(); - read_settings.resource_link.adjust(estimated_cost, bytes_read); return outcome.GetResultWithOwnership(); - } else { - read_settings.resource_link.accumulate(estimated_cost); const auto & error = outcome.GetError(); throw S3Exception(error.GetMessage(), error.GetErrorType()); } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index e73a9054928..7c22682dc76 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -118,8 +118,7 @@ struct ReadSettings ThrottlerPtr remote_throttler; ThrottlerPtr local_throttler; - // Resource to be used during reading - ResourceLink resource_link; + IOSchedulingSettings io_scheduling; size_t http_max_tries = 10; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index ff18a77f09f..160816ebbaa 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -11,7 +11,6 @@ #include #include -#include #include #include #include @@ -538,12 +537,11 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) auto & request = std::get<0>(*worker_data); - ResourceCost cost = request.GetContentLength(); - ResourceGuard rlock(write_settings.resource_link, cost); + CurrentThread::IOScope io_scope(write_settings.io_scheduling); + Stopwatch watch; auto outcome = client_ptr->UploadPart(request); watch.stop(); - rlock.unlock(); // Avoid acquiring other locks under resource lock ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); @@ -557,7 +555,6 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) if (!outcome.IsSuccess()) { ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } @@ -695,12 +692,11 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3PutObject); - ResourceCost cost = request.GetContentLength(); - ResourceGuard rlock(write_settings.resource_link, cost); + CurrentThread::IOScope io_scope(write_settings.io_scheduling); + Stopwatch watch; auto outcome = client_ptr->PutObject(request); watch.stop(); - rlock.unlock(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); if (blob_log) @@ -714,7 +710,6 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data } ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) { diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 84bb25439b5..cdc75e8c0e9 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -13,8 +13,7 @@ struct WriteSettings ThrottlerPtr remote_throttler; ThrottlerPtr local_throttler; - // Resource to be used during reading - ResourceLink resource_link; + IOSchedulingSettings io_scheduling; /// Filesystem cache settings bool enable_filesystem_cache_on_write_operations = false; diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index be339d021dc..d1502577bf7 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -119,27 +119,25 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); + read_settings.io_scheduling.read_resource_link.adjust(num_bytes_to_read, std::max(0, bytes_read)); } catch (...) { - read_settings.resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure + read_settings.io_scheduling.read_resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure throw; } - rlock.unlock(); if (bytes_read < 0) { - read_settings.resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to read from HDFS: {}, file path: {}. Error: {}", hdfs_uri, hdfs_file_path, std::string(hdfsGetLastError())); } - read_settings.resource_link.adjust(num_bytes_to_read, bytes_read); if (bytes_read) { diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 2c14b38ce01..65a5f45cd96 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -66,25 +66,21 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) { - ResourceGuard rlock(write_settings.resource_link, size); int bytes_written; try { + ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, size); bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + write_settings.io_scheduling.write_resource_link.adjust(size, std::max(0, bytes_written)); } catch (...) { - write_settings.resource_link.accumulate(size); // We assume no resource was used in case of failure + write_settings.io_scheduling.write_resource_link.accumulate(size); // We assume no resource was used in case of failure throw; } - rlock.unlock(); if (bytes_written < 0) - { - write_settings.resource_link.accumulate(size); // We assume no resource was used in case of failure throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); - } - write_settings.resource_link.adjust(size, bytes_written); if (write_settings.remote_throttler) write_settings.remote_throttler->add(bytes_written, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); From eb3c61915d4bfae9f1e1007ca02fcd2c0f50b585 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 14:27:39 +0000 Subject: [PATCH 0132/1722] fix mixed read and write links --- src/Common/HTTPConnectionPool.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index de7e10d044a..b6bf98894e4 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -400,15 +400,15 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) { - Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); - } else { - Session::setSendDataHooks(); - } - if (ResourceLink link = CurrentThread::getWriteResourceLink()) { - Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); } else { Session::setReceiveDataHooks(); } + if (ResourceLink link = CurrentThread::getWriteResourceLink()) { + Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + } else { + Session::setSendDataHooks(); + } std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); From f1f354f22b5d1573e970e5d2a5e2ea540e3f99f8 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 15:33:59 +0000 Subject: [PATCH 0133/1722] add test for granularity and total byte size of resource requests --- tests/integration/test_scheduler/test.py | 67 ++++++++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 8e37bd8d403..9940e16ea42 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -28,6 +28,73 @@ def start_cluster(): cluster.shutdown() +def test_s3_resource_request_granularity(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE), value String CODEC(NONE)) engine=MergeTree() order by key settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + total_bytes = 50000000 # Approximate data size + max_bytes_per_request = 2000000 # Should be ~1MB or less in general + min_bytes_per_request = 6000 # Small requests are ok, but we don't want hurt performance with too often resource requests + + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + write_bytes_before = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + node.query(f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'") + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + write_bytes_after = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + + assert write_bytes_after - write_bytes_before > 1.0 * total_bytes + assert write_bytes_after - write_bytes_before < 1.2 * total_bytes + assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) < max_bytes_per_request + assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) > min_bytes_per_request + + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + read_bytes_before = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + node.query(f"select count() from data where not ignore(*) SETTINGS workload='admin'") + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + read_bytes_after = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + + assert read_bytes_after - read_bytes_before > 1.0 * total_bytes + assert read_bytes_after - read_bytes_before < 1.2 * total_bytes + assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) < max_bytes_per_request + assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) > min_bytes_per_request + + def test_s3_disk(): node.query( f""" From b74cf1a356fe668721532c1d05fa08aeaa1bf023 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 13 Jun 2024 15:43:08 +0000 Subject: [PATCH 0134/1722] Automatic style fix --- tests/integration/test_scheduler/test.py | 30 +++++++++++++++++------- 1 file changed, 21 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 9940e16ea42..0ae297f04d1 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -36,9 +36,9 @@ def test_s3_resource_request_granularity(): """ ) - total_bytes = 50000000 # Approximate data size - max_bytes_per_request = 2000000 # Should be ~1MB or less in general - min_bytes_per_request = 6000 # Small requests are ok, but we don't want hurt performance with too often resource requests + total_bytes = 50000000 # Approximate data size + max_bytes_per_request = 2000000 # Should be ~1MB or less in general + min_bytes_per_request = 6000 # Small requests are ok, but we don't want hurt performance with too often resource requests writes_before = int( node.query( @@ -50,7 +50,9 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" ).strip() ) - node.query(f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'") + node.query( + f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'" + ) writes_after = int( node.query( f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/admin'" @@ -64,8 +66,12 @@ def test_s3_resource_request_granularity(): assert write_bytes_after - write_bytes_before > 1.0 * total_bytes assert write_bytes_after - write_bytes_before < 1.2 * total_bytes - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) < max_bytes_per_request - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) > min_bytes_per_request + assert (write_bytes_after - write_bytes_before) / ( + writes_after - writes_before + ) < max_bytes_per_request + assert (write_bytes_after - write_bytes_before) / ( + writes_after - writes_before + ) > min_bytes_per_request reads_before = int( node.query( @@ -77,7 +83,9 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" ).strip() ) - node.query(f"select count() from data where not ignore(*) SETTINGS workload='admin'") + node.query( + f"select count() from data where not ignore(*) SETTINGS workload='admin'" + ) reads_after = int( node.query( f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/admin'" @@ -91,8 +99,12 @@ def test_s3_resource_request_granularity(): assert read_bytes_after - read_bytes_before > 1.0 * total_bytes assert read_bytes_after - read_bytes_before < 1.2 * total_bytes - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) < max_bytes_per_request - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) > min_bytes_per_request + assert (read_bytes_after - read_bytes_before) / ( + reads_after - reads_before + ) < max_bytes_per_request + assert (read_bytes_after - read_bytes_before) / ( + reads_after - reads_before + ) > min_bytes_per_request def test_s3_disk(): From 937e1708259c518d27959961849da263163cc5a2 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 18:58:39 +0000 Subject: [PATCH 0135/1722] make ResourceGuards more straight-forward --- src/Common/HTTPConnectionPool.cpp | 10 +++--- .../Scheduler/Nodes/tests/ResourceTest.h | 9 ++++-- .../tests/gtest_dynamic_resource_manager.cpp | 7 ++++- .../Nodes/tests/gtest_resource_scheduler.cpp | 4 +++ src/Common/Scheduler/ResourceGuard.h | 31 ++++++++++++------- src/Common/Scheduler/ResourceRequest.h | 2 +- .../IO/ReadBufferFromAzureBlobStorage.cpp | 4 +-- .../IO/WriteBufferFromAzureBlobStorage.cpp | 6 +--- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 15 ++------- .../HDFS/WriteBufferFromHDFS.cpp | 15 ++------- 10 files changed, 50 insertions(+), 53 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index b6bf98894e4..18a6ab13c4f 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -276,21 +276,18 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks void start(int bytes) override { - // TODO(serxa): add metrics here or better in scheduler code (e.g. during enqueue, or better in REsourceGuard::REquest)? request.enqueue(bytes, link); request.wait(); } void finish(int bytes) override { - request.finish(); - link.adjust(request.cost, bytes); // success + request.finish(bytes, link); } void fail() override { - request.finish(); - link.accumulate(request.cost); // We assume no resource was used in case of failure + request.finish(0, link); } ResourceLink link; @@ -466,6 +463,9 @@ private: } } response_stream = nullptr; + // FIXME: We are not sure that response stream is fully read at this moment, so hooks could possible be called after this point, right? + // Session::setSendDataHooks(); + // Session::setReceiveDataHooks(); group->atConnectionDestroy(); diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index a5eb98f2a2f..c440cb176f8 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -232,12 +232,13 @@ struct ResourceTestManager : public ResourceTestBase ResourceTestManager & t; Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, PostponeLocking) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Postpone) , t(t_) { t.onEnqueue(link); lock(); t.onExecute(link); + consume(cost); } }; @@ -310,8 +311,9 @@ struct ResourceTestManager : public ResourceTestBase // NOTE: actually leader's request(s) make their own small busy period. void blockResource(ResourceLink link) { - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::Lock::Postpone); g.lock(); + g.consume(1); // NOTE: at this point we assume resource to be blocked by single request (1) busy_period.arrive_and_wait(); // (1) notify all followers that resource is blocked busy_period.arrive_and_wait(); // (2) wait all followers to enqueue their requests @@ -320,10 +322,11 @@ struct ResourceTestManager : public ResourceTestBase { getLinkData(link).left += total_requests + 1; busy_period.arrive_and_wait(); // (1) wait leader to block resource - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::Lock::Postpone); onEnqueue(link); busy_period.arrive_and_wait(); // (2) notify leader to unblock g.lock(); + g.consume(cost); onExecute(link); } }; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 4ac79977663..0f4aaab70aa 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -36,11 +36,16 @@ TEST(SchedulerDynamicResourceManager, Smoke) for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), ResourceGuard::PostponeLocking); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Postpone); gA.lock(); + gA.consume(1); gA.unlock(); ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); + gB.unlock(); + + ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1")); + gB.consume(2); } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ba573bf0c85..ff3054d6b7a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -111,21 +111,25 @@ TEST(SchedulerRoot, Smoke) { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); EXPECT_TRUE(fc1->requests.contains(&rg.request)); + rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); EXPECT_TRUE(fc1->requests.contains(&rg.request)); + rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); EXPECT_TRUE(fc2->requests.contains(&rg.request)); + rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); EXPECT_TRUE(fc2->requests.contains(&rg.request)); + rg.consume(1); } } diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 73aea4afdb6..93b6268a62c 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -42,12 +42,12 @@ namespace DB class ResourceGuard { public: - enum ResourceGuardCtor + enum class Lock { - LockStraightAway, /// Locks inside constructor (default) + StraightAway, /// Locks inside constructor (default) // WARNING: Only for tests. It is not exception-safe because `lock()` must be called after construction. - PostponeLocking /// Don't lock in constructor, but send request + Postpone /// Don't lock in constructor, but send request }; struct Metrics @@ -96,8 +96,6 @@ public: chassert(state == Finished); state = Enqueued; ResourceRequest::reset(cost_); - ProfileEvents::increment(metrics->requests); - ProfileEvents::increment(metrics->cost, cost_); link_.queue->enqueueRequestUsingBudget(this); } @@ -121,12 +119,16 @@ public: dequeued_cv.wait(lock, [this] { return state == Dequeued; }); } - void finish() + void finish(ResourceCost real_cost_, ResourceLink link_) { // lock(mutex) is not required because `Dequeued` request cannot be used by the scheduler thread chassert(state == Dequeued); state = Finished; + if (cost != real_cost_) + link_.adjust(cost, real_cost_); ResourceRequest::finish(); + ProfileEvents::increment(metrics->requests); + ProfileEvents::increment(metrics->cost, real_cost_); } void assertFinished() @@ -153,7 +155,7 @@ public: }; /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) - explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) + explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuard::Lock type = ResourceGuard::Lock::StraightAway) : link(link_) , request(Request::local(metrics)) { @@ -162,7 +164,7 @@ public: else if (link) { request.enqueue(cost, link); - if (ctor == LockStraightAway) + if (type == Lock::StraightAway) request.wait(); } } @@ -179,18 +181,25 @@ public: request.wait(); } - /// Report resource consumption has finished - void unlock() + void consume(ResourceCost cost) { + real_cost += cost; + } + + /// Report resource consumption has finished + void unlock(ResourceCost consumed = 0) + { + consume(consumed); if (link) { - request.finish(); + request.finish(real_cost, link); link.reset(); } } ResourceLink link; Request & request; + ResourceCost real_cost = 0; }; } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d64f624cec5..7b6a5af0fe6 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -45,7 +45,7 @@ constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); class ResourceRequest : public boost::intrusive::list_base_hook<> { public: - /// Cost of request execution; should be filled before request enqueueing. + /// Cost of request execution; should be filled before request enqueueing and remain constant until `finish()`. /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index ecc4168c729..626220a843e 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -116,15 +116,13 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() { ResourceGuard rlock(ResourceGuard::Metrics::getIORead(), read_settings.io_scheduling.read_resource_link, to_read_bytes); bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); - read_settings.io_scheduling.read_resource_link.adjust(to_read_bytes, bytes_read); - rlock.unlock(); // Do not hold resource under bandwidth throttler + rlock.unlock(bytes_read); // Do not hold resource under bandwidth throttler if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); break; } catch (const Azure::Core::RequestFailedException & e) { - read_settings.io_scheduling.read_resource_link.accumulate(to_read_bytes); // We assume no resource was used in case of failure ProfileEvents::increment(ProfileEvents::ReadBufferFromAzureRequestsErrors); LOG_DEBUG(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 82ce9d32f5a..d040200c31e 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -94,13 +94,11 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, cost); // Note that zero-cost requests are ignored func(); + rlock.unlock(cost); break; } catch (const Azure::Core::RequestFailedException & e) { - if (cost) - write_settings.io_scheduling.write_resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it - if (i == num_tries - 1 || !isRetryableAzureException(e)) throw; @@ -108,8 +106,6 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, } catch (...) { - if (cost) - write_settings.io_scheduling.write_resource_link.accumulate(cost); // We assume no resource was used in case of failure throw; } } diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index d1502577bf7..0fbd123508e 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -119,18 +119,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); - read_settings.io_scheduling.read_resource_link.adjust(num_bytes_to_read, std::max(0, bytes_read)); - } - catch (...) - { - read_settings.io_scheduling.read_resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure - throw; - } + ResourceGuard rlock(ResourceGuard::Metrics::getIORead(), read_settings.io_scheduling.read_resource_link, num_bytes_to_read); + int bytes_read = hdfsRead(fs.get(), fin, internal_buffer.begin(), safe_cast(num_bytes_to_read)); + rlock.unlock(std::max(0, bytes_read)); if (bytes_read < 0) { diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 65a5f45cd96..816de6676f1 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -66,18 +66,9 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) { - int bytes_written; - try - { - ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, size); - bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); - write_settings.io_scheduling.write_resource_link.adjust(size, std::max(0, bytes_written)); - } - catch (...) - { - write_settings.io_scheduling.write_resource_link.accumulate(size); // We assume no resource was used in case of failure - throw; - } + ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, size); + int bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + rlock.unlock(std::max(0, bytes_written)); if (bytes_written < 0) throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); From ba76a06f5677e7de556781a4c06cc947f392e0c5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 14 Jun 2024 01:35:08 +0000 Subject: [PATCH 0136/1722] potentially very serious bug is fixed for secure socket --- src/IO/ReadBufferFromPocoSocket.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index af58efc7e10..6361fed01dd 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -134,11 +134,14 @@ ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & s bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { - if (available()) + /// For secure socket it is important to check if any remaining data available in underlying decryption buffer - + /// read always retrives the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - + /// further poll() can block though there is still data to read in the underlying decryption buffer. + if (available() || socket.impl()->available()) return true; Stopwatch watch; - bool res = socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); + bool res = socket.impl()->poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); return res; } From 97aea863767a58fd65274777913865201ea906e3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 14 Jun 2024 01:56:05 +0000 Subject: [PATCH 0137/1722] fix style --- src/IO/ReadBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 6361fed01dd..bbf9f96404f 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -135,7 +135,7 @@ ReadBufferFromPocoSocketBase::ReadBufferFromPocoSocketBase(Poco::Net::Socket & s bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const { /// For secure socket it is important to check if any remaining data available in underlying decryption buffer - - /// read always retrives the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - + /// read always retrieves the whole encrypted frame from the wire and puts it into underlying buffer while returning only requested size - /// further poll() can block though there is still data to read in the underlying decryption buffer. if (available() || socket.impl()->available()) return true; From a5eeeb3422e956212c15382af13ca45012efc96b Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:01:14 +0000 Subject: [PATCH 0138/1722] bugfix: wrong estimated cost passed for budget adjusting --- src/Common/Scheduler/ISchedulerQueue.h | 19 +++++-------------- src/Common/Scheduler/ResouceLink.cpp | 25 ------------------------- src/Common/Scheduler/ResourceGuard.h | 7 ++++--- src/Common/Scheduler/ResourceLink.h | 4 ---- 4 files changed, 9 insertions(+), 46 deletions(-) delete mode 100644 src/Common/Scheduler/ResouceLink.cpp diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index 532f4bf6c63..b7a51870a24 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -22,10 +22,13 @@ public: {} // Wrapper for `enqueueRequest()` that should be used to account for available resource budget - void enqueueRequestUsingBudget(ResourceRequest * request) + // Returns `estimated_cost` that should be passed later to `adjustBudget()` + [[ nodiscard ]] ResourceCost enqueueRequestUsingBudget(ResourceRequest * request) { - request->cost = budget.ask(request->cost); + ResourceCost estimated_cost = request->cost; + request->cost = budget.ask(estimated_cost); enqueueRequest(request); + return estimated_cost; } // Should be called to account for difference between real and estimated costs @@ -34,18 +37,6 @@ public: budget.adjust(estimated_cost, real_cost); } - // Adjust budget to account for extra consumption of `cost` resource units - void consumeBudget(ResourceCost cost) - { - adjustBudget(0, cost); - } - - // Adjust budget to account for requested, but not consumed `cost` resource units - void accumulateBudget(ResourceCost cost) - { - adjustBudget(cost, 0); - } - /// Enqueue new request to be executed using underlying resource. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void enqueueRequest(ResourceRequest * request) = 0; diff --git a/src/Common/Scheduler/ResouceLink.cpp b/src/Common/Scheduler/ResouceLink.cpp deleted file mode 100644 index 2da5dba62dc..00000000000 --- a/src/Common/Scheduler/ResouceLink.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include -#include -#include - -namespace DB -{ -void ResourceLink::adjust(ResourceCost estimated_cost, ResourceCost real_cost) const -{ - if (queue) - queue->adjustBudget(estimated_cost, real_cost); -} - -void ResourceLink::consumed(ResourceCost cost) const -{ - if (queue) - queue->consumeBudget(cost); -} - -void ResourceLink::accumulate(DB::ResourceCost cost) const -{ - if (queue) - queue->accumulateBudget(cost); -} -} - diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 93b6268a62c..c46a3683455 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -96,7 +96,7 @@ public: chassert(state == Finished); state = Enqueued; ResourceRequest::reset(cost_); - link_.queue->enqueueRequestUsingBudget(this); + estimated_cost = link_.queue->enqueueRequestUsingBudget(this); // NOTE: it modifies `cost` and enqueues request } // This function is executed inside scheduler thread and wakes thread issued this `request`. @@ -124,8 +124,8 @@ public: // lock(mutex) is not required because `Dequeued` request cannot be used by the scheduler thread chassert(state == Dequeued); state = Finished; - if (cost != real_cost_) - link_.adjust(cost, real_cost_); + if (estimated_cost != real_cost_) + link_.queue->adjustBudget(estimated_cost, real_cost_); ResourceRequest::finish(); ProfileEvents::increment(metrics->requests); ProfileEvents::increment(metrics->cost, real_cost_); @@ -149,6 +149,7 @@ public: const Metrics * metrics = nullptr; // Must be initialized before use private: + ResourceCost estimated_cost = 0; // Stores initial `cost` value in case budget was used to modify it std::mutex mutex; std::condition_variable dequeued_cv; RequestState state = Finished; diff --git a/src/Common/Scheduler/ResourceLink.h b/src/Common/Scheduler/ResourceLink.h index 6dd3be930ca..a4e2adbd963 100644 --- a/src/Common/Scheduler/ResourceLink.h +++ b/src/Common/Scheduler/ResourceLink.h @@ -17,10 +17,6 @@ struct ResourceLink bool operator==(const ResourceLink &) const = default; explicit operator bool() const { return queue != nullptr; } - void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const; - void consumed(ResourceCost cost) const; - void accumulate(ResourceCost cost) const; - void reset() { queue = nullptr; From ec71d35aa64e22d62715d313f642b81f9e74f36c Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:12:23 +0000 Subject: [PATCH 0139/1722] add test for scheduler queue budget --- .../Nodes/tests/gtest_resource_scheduler.cpp | 42 ++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ff3054d6b7a..ddfe0cfbc6f 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -1,11 +1,13 @@ #include -#include - #include +#include +#include + #include #include +#include using namespace DB; @@ -22,6 +24,17 @@ struct ResourceTest : public ResourceTestBase { scheduler.stop(true); } + + std::mutex rng_mutex; + pcg64 rng{randomSeed()}; + + template + T randomInt(T from, T to) + { + std::uniform_int_distribution distribution(from, to); + std::lock_guard lock(rng_mutex); + return distribution(rng); + } }; struct ResourceHolder @@ -133,6 +146,31 @@ TEST(SchedulerRoot, Smoke) } } +TEST(SchedulerRoot, Budget) +{ + ResourceTest t; + + ResourceHolder r1(t); + r1.add("/", "1"); + r1.add("/prio"); + auto a = r1.addQueue("/prio/A", ""); + r1.registerResource(); + + ResourceCost total_real_cost = 0; + int total_requests = 10; + for (int i = 0 ; i < total_requests; i++) + { + ResourceCost est_cost = t.randomInt(1, 10); + ResourceCost real_cost = t.randomInt(0, 10); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a, est_cost); + rg.consume(real_cost); + total_real_cost += real_cost; + } + + EXPECT_EQ(total_requests, a.queue->dequeued_requests); + EXPECT_EQ(total_real_cost, a.queue->dequeued_cost - a.queue->getBudget()); +} + TEST(SchedulerRoot, Cancel) { ResourceTest t; From 14a13d54c0ff56b0e6326ac75bb7136e44d814d1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 14 Jun 2024 15:56:14 +0000 Subject: [PATCH 0140/1722] fix UB misaligned address --- src/IO/WriteBufferFromPocoSocketChunked.h | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 4325ab2bd4b..c668ea2c505 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -6,6 +6,18 @@ #include +namespace +{ + +template +const T & setValue(T * typed_ptr, std::type_identity_t val) +{ + memcpy(typed_ptr, &val, sizeof(T)); + return *typed_ptr; +} + +} + namespace DB { @@ -40,7 +52,7 @@ public: chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); - *chunk_size_ptr = 0; + setValue(chunk_size_ptr, 0); /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); pos += std::min(available(), sizeof(*chunk_size_ptr)); @@ -58,7 +70,7 @@ public: } /// Fill up current chunk size - *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); if (!chunk_started) LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", @@ -174,7 +186,7 @@ protected: pos -= sizeof(*chunk_size_ptr); else // fill up current chunk size { - *chunk_size_ptr = toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr))); + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); if (!chunk_started) { chunk_started = true; From 709ef2ba8539bfe0316817374e9fe4c0768e502b Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 16:09:00 +0000 Subject: [PATCH 0141/1722] add metrics and budget checks --- tests/integration/test_scheduler/test.py | 59 ++++++++++++++++++++---- 1 file changed, 51 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 9940e16ea42..c6338ec3eb1 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -28,6 +28,19 @@ def start_cluster(): cluster.shutdown() +def check_profile_event_for_query(workload, profile_event, amount=1): + node.query("system flush logs") + query_pattern = f"workload='{workload}'".replace("'", "\\'") + assert ( + int( + node.query( + f"select ProfileEvents['{profile_event}'] from system.query_log where query ilike '%{query_pattern}%' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" + ) + ) + == amount + ) + + def test_s3_resource_request_granularity(): node.query( f""" @@ -50,6 +63,11 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" ).strip() ) + write_budget_before = int( + node.query( + f"select budget from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) node.query(f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'") writes_after = int( node.query( @@ -61,11 +79,22 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" ).strip() ) + write_budget_after = int( + node.query( + f"select budget from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) - assert write_bytes_after - write_bytes_before > 1.0 * total_bytes - assert write_bytes_after - write_bytes_before < 1.2 * total_bytes - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) < max_bytes_per_request - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) > min_bytes_per_request + write_requests = writes_after - writes_before + write_bytes = (write_bytes_after - write_bytes_before) - (write_budget_after - write_budget_before) + assert write_bytes > 1.0 * total_bytes + assert write_bytes < 1.05 * total_bytes + assert write_bytes / write_requests < max_bytes_per_request + assert write_bytes / write_requests > min_bytes_per_request + check_profile_event_for_query("admin", "SchedulerIOWriteRequests", write_requests) + check_profile_event_for_query("admin", "SchedulerIOWriteBytes", write_bytes) + + node.query(f"optimize table data final") reads_before = int( node.query( @@ -77,6 +106,11 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" ).strip() ) + read_budget_before = int( + node.query( + f"select budget from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) node.query(f"select count() from data where not ignore(*) SETTINGS workload='admin'") reads_after = int( node.query( @@ -88,11 +122,20 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" ).strip() ) + read_budget_after = int( + node.query( + f"select budget from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) - assert read_bytes_after - read_bytes_before > 1.0 * total_bytes - assert read_bytes_after - read_bytes_before < 1.2 * total_bytes - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) < max_bytes_per_request - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) > min_bytes_per_request + read_bytes = (read_bytes_after - read_bytes_before) - (read_budget_after - read_budget_before) + read_requests = reads_after - reads_before + assert read_bytes > 1.0 * total_bytes + assert read_bytes < 1.05 * total_bytes + assert read_bytes / read_requests < max_bytes_per_request + assert read_bytes / read_requests > min_bytes_per_request + check_profile_event_for_query("admin", "SchedulerIOReadRequests", read_requests) + check_profile_event_for_query("admin", "SchedulerIOReadBytes", read_bytes) def test_s3_disk(): From f3c3d419bbce4f7edecf807fdbaca8557ee36d1f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 16:17:57 +0000 Subject: [PATCH 0142/1722] Automatic style fix --- tests/integration/test_scheduler/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 11525d3fd62..5c7da0e2516 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -88,7 +88,9 @@ def test_s3_resource_request_granularity(): ) write_requests = writes_after - writes_before - write_bytes = (write_bytes_after - write_bytes_before) - (write_budget_after - write_budget_before) + write_bytes = (write_bytes_after - write_bytes_before) - ( + write_budget_after - write_budget_before + ) assert write_bytes > 1.0 * total_bytes assert write_bytes < 1.05 * total_bytes assert write_bytes / write_requests < max_bytes_per_request @@ -132,7 +134,9 @@ def test_s3_resource_request_granularity(): ).strip() ) - read_bytes = (read_bytes_after - read_bytes_before) - (read_budget_after - read_budget_before) + read_bytes = (read_bytes_after - read_bytes_before) - ( + read_budget_after - read_budget_before + ) read_requests = reads_after - reads_before assert read_bytes > 1.0 * total_bytes assert read_bytes < 1.05 * total_bytes From 5b082051451356b2c1d3152489e5d51cd75d2d6a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 15 Jun 2024 00:22:51 +0000 Subject: [PATCH 0143/1722] some refactoring --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 207 ++++++++++++++++++ src/IO/WriteBufferFromPocoSocketChunked.h | 220 +------------------- 2 files changed, 217 insertions(+), 210 deletions(-) create mode 100644 src/IO/WriteBufferFromPocoSocketChunked.cpp diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp new file mode 100644 index 00000000000..324f8ae3a02 --- /dev/null +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -0,0 +1,207 @@ +#include +#include +#include + + +namespace +{ + +template +const T & setValue(T * typed_ptr, std::type_identity_t val) +{ + memcpy(typed_ptr, &val, sizeof(T)); + return *typed_ptr; +} + +} + +namespace DB +{ + +void WriteBufferFromPocoSocketChunked::enableChunked() +{ + chunked = true; + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); +} + +void WriteBufferFromPocoSocketChunked::finishChunk() +{ + if (!chunked) + return; + + if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) + { + if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk + return; + + /// If current chunk is empty it means we are finishing a chunk previously sent by next(), + /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. + /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer + /// so it should be a beginning of the buffer. + + chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); + + setValue(chunk_size_ptr, 0); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Previously finished chunk wasn't sent yet + if (last_finish_chunk == chunk_size_ptr) + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + } + + /// Fill up current chunk size + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); + + if (!chunk_started) + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + else + { + chunk_started = false; + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + + LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); + + if (available() < sizeof(*chunk_size_ptr)) + { + finishing = available(); + pos += available(); + chunk_size_ptr = reinterpret_cast(pos); + return; + } + + /// Buffer end-of-chunk + *reinterpret_cast(pos) = 0; + pos += sizeof(*chunk_size_ptr); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(pos); + pos += std::min(available(), sizeof(*chunk_size_ptr)); + + last_finish_chunk = chunk_size_ptr; +} + +WriteBufferFromPocoSocketChunked::~WriteBufferFromPocoSocketChunked() +{ + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void WriteBufferFromPocoSocketChunked::nextImpl() +{ + if (!chunked) + { + WriteBufferFromPocoSocket::nextImpl(); + return; + } + + /// next() after finishChunk ar the end of the buffer + if (finishing < sizeof(*chunk_size_ptr)) + { + pos -= finishing; + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Send end-of-chunk directly + UInt32 s = 0; + socketSendBytes(reinterpret_cast(&s), sizeof(s)); + + finishing = sizeof(*chunk_size_ptr); + + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Send end-of-chunk buffered by finishChunk + if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) + { + pos -= sizeof(*chunk_size_ptr); + /// Send end-of-chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = chunk_size_ptr; + + return; + } + + /// Prevent sending empty chunk + if (offset() == sizeof(*chunk_size_ptr)) + { + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + return; + } + + /// Finish chunk at the end of the buffer + if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) + { + pos = reinterpret_cast(chunk_size_ptr); + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; + + return; + } + + if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk + pos -= sizeof(*chunk_size_ptr); + else // fill up current chunk size + { + setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); + if (!chunk_started) + { + chunk_started = true; + LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", + ourAddress().toString(), peerAddress().toString(), + static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), + *chunk_size_ptr); + } + else + LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); + } + /// Send current chunk + WriteBufferFromPocoSocket::nextImpl(); + /// Initialize next chunk + chunk_size_ptr = reinterpret_cast(working_buffer.begin()); + nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); + + last_finish_chunk = nullptr; +} + +void WriteBufferFromPocoSocketChunked::finalizeImpl() +{ + if (offset() == sizeof(*chunk_size_ptr)) + pos -= sizeof(*chunk_size_ptr); + WriteBufferFromPocoSocket::finalizeImpl(); +} + +} diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index c668ea2c505..269c6d66dda 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -1,23 +1,9 @@ #pragma once -#include "base/defines.h" #include #include -#include -namespace -{ - -template -const T & setValue(T * typed_ptr, std::type_identity_t val) -{ - memcpy(typed_ptr, &val, sizeof(T)); - return *typed_ptr; -} - -} - namespace DB { @@ -27,208 +13,22 @@ public: explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} - void enableChunked() - { - chunked = true; - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(pos); - pos += std::min(available(), sizeof(*chunk_size_ptr)); - } - - void finishChunk() - { - if (!chunked) - return; - - if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) - { - if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk - return; - - /// If current chunk is empty it means we are finishing a chunk previously sent by next(), - /// we want to convert current chunk header into end-of-chunk marker and initialize next chunk. - /// We don't need to worry about if it's the end of the buffer because next() always sends the whole buffer - /// so it should be a beginning of the buffer. - - chassert(reinterpret_cast(chunk_size_ptr) == working_buffer.begin()); - - setValue(chunk_size_ptr, 0); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(pos); - pos += std::min(available(), sizeof(*chunk_size_ptr)); - - last_finish_chunk = chunk_size_ptr; - - return; - } - - /// Previously finished chunk wasn't sent yet - if (last_finish_chunk == chunk_size_ptr) - { - chunk_started = false; - LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); - } - - /// Fill up current chunk size - setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); - - if (!chunk_started) - LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", - ourAddress().toString(), peerAddress().toString(), - static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), - *chunk_size_ptr); - else - { - chunk_started = false; - LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); - } - - LOG_TEST(log, "{} -> {} Chunk send ended.", ourAddress().toString(), peerAddress().toString()); - - if (available() < sizeof(*chunk_size_ptr)) - { - finishing = available(); - pos += available(); - chunk_size_ptr = reinterpret_cast(pos); - return; - } - - /// Buffer end-of-chunk - *reinterpret_cast(pos) = 0; - pos += sizeof(*chunk_size_ptr); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(pos); - pos += std::min(available(), sizeof(*chunk_size_ptr)); - - last_finish_chunk = chunk_size_ptr; - } - - ~WriteBufferFromPocoSocketChunked() override - { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + void enableChunked(); + void finishChunk(); + ~WriteBufferFromPocoSocketChunked() override; protected: - void nextImpl() override - { - if (!chunked) - { - WriteBufferFromPocoSocket::nextImpl(); - return; - } + void nextImpl() override; + void finalizeImpl() override; + Poco::Net::SocketAddress peerAddress() const { return peer_address; } + Poco::Net::SocketAddress ourAddress() const { return our_address; } - /// next() after finishChunk ar the end of the buffer - if (finishing < sizeof(*chunk_size_ptr)) - { - pos -= finishing; - /// Send current chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Send end-of-chunk directly - UInt32 s = 0; - socketSendBytes(reinterpret_cast(&s), sizeof(s)); - - finishing = sizeof(*chunk_size_ptr); - - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = chunk_size_ptr; - - return; - } - - /// Send end-of-chunk buffered by finishChunk - if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) - { - pos -= sizeof(*chunk_size_ptr); - /// Send end-of-chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = chunk_size_ptr; - - return; - } - - /// Prevent sending empty chunk - if (offset() == sizeof(*chunk_size_ptr)) - { - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - return; - } - - /// Finish chunk at the end of the buffer - if (working_buffer.end() - reinterpret_cast(chunk_size_ptr) <= static_cast(sizeof(*chunk_size_ptr))) - { - pos = reinterpret_cast(chunk_size_ptr); - /// Send current chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = nullptr; - - return; - } - - if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk - pos -= sizeof(*chunk_size_ptr); - else // fill up current chunk size - { - setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); - if (!chunk_started) - { - chunk_started = true; - LOG_TEST(log, "{} -> {} Chunk send started. Message {}, size {}", - ourAddress().toString(), peerAddress().toString(), - static_cast(*(reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr))), - *chunk_size_ptr); - } - else - LOG_TEST(log, "{} -> {} Chunk send continued. Size {}", ourAddress().toString(), peerAddress().toString(), *chunk_size_ptr); - } - /// Send current chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = nullptr; - } - - void finalizeImpl() override - { - if (offset() == sizeof(*chunk_size_ptr)) - pos -= sizeof(*chunk_size_ptr); - WriteBufferFromPocoSocket::finalizeImpl(); - } - - Poco::Net::SocketAddress peerAddress() - { - return peer_address; - } - - Poco::Net::SocketAddress ourAddress() - { - return our_address; - } private: LoggerPtr log; bool chunked = false; - UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk - bool chunk_started = false; // chunk started flag - UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer + UInt32 * last_finish_chunk = nullptr; // pointer to the last chunk header created by finishChunk + bool chunk_started = false; // chunk started flag + UInt32 * chunk_size_ptr = nullptr; // pointer to the chunk size holder in the buffer size_t finishing = sizeof(*chunk_size_ptr); // indicates not enough buffer for end-of-chunk marker }; From e91dd71d4e55fe80d1c230a87eee7ad84333d9c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:01:06 +0200 Subject: [PATCH 0144/1722] Settings normalization --- src/Interpreters/InterpreterSetQuery.cpp | 26 ++++++++++-------------- src/Parsers/ParserQueryWithOutput.cpp | 11 ---------- 2 files changed, 11 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 7e68fc5c4c1..cac44c7747b 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -9,6 +9,7 @@ #include #include + namespace DB { @@ -45,9 +46,7 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel // It is flattened later, when we process UNION ALL/DISTINCT. const auto * last_select = children.back()->as(); if (last_select && last_select->settings()) - { InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(); - } } void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_) @@ -55,6 +54,16 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (!ast) return; + /// First apply the outermost settings. Then they could be overridden by deeper settings. + if (const auto * query_with_output = dynamic_cast(ast.get())) + { + if (query_with_output->settings_ast) + InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(); + + if (const auto * create_query = ast->as(); create_query && create_query->select) + applySettingsFromSelectWithUnion(create_query->select->as(), context_); + } + if (const auto * select_query = ast->as()) { if (auto new_settings = select_query->settings()) @@ -71,19 +80,6 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } - else if (const auto * query_with_output = dynamic_cast(ast.get())) - { - if (query_with_output->settings_ast) - InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(); - - if (const auto * create_query = ast->as()) - { - if (create_query->select) - { - applySettingsFromSelectWithUnion(create_query->select->as(), context_); - } - } - } else if (auto * insert_query = ast->as()) { context_->setInsertFormat(insert_query->format); diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6d8a1258555..ac8f7d560e0 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -198,16 +197,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) return false; query_with_output.children.push_back(query_with_output.settings_ast); - - // SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery) - // Pass them manually, to apply in InterpreterSelectQuery::initSettings() - if (query->as()) - { - auto settings = query_with_output.settings_ast->clone(); - assert_cast(settings.get())->print_in_format = false; - QueryWithOutputSettingsPushDownVisitor::Data data{settings}; - QueryWithOutputSettingsPushDownVisitor(data).visit(query); - } } else break; From f5da9e424075fa755edd1a869e199f4861011be2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:01:44 +0200 Subject: [PATCH 0145/1722] Add a test --- ...QueryWithOutputSettingsPushDownVisitor.cpp | 56 ------------------- .../QueryWithOutputSettingsPushDownVisitor.h | 39 ------------- .../03172_format_settings_clauses.reference | 14 +++++ .../03172_format_settings_clauses.sql | 30 ++++++++++ 4 files changed, 44 insertions(+), 95 deletions(-) delete mode 100644 src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp delete mode 100644 src/Parsers/QueryWithOutputSettingsPushDownVisitor.h create mode 100644 tests/queries/0_stateless/03172_format_settings_clauses.reference create mode 100644 tests/queries/0_stateless/03172_format_settings_clauses.sql diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp deleted file mode 100644 index 8cf0d0063ae..00000000000 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -bool QueryWithOutputSettingsPushDownMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) -{ - if (node->as()) - return true; - if (node->as()) - return true; - if (child->as()) - return true; - return false; -} - -void QueryWithOutputSettingsPushDownMatcher::visit(ASTPtr & ast, Data & data) -{ - if (auto * select_query = ast->as()) - visit(*select_query, ast, data); -} - -void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query, ASTPtr &, Data & data) -{ - ASTPtr select_settings_ast = select_query.settings(); - if (!select_settings_ast) - { - select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, data.settings_ast->clone()); - return; - } - - SettingsChanges & select_settings = select_settings_ast->as().changes; - SettingsChanges & settings = data.settings_ast->as().changes; - - for (auto & setting : settings) - { - auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) - { - return select_setting.name == setting.name; - }); - if (it == select_settings.end()) - select_settings.push_back(setting); - else - it->value = setting.value; - } -} - -} diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h deleted file mode 100644 index fde8a07b555..00000000000 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ASTSelectQuery; -struct SettingChange; -class SettingsChanges; - -/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: -/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput) -/// -/// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 -> -/// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1 -/// -/// Otherwise settings after FORMAT will not be applied. -class QueryWithOutputSettingsPushDownMatcher -{ -public: - using Visitor = InDepthNodeVisitor; - - struct Data - { - const ASTPtr & settings_ast; - }; - - static bool needChildVisit(ASTPtr & node, const ASTPtr & child); - static void visit(ASTPtr & ast, Data & data); - -private: - static void visit(ASTSelectQuery &, ASTPtr &, Data &); -}; - -using QueryWithOutputSettingsPushDownVisitor = QueryWithOutputSettingsPushDownMatcher::Visitor; - -} diff --git a/tests/queries/0_stateless/03172_format_settings_clauses.reference b/tests/queries/0_stateless/03172_format_settings_clauses.reference new file mode 100644 index 00000000000..8a98b137f4b --- /dev/null +++ b/tests/queries/0_stateless/03172_format_settings_clauses.reference @@ -0,0 +1,14 @@ +1 +2 +1 +2 +1 +2 +1 +1 +3 +3 +3 +3 +3 +1 diff --git a/tests/queries/0_stateless/03172_format_settings_clauses.sql b/tests/queries/0_stateless/03172_format_settings_clauses.sql new file mode 100644 index 00000000000..0d1aa4dcfbb --- /dev/null +++ b/tests/queries/0_stateless/03172_format_settings_clauses.sql @@ -0,0 +1,30 @@ +SET max_block_size = 10, max_threads = 1; + +-- Take the following example: +SELECT 1 UNION ALL SELECT 2 FORMAT TSV; + +-- Each subquery can be put in parentheses and have its own settings: +(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV; + +-- And the whole query can have settings: +(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV SETTINGS max_block_size = 3; + +-- A single query with output is parsed in the same way as the UNION ALL chain: +SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV SETTINGS max_block_size = 3; + +-- So while these forms have a slightly different meaning, they both exist: +SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV; +SELECT getSetting('max_block_size') FORMAT TSV SETTINGS max_block_size = 3; + +-- And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order. +-- But while this work: +(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) FORMAT TSV SETTINGS max_block_size = 3; + +-- This does not work automatically, unless we explicitly allow different orders: +(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) SETTINGS max_block_size = 3 FORMAT TSV; + +-- Inevitably, we allow this: +SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 SETTINGS max_block_size = 3 FORMAT TSV; +/*^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^*/ +-- Because this part is consumed into ASTSelectWithUnionQuery +-- and the rest into ASTQueryWithOutput. From 778807a8883901debc1bfeb72e17b37eb06bcaf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 08:22:36 +0200 Subject: [PATCH 0146/1722] Fix error; remove garbage --- src/Client/ClientBase.cpp | 33 +++------------------------------ 1 file changed, 3 insertions(+), 30 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 854cc3fef8b..ad4964b4b7c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -1937,41 +1938,13 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin global_context->setSettings(*old_settings); }); - auto apply_query_settings = [&](const IAST & settings_ast) - { - if (!old_settings) - old_settings.emplace(global_context->getSettingsRef()); - global_context->applySettingsChanges(settings_ast.as()->changes); - global_context->resetSettingsToDefaultValue(settings_ast.as()->default_settings); - }; - - const auto * insert = parsed_query->as(); - if (const auto * select = parsed_query->as(); select && select->settings()) - apply_query_settings(*select->settings()); - else if (const auto * select_with_union = parsed_query->as()) - { - const ASTs & children = select_with_union->list_of_selects->children; - if (!children.empty()) - { - // On the client it is enough to apply settings only for the - // last SELECT, since the only thing that is important to apply - // on the client is format settings. - const auto * last_select = children.back()->as(); - if (last_select && last_select->settings()) - { - apply_query_settings(*last_select->settings()); - } - } - } - else if (const auto * query_with_output = parsed_query->as(); query_with_output && query_with_output->settings_ast) - apply_query_settings(*query_with_output->settings_ast); - else if (insert && insert->settings_ast) - apply_query_settings(*insert->settings_ast); + InterpreterSetQuery::applySettingsFromQuery(parsed_query, global_context); if (!connection->checkConnected(connection_parameters.timeouts)) connect(); ASTPtr input_function; + const auto * insert = parsed_query->as(); if (insert && insert->select) insert->tryFindInputFunction(input_function); From 87a2e5f39018ef16ab6fdd79ad934dce6c45aaf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 08:30:55 +0200 Subject: [PATCH 0147/1722] Fix error --- src/Client/ClientBase.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ad4964b4b7c..958a1f50813 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1932,10 +1932,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin { /// Temporarily apply query settings to context. - std::optional old_settings; + Settings old_settings = global_context->getSettings(); SCOPE_EXIT_SAFE({ - if (old_settings) - global_context->setSettings(*old_settings); + global_context->setSettings(old_settings); }); InterpreterSetQuery::applySettingsFromQuery(parsed_query, global_context); From 67a539292e5e62f0bf470d11a5b224ef105bea02 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 08:33:11 +0200 Subject: [PATCH 0148/1722] Update test --- tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference | 4 ++-- tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference index 22405bf1866..a8b99666654 100644 --- a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference +++ b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference @@ -1,7 +1,7 @@ 1 1 1 -1 -1 +2 +1 2 2 diff --git a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh index b70c28422c9..173cc949500 100755 --- a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh +++ b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) FORMAT CSV SETTINGS max_block_size = 1' # push down append ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_compress_block_size = 1 FORMAT CSV SETTINGS max_block_size = 1' -# overwrite on push down (since these settings goes latest) +# not overwrite on push down ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 2 FORMAT CSV SETTINGS max_block_size = 1' # on push-down ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 1 FORMAT CSV' From 4817657375a37c374eb4be72793cba434c16a815 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 17 Jun 2024 23:43:03 +0200 Subject: [PATCH 0149/1722] Revert "Revert "Fix AWS ECS"" --- contrib/aws-crt-cpp | 2 +- .../ProxyConfigurationResolverProvider.cpp | 35 +++++++------------ .../ProxyConfigurationResolverProvider.h | 5 ++- src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- src/IO/S3/Client.cpp | 6 ++-- src/IO/S3/PocoHTTPClient.cpp | 20 +++++++++-- src/IO/S3/PocoHTTPClient.h | 14 ++++---- src/IO/S3/PocoHTTPClientFactory.cpp | 5 ++- .../0_stateless/03170_ecs_crash.reference | 4 +++ tests/queries/0_stateless/03170_ecs_crash.sh | 9 +++++ 10 files changed, 61 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/03170_ecs_crash.reference create mode 100755 tests/queries/0_stateless/03170_ecs_crash.sh diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index f532d6abc0d..0217761556a 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit f532d6abc0d2b0d8b5d6fe9e7c51eaedbe4afbd0 +Subproject commit 0217761556a7ba7ec537fe933d0ab1159096746e diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index b06073121e7..a46837bfdb9 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -112,9 +112,8 @@ namespace return configuration.has(config_prefix + ".uri"); } - /* - * New syntax requires protocol prefix " or " - * */ + /* New syntax requires protocol prefix " or " + */ std::optional getProtocolPrefix( ProxyConfiguration::Protocol request_protocol, const String & config_prefix, @@ -130,22 +129,18 @@ namespace return protocol_prefix; } - template std::optional calculatePrefixBasedOnSettingsSyntax( + bool new_syntax, ProxyConfiguration::Protocol request_protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ) { if (!configuration.has(config_prefix)) - { return std::nullopt; - } - if constexpr (new_syntax) - { + if (new_syntax) return getProtocolPrefix(request_protocol, config_prefix, configuration); - } return config_prefix; } @@ -155,24 +150,21 @@ std::shared_ptr ProxyConfigurationResolverProvider:: Protocol request_protocol, const Poco::Util::AbstractConfiguration & configuration) { - if (auto resolver = getFromSettings(request_protocol, "proxy", configuration)) - { + if (auto resolver = getFromSettings(true, request_protocol, "proxy", configuration)) return resolver; - } return std::make_shared( request_protocol, isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration)); } -template std::shared_ptr ProxyConfigurationResolverProvider::getFromSettings( + bool new_syntax, Protocol request_protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration -) + const Poco::Util::AbstractConfiguration & configuration) { - auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(request_protocol, config_prefix, configuration); + auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(new_syntax, request_protocol, config_prefix, configuration); if (!prefix_opt) { @@ -195,20 +187,17 @@ std::shared_ptr ProxyConfigurationResolverProvider:: std::shared_ptr ProxyConfigurationResolverProvider::getFromOldSettingsFormat( Protocol request_protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration -) + const Poco::Util::AbstractConfiguration & configuration) { - /* - * First try to get it from settings only using the combination of config_prefix and configuration. + /* First try to get it from settings only using the combination of config_prefix and configuration. * This logic exists for backward compatibility with old S3 storage specific proxy configuration. * */ - if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(request_protocol, config_prefix + ".proxy", configuration)) + if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(false, request_protocol, config_prefix + ".proxy", configuration)) { return resolver; } - /* - * In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. + /* In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. * Falls back to Environment resolver if no configuration is found. * */ return ProxyConfigurationResolverProvider::get(request_protocol, configuration); diff --git a/src/Common/ProxyConfigurationResolverProvider.h b/src/Common/ProxyConfigurationResolverProvider.h index ebf22f7e92a..357b218e499 100644 --- a/src/Common/ProxyConfigurationResolverProvider.h +++ b/src/Common/ProxyConfigurationResolverProvider.h @@ -33,12 +33,11 @@ public: ); private: - template static std::shared_ptr getFromSettings( + bool is_new_syntax, Protocol protocol, const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration - ); + const Poco::Util::AbstractConfiguration & configuration); }; } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 303ffb744b5..4f883a9b4ed 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -221,7 +221,7 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( if (iter == http_header_entries.end()) { - http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}", VERSION_STRING)); + http_header_entries.emplace_back(user_agent, fmt::format("ClickHouse/{}{}", VERSION_STRING, VERSION_OFFICIAL)); } if (!delay_initialization && use_external_buffer) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 9229342b8c1..cbb61deea9f 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -972,10 +972,10 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT { auto context = Context::getGlobalContextInstance(); chassert(context); - auto proxy_configuration_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::protocolFromString(protocol), context->getConfigRef()); + auto proxy_configuration_resolver = ProxyConfigurationResolverProvider::get(ProxyConfiguration::protocolFromString(protocol), context->getConfigRef()); - auto per_request_configuration = [=] () { return proxy_configuration_resolver->resolve(); }; - auto error_report = [=] (const DB::ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); }; + auto per_request_configuration = [=]{ return proxy_configuration_resolver->resolve(); }; + auto error_report = [=](const ProxyConfiguration & req) { proxy_configuration_resolver->errorReport(req); }; auto config = PocoHTTPClientConfiguration( per_request_configuration, diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 1cef43530e0..04982f14f36 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,4 +1,5 @@ #include +#include #include "config.h" #if USE_AWS_S3 @@ -17,6 +18,7 @@ #include #include #include +#include #include #include @@ -29,6 +31,7 @@ #include + static const int SUCCESS_RESPONSE_MIN = 200; static const int SUCCESS_RESPONSE_MAX = 299; @@ -84,7 +87,7 @@ namespace DB::S3 { PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( - std::function per_request_configuration_, + std::function per_request_configuration_, const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, @@ -94,7 +97,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, - std::function error_report_) + std::function error_report_) : per_request_configuration(per_request_configuration_) , force_region(force_region_) , remote_host_filter(remote_host_filter_) @@ -107,6 +110,8 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { + /// This is used to identify configurations created by us. + userAgent = std::string(VERSION_FULL) + VERSION_OFFICIAL; } void PocoHTTPClientConfiguration::updateSchemeAndRegion() @@ -166,6 +171,17 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config { } +PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration) + : timeouts(ConnectionTimeouts() + .withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000)) + .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) + .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) + .withTCPKeepAliveTimeout(Poco::Timespan( + client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))), + remote_host_filter(Context::getGlobalContextInstance()->getRemoteHostFilter()) +{ +} + std::shared_ptr PocoHTTPClient::MakeRequest( const std::shared_ptr & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 88251b964e2..18a21649167 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -38,7 +38,7 @@ class PocoHTTPClient; struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration { - std::function per_request_configuration; + std::function per_request_configuration; String force_region; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; @@ -54,13 +54,13 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT; size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST; - std::function error_report; + std::function error_report; void updateSchemeAndRegion(); private: PocoHTTPClientConfiguration( - std::function per_request_configuration_, + std::function per_request_configuration_, const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, @@ -70,8 +70,7 @@ private: bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, - std::function error_report_ - ); + std::function error_report_); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. friend ClientFactory; @@ -120,6 +119,7 @@ class PocoHTTPClient : public Aws::Http::HttpClient { public: explicit PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration); + explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & client_configuration); ~PocoHTTPClient() override = default; std::shared_ptr MakeRequest( @@ -166,8 +166,8 @@ protected: static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; - std::function per_request_configuration; - std::function error_report; + std::function per_request_configuration; + std::function error_report; ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index ef7af2d01ba..abec907778c 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -15,7 +15,10 @@ namespace DB::S3 std::shared_ptr PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const { - return std::make_shared(static_cast(client_configuration)); + if (client_configuration.userAgent.starts_with("ClickHouse")) + return std::make_shared(static_cast(client_configuration)); + else /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost. + return std::make_shared(client_configuration); } std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( diff --git a/tests/queries/0_stateless/03170_ecs_crash.reference b/tests/queries/0_stateless/03170_ecs_crash.reference new file mode 100644 index 00000000000..acd7c60768b --- /dev/null +++ b/tests/queries/0_stateless/03170_ecs_crash.reference @@ -0,0 +1,4 @@ +1 2 3 +4 5 6 +7 8 9 +0 0 0 diff --git a/tests/queries/0_stateless/03170_ecs_crash.sh b/tests/queries/0_stateless/03170_ecs_crash.sh new file mode 100755 index 00000000000..fa6870c4cf2 --- /dev/null +++ b/tests/queries/0_stateless/03170_ecs_crash.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Previous versions crashed in attempt to use this authentication method (regardless of whether it was able to authenticate): +AWS_CONTAINER_CREDENTIALS_FULL_URI=http://localhost:1338/latest/meta-data/container/security-credentials $CLICKHOUSE_LOCAL -q "select * from s3('http://localhost:11111/test/a.tsv')" From a9bfaf6454f38ce9e6ef57fdcaba864b47d49e9f Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 11:52:40 +0000 Subject: [PATCH 0150/1722] style --- src/Common/HTTPConnectionPool.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index a713bd14d62..bac12fd438d 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -383,16 +383,15 @@ private: auto idle = idleTime(); // Reset data hooks for IO scheduling - if (ResourceLink link = CurrentThread::getReadResourceLink()) { + if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); - } else { + else Session::setReceiveDataHooks(); - } - if (ResourceLink link = CurrentThread::getWriteResourceLink()) { + + if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); - } else { + else Session::setSendDataHooks(); - } std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); From 1df895f3dadcbb65d246927b79f42144e5fc1af2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jun 2024 14:04:40 +0200 Subject: [PATCH 0151/1722] Minor changes --- src/Access/SettingsConstraints.cpp | 8 ++++---- src/Interpreters/InterpreterSetQuery.cpp | 10 +++++----- src/Interpreters/InterpreterSetQuery.h | 2 +- .../03003_compatibility_setting_bad_value.sql | 1 - 4 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index a274f6b54f2..7506e365035 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -219,8 +219,8 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang }); } -template -bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) +template +bool getNewValueToCheck(const SettingsT & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) { Field current_value; bool has_current_value = current_settings.tryGet(change.name, current_value); @@ -230,12 +230,12 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel return false; if (throw_on_failure) - new_value = T::castValueUtil(change.name, change.value); + new_value = SettingsT::castValueUtil(change.name, change.value); else { try { - new_value = T::castValueUtil(change.name, change.value); + new_value = SettingsT::castValueUtil(change.name, change.value); } catch (...) { diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index cac44c7747b..15d4ba56d8d 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -46,7 +46,7 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel // It is flattened later, when we process UNION ALL/DISTINCT. const auto * last_select = children.back()->as(); if (last_select && last_select->settings()) - InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(); + InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(false); } void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_) @@ -58,7 +58,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (const auto * query_with_output = dynamic_cast(ast.get())) { if (query_with_output->settings_ast) - InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(); + InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(false); if (const auto * create_query = ast->as(); create_query && create_query->select) applySettingsFromSelectWithUnion(create_query->select->as(), context_); @@ -67,7 +67,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (const auto * select_query = ast->as()) { if (auto new_settings = select_query->settings()) - InterpreterSetQuery(new_settings, context_).executeForCurrentContext(); + InterpreterSetQuery(new_settings, context_).executeForCurrentContext(false); } else if (const auto * select_with_union_query = ast->as()) { @@ -76,7 +76,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta else if (const auto * explain_query = ast->as()) { if (explain_query->settings_ast) - InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(); + InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(false); applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } @@ -84,7 +84,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta { context_->setInsertFormat(insert_query->format); if (insert_query->settings_ast) - InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(); + InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(false); } } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index 2438762f347..f50105c39f4 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -23,7 +23,7 @@ public: /** Set setting for current context (query context). * It is used for interpretation of SETTINGS clause in SELECT query. */ - void executeForCurrentContext(bool ignore_setting_constraints = false); + void executeForCurrentContext(bool ignore_setting_constraints); bool supportsTransactions() const override { return true; } diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql index 48e98798c51..b9fbfd917fc 100644 --- a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql +++ b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql @@ -1,2 +1 @@ select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS} - From 11d54f4809a8b58773f13664e6b842cb6c7dce48 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jun 2024 15:18:53 +0200 Subject: [PATCH 0152/1722] Pass-through settings from the client --- programs/client/Client.cpp | 3 +++ programs/server/Server.cpp | 2 +- src/Access/AccessControl.cpp | 8 +++++++- src/Access/AccessControl.h | 5 ++++- src/Client/ClientBase.cpp | 1 - .../0_stateless/03003_compatibility_setting_bad_value.sql | 2 +- 6 files changed, 16 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index efe23d57478..22a035fbd71 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1184,6 +1184,9 @@ void Client::processConfig() global_context->setQueryKindInitial(); global_context->setQuotaClientKey(config().getString("quota_key", "")); global_context->setQueryKind(query_kind); + + /// Allow to pass-through unknown settings to the server. + global_context->getAccessControl().allowAllSettings(); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6414f7f6ea5..2f1d07790e1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1824,7 +1824,7 @@ try auto & access_control = global_context->getAccessControl(); try { - access_control.setUpFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); + access_control.setupFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); } catch (...) { diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index c3bb42160ad..9831621d6ac 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -264,7 +264,7 @@ AccessControl::AccessControl() AccessControl::~AccessControl() = default; -void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, +void AccessControl::setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_) { if (config_.has("custom_settings_prefixes")) @@ -852,4 +852,10 @@ const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const return *external_authenticators; } + +void AccessControl::allowAllSettings() +{ + custom_settings_prefixes->registerPrefixes({""}); +} + } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index d1537219a06..f408f6dfb0d 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -54,7 +54,7 @@ public: ~AccessControl() override; /// Initializes access storage (user directories). - void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, + void setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_); /// Parses access entities from a configuration loaded from users.xml. @@ -235,6 +235,9 @@ public: /// Gets manager of notifications. AccessChangesNotifier & getChangesNotifier(); + /// Allow all setting names - this can be used in clients to pass-through unknown settings to the server. + void allowAllSettings(); + private: class ContextAccessCache; class CustomSettingsPrefixes; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 958a1f50813..617a56cfd95 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2958,7 +2958,6 @@ void ClientBase::init(int argc, char ** argv) boost::replace_all(arg, "−", "--"); } - OptionsDescription options_description; options_description.main_description.emplace(createOptionsDescription("Main options", terminal_width)); diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql index b9fbfd917fc..3a09eec7452 100644 --- a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql +++ b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql @@ -1 +1 @@ -select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS} +select 42 settings compatibility=NULL; -- {clientError BAD_GET} From 7a01b8189cadb9b720182548c3c49f9267f8a606 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 19 Jun 2024 00:49:21 +0000 Subject: [PATCH 0153/1722] fix typo, renames --- base/poco/Net/include/Poco/Net/HTTPSession.h | 2 +- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 6 +++--- .../Nodes/tests/gtest_dynamic_resource_manager.cpp | 2 +- src/Common/Scheduler/ResourceGuard.h | 10 +++++----- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index b0e59443f9b..b25ad68cc67 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -49,7 +49,7 @@ namespace Net /// Called when sending/receiving of data `bytes` is successfully finished. virtual void fail() = 0; - /// If an error occured during send/receive `fail()` is called instead of `finish()`. + /// If an error occurred during send/receive `fail()` is called instead of `finish()`. }; diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index c440cb176f8..c787a686a09 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -232,7 +232,7 @@ struct ResourceTestManager : public ResourceTestBase ResourceTestManager & t; Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Postpone) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Defer) , t(t_) { t.onEnqueue(link); @@ -311,7 +311,7 @@ struct ResourceTestManager : public ResourceTestBase // NOTE: actually leader's request(s) make their own small busy period. void blockResource(ResourceLink link) { - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::Lock::Postpone); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::Lock::Defer); g.lock(); g.consume(1); // NOTE: at this point we assume resource to be blocked by single request (1) @@ -322,7 +322,7 @@ struct ResourceTestManager : public ResourceTestBase { getLinkData(link).left += total_requests + 1; busy_period.arrive_and_wait(); // (1) wait leader to block resource - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::Lock::Postpone); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::Lock::Defer); onEnqueue(link); busy_period.arrive_and_wait(); // (2) notify leader to unblock g.lock(); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 0f4aaab70aa..3328196cced 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -36,7 +36,7 @@ TEST(SchedulerDynamicResourceManager, Smoke) for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Postpone); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer); gA.lock(); gA.consume(1); gA.unlock(); diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index c46a3683455..2e735aae656 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -44,10 +44,10 @@ class ResourceGuard public: enum class Lock { - StraightAway, /// Locks inside constructor (default) + Default, /// Locks inside constructor // WARNING: Only for tests. It is not exception-safe because `lock()` must be called after construction. - Postpone /// Don't lock in constructor, but send request + Defer /// Don't lock in constructor, but send request }; struct Metrics @@ -155,8 +155,8 @@ public: RequestState state = Finished; }; - /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) - explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuard::Lock type = ResourceGuard::Lock::StraightAway) + /// Creates pending request for resource; blocks while resource is not available (unless `Lock::Defer`) + explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuard::Lock type = ResourceGuard::Lock::Default) : link(link_) , request(Request::local(metrics)) { @@ -165,7 +165,7 @@ public: else if (link) { request.enqueue(cost, link); - if (type == Lock::StraightAway) + if (type == Lock::Default) request.wait(); } } From 16c3e36b5a2f3203eb87161f4320ea5e70865fc4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jun 2024 22:50:52 +0200 Subject: [PATCH 0154/1722] Simplification --- src/Client/ClientBase.cpp | 142 +++++++++++++++++------------------ src/Client/ClientBase.h | 1 + src/Interpreters/Session.cpp | 8 +- src/Interpreters/Session.h | 3 +- 4 files changed, 74 insertions(+), 80 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 617a56cfd95..490a560de2d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -439,7 +439,7 @@ void ClientBase::sendExternalTables(ASTPtr parsed_query) std::vector data; for (auto & table : external_tables) - data.emplace_back(table.getData(global_context)); + data.emplace_back(table.getData(query_context)); connection->sendExternalTablesData(data); } @@ -652,10 +652,10 @@ try /// intermixed with data with parallel formatting. /// It may increase code complexity significantly. if (!extras_into_stdout || select_only_into_file) - output_format = global_context->getOutputFormatParallelIfPossible( + output_format = query_context->getOutputFormatParallelIfPossible( current_format, out_file_buf ? *out_file_buf : *out_buf, block); else - output_format = global_context->getOutputFormat( + output_format = query_context->getOutputFormat( current_format, out_file_buf ? *out_file_buf : *out_buf, block); output_format->setAutoFlush(); @@ -949,7 +949,7 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) /// But for asynchronous inserts we don't extract data, because it's needed /// to be done on server side in that case (for coalescing the data from multiple inserts on server side). const auto * insert = parsed_query->as(); - if (insert && isSyncInsertWithData(*insert, global_context)) + if (insert && isSyncInsertWithData(*insert, query_context)) query_to_execute = full_query.substr(0, insert->data - full_query.data()); else query_to_execute = full_query; @@ -1067,7 +1067,7 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } } - const auto & settings = global_context->getSettingsRef(); + const auto & settings = query_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; int retries_left = 10; @@ -1082,10 +1082,10 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa connection_parameters.timeouts, query, query_parameters, - global_context->getCurrentQueryId(), + query_context->getCurrentQueryId(), query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), + &query_context->getSettingsRef(), + &query_context->getClientInfo(), true, [&](const Progress & progress) { onProgress(progress); }); @@ -1275,7 +1275,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { - global_context->setSetting("session_timezone", tz); + query_context->setSetting("session_timezone", tz); } @@ -1471,13 +1471,13 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query) { - if (!global_context->hasInsertionTable() && insert_query.table) + if (!query_context->hasInsertionTable() && insert_query.table) { String table = insert_query.table->as().shortName(); if (!table.empty()) { String database = insert_query.database ? insert_query.database->as().shortName() : ""; - global_context->setInsertionTable(StorageID(database, table)); + query_context->setInsertionTable(StorageID(database, table)); } } } @@ -1528,7 +1528,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars const auto & parsed_insert_query = parsed_query->as(); if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && !isStdinNotEmptyAndValid(std_in)))) { - const auto & settings = global_context->getSettingsRef(); + const auto & settings = query_context->getSettingsRef(); if (settings.throw_if_no_data_to_insert) throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert"); else @@ -1542,10 +1542,10 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars connection_parameters.timeouts, query, query_parameters, - global_context->getCurrentQueryId(), + query_context->getCurrentQueryId(), query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), + &query_context->getSettingsRef(), + &query_context->getClientInfo(), true, [&](const Progress & progress) { onProgress(progress); }); @@ -1593,7 +1593,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des /// Set callback to be called on file progress. if (tty_buf) - progress_indication.setFileProgressCallback(global_context, *tty_buf); + progress_indication.setFileProgressCallback(query_context, *tty_buf); } /// If data fetched from file (maybe compressed file) @@ -1627,10 +1627,10 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des } StorageFile::CommonArguments args{ - WithContext(global_context), + WithContext(query_context), parsed_insert_query->table_id, current_format, - getFormatSettings(global_context), + getFormatSettings(query_context), compression_method, columns_for_storage_file, ConstraintsDescription{}, @@ -1638,7 +1638,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des {}, String{}, }; - StoragePtr storage = std::make_shared(in_file, global_context->getUserFilesPath(), args); + StoragePtr storage = std::make_shared(in_file, query_context->getUserFilesPath(), args); storage->startup(); SelectQueryInfo query_info; @@ -1647,18 +1647,18 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des auto metadata = storage->getInMemoryMetadataPtr(); QueryPlan plan; storage->read( - plan, - sample.getNames(), - storage->getStorageSnapshot(metadata, global_context), - query_info, - global_context, - {}, - global_context->getSettingsRef().max_block_size, - getNumberOfPhysicalCPUCores()); + plan, + sample.getNames(), + storage->getStorageSnapshot(metadata, query_context), + query_info, + query_context, + {}, + query_context->getSettingsRef().max_block_size, + getNumberOfPhysicalCPUCores()); auto builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(global_context), - BuildQueryPipelineSettings::fromContext(global_context)); + QueryPlanOptimizationSettings::fromContext(query_context), + BuildQueryPipelineSettings::fromContext(query_context)); QueryPlanResourceHolder resources; auto pipe = QueryPipelineBuilder::getPipe(std::move(*builder), resources); @@ -1719,14 +1719,14 @@ void ClientBase::sendDataFrom(ReadBuffer & buf, Block & sample, const ColumnsDes current_format = insert->format; } - auto source = global_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size); + auto source = query_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size); Pipe pipe(source); if (columns_description.hasDefaults()) { pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, columns_description, *source, global_context); + return std::make_shared(header, columns_description, *source, query_context); }); } @@ -1872,6 +1872,9 @@ void ClientBase::cancelQuery() void ClientBase::processParsedSingleQuery(const String & full_query, const String & query_to_execute, ASTPtr parsed_query, std::optional echo_query_, bool report_error) { + query_context = Context::createCopy(global_context); + query_context->makeQueryContext(); + resetOutput(); have_error = false; cancelled = false; @@ -1888,12 +1891,12 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (is_interactive) { - global_context->setCurrentQueryId(""); + query_context->setCurrentQueryId(""); // Generate a new query_id for (const auto & query_id_format : query_id_formats) { writeString(query_id_format.first, std_out); - writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", global_context->getCurrentQueryId())), std_out); + writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", query_context->getCurrentQueryId())), std_out); writeChar('\n', std_out); std_out.next(); } @@ -1920,7 +1923,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin auto password = auth_data->getPassword(); if (password) - global_context->getAccessControl().checkPasswordComplexityRules(*password); + query_context->getAccessControl().checkPasswordComplexityRules(*password); } } } @@ -1930,47 +1933,40 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); + /// Apply query settings to context, as they can affect the behavior on client-side. + InterpreterSetQuery::applySettingsFromQuery(parsed_query, query_context); + + if (!connection->checkConnected(connection_parameters.timeouts)) + connect(); + + ASTPtr input_function; + const auto * insert = parsed_query->as(); + if (insert && insert->select) + insert->tryFindInputFunction(input_function); + + bool is_async_insert_with_inlined_data = query_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); + + if (is_async_insert_with_inlined_data) { - /// Temporarily apply query settings to context. - Settings old_settings = global_context->getSettings(); - SCOPE_EXIT_SAFE({ - global_context->setSettings(old_settings); - }); + bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in); + bool have_external_data = have_data_in_stdin || insert->infile; - InterpreterSetQuery::applySettingsFromQuery(parsed_query, global_context); - - if (!connection->checkConnected(connection_parameters.timeouts)) - connect(); - - ASTPtr input_function; - const auto * insert = parsed_query->as(); - if (insert && insert->select) - insert->tryFindInputFunction(input_function); - - bool is_async_insert_with_inlined_data = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); - - if (is_async_insert_with_inlined_data) - { - bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in); - bool have_external_data = have_data_in_stdin || insert->infile; - - if (have_external_data) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Processing async inserts with both inlined and external data (from stdin or infile) is not supported"); - } - - /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. - if (insert && (!insert->select || input_function) && !is_async_insert_with_inlined_data) - { - if (input_function && insert->format.empty()) - throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()"); - - processInsertQuery(query_to_execute, parsed_query); - } - else - processOrdinaryQuery(query_to_execute, parsed_query); + if (have_external_data) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Processing async inserts with both inlined and external data (from stdin or infile) is not supported"); } + /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. + if (insert && (!insert->select || input_function) && !is_async_insert_with_inlined_data) + { + if (input_function && insert->format.empty()) + throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()"); + + processInsertQuery(query_to_execute, parsed_query); + } + else + processOrdinaryQuery(query_to_execute, parsed_query); + /// Do not change context (current DB, settings) in case of an exception. if (!have_error) { @@ -2651,10 +2647,8 @@ bool ClientBase::processMultiQueryFromFile(const String & file_name) if (!has_log_comment) { - Settings settings = global_context->getSettings(); /// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]" - settings.log_comment = fs::absolute(fs::path(file_name)); - global_context->setSettings(settings); + global_context->setSetting("log_comment", String(fs::absolute(fs::path(file_name)))); } return executeMultiQuery(queries_from_file); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 220fcddc038..228a9d65ea7 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -198,6 +198,7 @@ protected: /// since other members can use them. SharedContextHolder shared_context; ContextMutablePtr global_context; + ContextMutablePtr query_context; bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool is_multiquery = false; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 396562189e0..9dd686290db 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -532,7 +532,7 @@ ContextMutablePtr Session::makeSessionContext() session_context->checkSettingsConstraints(settings_from_auth_server, SettingSource::QUERY); session_context->applySettingsChanges(settings_from_auth_server); - recordLoginSucess(session_context); + recordLoginSuccess(session_context); return session_context; } @@ -596,7 +596,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); - recordLoginSucess(session_context); + recordLoginSuccess(session_context); return session_context; } @@ -672,13 +672,13 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t user = query_context->getUser(); /// Interserver does not create session context - recordLoginSucess(query_context); + recordLoginSuccess(query_context); return query_context; } -void Session::recordLoginSucess(ContextPtr login_context) const +void Session::recordLoginSuccess(ContextPtr login_context) const { if (notified_session_log_about_login) return; diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 14f6f806acd..fc41c78e666 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -102,8 +102,7 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; - void recordLoginSucess(ContextPtr login_context) const; - + void recordLoginSuccess(ContextPtr login_context) const; mutable bool notified_session_log_about_login = false; const UUID auth_id; From fa5d4cfea183c64e3ff088f922c0960a3c3951e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 00:41:14 +0200 Subject: [PATCH 0155/1722] Fix error --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 490a560de2d..8fcb9632be0 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -949,7 +949,7 @@ void ClientBase::processTextAsSingleQuery(const String & full_query) /// But for asynchronous inserts we don't extract data, because it's needed /// to be done on server side in that case (for coalescing the data from multiple inserts on server side). const auto * insert = parsed_query->as(); - if (insert && isSyncInsertWithData(*insert, query_context)) + if (insert && isSyncInsertWithData(*insert, global_context)) query_to_execute = full_query.substr(0, insert->data - full_query.data()); else query_to_execute = full_query; From a8d1d1ecff5f287beada5af68269e597f2be9df7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 14 Jun 2024 01:24:17 +0000 Subject: [PATCH 0156/1722] Allow query profiler period to be longer than 4 seconds --- src/Common/QueryProfiler.cpp | 20 ++++++++++---------- src/Common/QueryProfiler.h | 10 +++++----- src/Interpreters/ThreadStatusExt.cpp | 12 ++++++------ 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 746010b5462..78bc7ee7e54 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -110,7 +110,7 @@ namespace errno = saved_errno; } - [[maybe_unused]] constexpr UInt32 TIMER_PRECISION = 1e9; + [[maybe_unused]] constexpr UInt64 TIMER_PRECISION = 1e9; } namespace ErrorCodes @@ -167,18 +167,18 @@ void Timer::createIfNecessary(UInt64 thread_id, int clock_type, int pause_signal } } -void Timer::set(UInt32 period) +void Timer::set(UInt64 period) { /// Too high frequency can introduce infinite busy loop of signal handlers. We will limit maximum frequency (with 1000 signals per second). - period = std::max(period, 1000000); + period = std::max(period, 1000000); /// Randomize offset as uniform random value from 0 to period - 1. /// It will allow to sample short queries even if timer period is large. /// (For example, with period of 1 second, query with 50 ms duration will be sampled with 1 / 20 probability). /// It also helps to avoid interference (moire). - UInt32 period_rand = std::uniform_int_distribution(0, period)(thread_local_rng); + UInt64 period_rand = std::uniform_int_distribution(0, period)(thread_local_rng); - struct timespec interval{.tv_sec = period / TIMER_PRECISION, .tv_nsec = period % TIMER_PRECISION}; - struct timespec offset{.tv_sec = period_rand / TIMER_PRECISION, .tv_nsec = period_rand % TIMER_PRECISION}; + struct timespec interval{.tv_sec = time_t(period / TIMER_PRECISION), .tv_nsec = long(period % TIMER_PRECISION)}; + struct timespec offset{.tv_sec = time_t(period_rand / TIMER_PRECISION), .tv_nsec = long(period_rand % TIMER_PRECISION)}; struct itimerspec timer_spec = {.it_interval = interval, .it_value = offset}; if (timer_settime(*timer_id, 0, &timer_spec, nullptr)) @@ -229,7 +229,7 @@ void Timer::cleanup() template QueryProfilerBase::QueryProfilerBase( - [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) + [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt64 period, [[maybe_unused]] int pause_signal_) : log(getLogger("QueryProfiler")), pause_signal(pause_signal_) { #if defined(SANITIZER) @@ -270,7 +270,7 @@ QueryProfilerBase::QueryProfilerBase( template -void QueryProfilerBase::setPeriod([[maybe_unused]] UInt32 period_) +void QueryProfilerBase::setPeriod([[maybe_unused]] UInt64 period_) { #if defined(SANITIZER) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); @@ -307,7 +307,7 @@ void QueryProfilerBase::cleanup() template class QueryProfilerBase; template class QueryProfilerBase; -QueryProfilerReal::QueryProfilerReal(UInt64 thread_id, UInt32 period) +QueryProfilerReal::QueryProfilerReal(UInt64 thread_id, UInt64 period) : QueryProfilerBase(thread_id, CLOCK_MONOTONIC, period, SIGUSR1) {} @@ -320,7 +320,7 @@ void QueryProfilerReal::signalHandler(int sig, siginfo_t * info, void * context) writeTraceInfo(TraceType::Real, sig, info, context); } -QueryProfilerCPU::QueryProfilerCPU(UInt64 thread_id, UInt32 period) +QueryProfilerCPU::QueryProfilerCPU(UInt64 thread_id, UInt64 period) : QueryProfilerBase(thread_id, CLOCK_THREAD_CPUTIME_ID, period, SIGUSR2) {} diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index ea4cc73bca6..e3ab0b2e094 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -40,7 +40,7 @@ public: ~Timer(); void createIfNecessary(UInt64 thread_id, int clock_type, int pause_signal); - void set(UInt32 period); + void set(UInt64 period); void stop(); void cleanup(); @@ -54,10 +54,10 @@ template class QueryProfilerBase { public: - QueryProfilerBase(UInt64 thread_id, int clock_type, UInt32 period, int pause_signal_); + QueryProfilerBase(UInt64 thread_id, int clock_type, UInt64 period, int pause_signal_); ~QueryProfilerBase(); - void setPeriod(UInt32 period_); + void setPeriod(UInt64 period_); private: void cleanup(); @@ -76,7 +76,7 @@ private: class QueryProfilerReal : public QueryProfilerBase { public: - QueryProfilerReal(UInt64 thread_id, UInt32 period); /// NOLINT + QueryProfilerReal(UInt64 thread_id, UInt64 period); /// NOLINT static void signalHandler(int sig, siginfo_t * info, void * context); }; @@ -85,7 +85,7 @@ public: class QueryProfilerCPU : public QueryProfilerBase { public: - QueryProfilerCPU(UInt64 thread_id, UInt32 period); /// NOLINT + QueryProfilerCPU(UInt64 thread_id, UInt64 period); /// NOLINT static void signalHandler(int sig, siginfo_t * info, void * context); }; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 9ca521a4ab3..0ce183944cc 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -470,11 +470,11 @@ void ThreadStatus::initGlobalProfiler([[maybe_unused]] UInt64 global_profiler_re { if (global_profiler_real_time_period > 0) query_profiler_real = std::make_unique(thread_id, - /* period= */ static_cast(global_profiler_real_time_period)); + /* period= */ global_profiler_real_time_period); if (global_profiler_cpu_time_period > 0) query_profiler_cpu = std::make_unique(thread_id, - /* period= */ static_cast(global_profiler_cpu_time_period)); + /* period= */ global_profiler_cpu_time_period); } catch (...) { @@ -503,18 +503,18 @@ void ThreadStatus::initQueryProfiler() { if (!query_profiler_real) query_profiler_real = std::make_unique(thread_id, - /* period= */ static_cast(settings.query_profiler_real_time_period_ns)); + /* period= */ settings.query_profiler_real_time_period_ns); else - query_profiler_real->setPeriod(static_cast(settings.query_profiler_real_time_period_ns)); + query_profiler_real->setPeriod(settings.query_profiler_real_time_period_ns); } if (settings.query_profiler_cpu_time_period_ns > 0) { if (!query_profiler_cpu) query_profiler_cpu = std::make_unique(thread_id, - /* period= */ static_cast(settings.query_profiler_cpu_time_period_ns)); + /* period= */ settings.query_profiler_cpu_time_period_ns); else - query_profiler_cpu->setPeriod(static_cast(settings.query_profiler_cpu_time_period_ns)); + query_profiler_cpu->setPeriod(settings.query_profiler_cpu_time_period_ns); } } catch (...) From 26e396184a3bc1aca1d9501d32cad1ed56b77e9d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 20 Jun 2024 18:49:27 +0000 Subject: [PATCH 0157/1722] Try to appease clang-tidy --- src/Common/QueryProfiler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 78bc7ee7e54..85c92ec292d 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -177,8 +177,8 @@ void Timer::set(UInt64 period) /// It also helps to avoid interference (moire). UInt64 period_rand = std::uniform_int_distribution(0, period)(thread_local_rng); - struct timespec interval{.tv_sec = time_t(period / TIMER_PRECISION), .tv_nsec = long(period % TIMER_PRECISION)}; - struct timespec offset{.tv_sec = time_t(period_rand / TIMER_PRECISION), .tv_nsec = long(period_rand % TIMER_PRECISION)}; + struct timespec interval{.tv_sec = time_t(period / TIMER_PRECISION), .tv_nsec = int64_t(period % TIMER_PRECISION)}; + struct timespec offset{.tv_sec = time_t(period_rand / TIMER_PRECISION), .tv_nsec = int64_t(period_rand % TIMER_PRECISION)}; struct itimerspec timer_spec = {.it_interval = interval, .it_value = offset}; if (timer_settime(*timer_id, 0, &timer_spec, nullptr)) From f9b70ea77a3b3059d9c784e2fe4b90000e75d3d4 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sat, 22 Jun 2024 09:30:16 +0200 Subject: [PATCH 0158/1722] Add JSONCompactWithProgressRowOutputFormat --- src/Formats/registerFormats.cpp | 2 + ...JSONCompactWithProgressRowOutputFormat.cpp | 125 ++++++++++++++++++ .../JSONCompactWithProgressRowOutputFormat.h | 53 ++++++++ 3 files changed, 180 insertions(+) create mode 100644 src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 57ca1bb49c8..770b747fafd 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -95,6 +95,7 @@ void registerOutputFormatMarkdown(FormatFactory & factory); void registerOutputFormatPostgreSQLWire(FormatFactory & factory); void registerOutputFormatPrometheus(FormatFactory & factory); void registerOutputFormatSQLInsert(FormatFactory & factory); +void registerOutputFormatJSONCompactWithProgress(FormatFactory & factory); /// Input only formats. @@ -242,6 +243,7 @@ void registerFormats() registerOutputFormatCapnProto(factory); registerOutputFormatPrometheus(factory); registerOutputFormatSQLInsert(factory); + registerOutputFormatJSONCompactWithProgress(factory); registerInputFormatRegexp(factory); registerInputFormatJSONAsString(factory); diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp new file mode 100644 index 00000000000..78cf5b9a003 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -0,0 +1,125 @@ +#include +#include +#include + +#include + +#include + + + +namespace DB +{ + +JSONCompactWithProgressRowOutputFormat::JSONCompactWithProgressRowOutputFormat( + WriteBuffer & out_, + const Block & header, + const FormatSettings & settings_, + bool yield_strings_) + : JSONRowOutputFormat(out_, header, settings_, yield_strings_) +{ +} + +void JSONCompactWithProgressRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) +{ + JSONUtils::writeFieldFromColumn(column, serialization, row_num, yield_strings, settings, *ostr); + ++field_number; + LOG_DEBUG(getLogger("JSONCompactWithProgressRowOutputFormat"), "Field number: {}", field_number); +} + +void JSONCompactWithProgressRowOutputFormat::writeFieldDelimiter() +{ + JSONUtils::writeFieldCompactDelimiter(*ostr); +} + +void JSONCompactWithProgressRowOutputFormat::writeRowStartDelimiter() +{ + if (has_progress) + writeProgress(); + JSONUtils::writeCompactArrayStart(*ostr, 2); +} + +void JSONCompactWithProgressRowOutputFormat::writeRowEndDelimiter() +{ + JSONUtils::writeCompactArrayEnd(*ostr); + field_number = 0; + ++row_count; +} + +void JSONCompactWithProgressRowOutputFormat::writeBeforeTotals() +{ + JSONUtils::writeFieldDelimiter(*ostr, 2); + JSONUtils::writeCompactArrayStart(*ostr, 1, "totals"); +} + +void JSONCompactWithProgressRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + JSONUtils::writeCompactColumns(columns, serializations, row_num, yield_strings, settings, *ostr); +} + +void JSONCompactWithProgressRowOutputFormat::writeAfterTotals() +{ + JSONUtils::writeCompactArrayEnd(*ostr); +} + +void JSONCompactWithProgressRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + JSONUtils::writeCompactArrayStart(*ostr, 2, title); + JSONUtils::writeCompactColumns(columns, serializations, row_num, yield_strings, settings, *ostr); + JSONUtils::writeCompactArrayEnd(*ostr); +} + +void JSONCompactWithProgressRowOutputFormat::onProgress(const Progress & value) +{ + LOG_DEBUG(getLogger("JSONCompactWithProgressRowOutputFormat"), "onProgress: {}", value.read_rows); + + progress.incrementPiecewiseAtomically(value); + String progress_line; + WriteBufferFromString buf(progress_line); + writeCString("{\"progress\":", buf); + progress.writeJSON(buf); + writeCString("}\n", buf); + buf.finalize(); + std::lock_guard lock(progress_lines_mutex); + progress_lines.emplace_back(std::move(progress_line)); + has_progress = true; +} + + +void JSONCompactWithProgressRowOutputFormat::flush() +{ + if (has_progress) + writeProgress(); + JSONRowOutputFormat::flush(); +} + +void JSONCompactWithProgressRowOutputFormat::writeSuffix() +{ + if (has_progress) + writeProgress(); + JSONRowOutputFormat::writeSuffix(); +} + +void JSONCompactWithProgressRowOutputFormat::writeProgress() +{ + std::lock_guard lock(progress_lines_mutex); + for (const auto & progress_line : progress_lines) + writeString(progress_line, *ostr); + progress_lines.clear(); + has_progress = false; +} + +void registerOutputFormatJSONCompactWithProgress(FormatFactory & factory) +{ + factory.registerOutputFormat("JSONCompactWithProgress", []( + WriteBuffer & buf, + const Block & sample, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings, false); + }); + + factory.markOutputFormatSupportsParallelFormatting("JSONCompactWithProgress"); +} + +} diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h new file mode 100644 index 00000000000..4bc10d41f19 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct FormatSettings; + +/** The stream for outputting data in the JSONCompact- formats. + */ +class JSONCompactWithProgressRowOutputFormat final : public JSONRowOutputFormat +{ +public: + JSONCompactWithProgressRowOutputFormat( + WriteBuffer & out_, + const Block & header, + const FormatSettings & settings_, + bool yield_strings_); + + String getName() const override { return "JSONCompactWithProgressRowOutputFormat"; } + + void onProgress(const Progress & value) override; + void flush() override; + +private: + void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + bool supportTotals() const override { return true; } + bool supportExtremes() const override { return true; } + void writeBeforeTotals() override; + void writeAfterTotals() override; + void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; + void writeTotals(const Columns & columns, size_t row_num) override; + + void writeProgress(); + void writeSuffix() override; + + Progress progress; + std::vector progress_lines; + std::mutex progress_lines_mutex; + /// To not lock mutex and check progress_lines every row, + /// we will use atomic flag that progress_lines is not empty. + std::atomic_bool has_progress = false; +}; + +} From e064171a68fcac110d27ff36a51b7a6bb4fbb251 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 03:58:11 +0200 Subject: [PATCH 0159/1722] Fix errors --- programs/client/Client.cpp | 10 ---------- programs/local/LocalServer.cpp | 3 --- src/Client/ClientBase.cpp | 4 +++- src/Client/LocalConnection.cpp | 1 - src/Client/LocalConnection.h | 2 -- .../0_stateless/00857_global_joinsavel_table_alias.sql | 1 - 6 files changed, 3 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 22a035fbd71..ab02d9fac74 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -19,7 +19,6 @@ #include #include -#include #include #include #include @@ -311,7 +310,6 @@ int Client::main(const std::vector & /*args*/) try { UseSSL use_ssl; - auto & thread_status = MainThreadStatus::getInstance(); setupSignalHandler(); std::cout << std::fixed << std::setprecision(3); @@ -326,14 +324,6 @@ try initTTYBuffer(toProgressOption(config().getString("progress", "default"))); ASTAlterCommand::setFormatAlterCommandsWithParentheses(true); - { - // All that just to set DB::CurrentThread::get().getGlobalContext() - // which is required for client timezone (pushed from server) to work. - auto thread_group = std::make_shared(); - const_cast(thread_group->global_context) = global_context; - thread_status.attachToGroup(thread_group, false); - } - /// Includes delayed_interactive. if (is_interactive) { diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index cb1c35743b2..e5f4bac852c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -27,10 +27,8 @@ #include #include #include -#include #include #include -#include #include #include #include @@ -48,7 +46,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 38aa0ed8b14..03f088f2b61 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1872,7 +1872,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin ASTPtr parsed_query, std::optional echo_query_, bool report_error) { query_context = Context::createCopy(global_context); - query_context->makeQueryContext(); + CurrentThread::QueryScope query_scope(query_context); resetOutput(); have_error = false; @@ -2926,6 +2926,8 @@ void ClientBase::init(int argc, char ** argv) /// Don't parse options with Poco library, we prefer neat boost::program_options. stopOptionsProcessing(); + MainThreadStatus::getInstance(); + stdin_is_a_tty = isatty(STDIN_FILENO); stdout_is_a_tty = isatty(STDOUT_FILENO); stderr_is_a_tty = isatty(STDERR_FILENO); diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index c7494e31605..e63e5793505 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -125,7 +125,6 @@ void LocalConnection::sendQuery( state->query_id = query_id; state->query = query; - state->query_scope_holder = std::make_unique(query_context); state->stage = QueryProcessingStage::Enum(stage); state->profile_queue = std::make_shared(std::numeric_limits::max()); CurrentThread::attachInternalProfileEventsQueue(state->profile_queue); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 899d134cce5..bdd0b481529 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -61,8 +61,6 @@ struct LocalQueryState /// Time after the last check to stop the request and send the progress. Stopwatch after_send_progress; Stopwatch after_send_profile_events; - - std::unique_ptr query_scope_holder; }; diff --git a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql index 2044a9b8d22..092b071cb48 100644 --- a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql +++ b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql @@ -1,4 +1,3 @@ - DROP TABLE IF EXISTS local_table; DROP TABLE IF EXISTS other_table; From 20860ed8b04f6588de12b4e59baf932f02f90d27 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 23 Jun 2024 14:34:54 +0200 Subject: [PATCH 0160/1722] Enable onProgress calls to JSONCompactWithProgressRowOutputFormat --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 78cf5b9a003..2cc7d99f8e7 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -118,8 +118,6 @@ void registerOutputFormatJSONCompactWithProgress(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings, false); }); - - factory.markOutputFormatSupportsParallelFormatting("JSONCompactWithProgress"); } } From 84f81c61853f34d765475309932d73af55e25d0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 17:18:32 +0200 Subject: [PATCH 0161/1722] Fix error --- src/Client/Suggest.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 0188ebc8173..c1f163939e8 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -96,6 +96,10 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p loading_thread = std::thread([my_context = Context::createCopy(context), connection_parameters, suggestion_limit, this] { ThreadStatus thread_status; + my_context->makeQueryContext(); + auto group = ThreadGroup::createForQuery(my_context); + CurrentThread::attachToGroup(group); + for (size_t retry = 0; retry < 10; ++retry) { try From 602fa5cbadba2924bc4e57f26f5f37b00d7b086e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 17:28:54 +0200 Subject: [PATCH 0162/1722] Fix error --- programs/local/LocalServer.cpp | 1 - src/Client/ClientBase.h | 3 --- 2 files changed, 4 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e5f4bac852c..45641b999b6 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -428,7 +428,6 @@ int LocalServer::main(const std::vector & /*args*/) try { UseSSL use_ssl; - thread_status.emplace(); StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true)); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 228a9d65ea7..83b99696373 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -244,9 +244,6 @@ protected: Settings cmd_settings; MergeTreeSettings cmd_merge_tree_settings; - /// thread status should be destructed before shared context because it relies on process list. - std::optional thread_status; - ServerConnectionPtr connection; ConnectionParameters connection_parameters; From e0ef26285a5cda87e5073c6a0aaecb67f9609a96 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 19:46:00 +0200 Subject: [PATCH 0163/1722] Update submodule --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 1c2946bfcb7..6463c9cbf47 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 1c2946bfcb7f1e3ae0a858de0b59d4f1a7b4ccaf +Subproject commit 6463c9cbf47cab78e4a4fa97a866942f201c6a58 From 9948150b87c6ee4531e0130de095035e2f228ec1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jun 2024 00:19:20 +0200 Subject: [PATCH 0164/1722] Fix error --- src/IO/S3/PocoHTTPClient.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 18a21649167..3b7ec4d1d9c 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -172,7 +172,7 @@ protected: const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; bool s3_use_adaptive_timeouts = true; - bool enable_s3_requests_logging; + bool enable_s3_requests_logging = false; bool for_disk_s3; /// Limits get request per second rate for GET, SELECT and all other requests, excluding throttled by put throttler From e1c60c4e40a03db459e71dd2b54b082d5205654d Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Mon, 24 Jun 2024 10:11:46 +0200 Subject: [PATCH 0165/1722] Remove debug output --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 2cc7d99f8e7..39532fb76fb 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -4,9 +4,6 @@ #include -#include - - namespace DB { @@ -24,7 +21,6 @@ void JSONCompactWithProgressRowOutputFormat::writeField(const IColumn & column, { JSONUtils::writeFieldFromColumn(column, serialization, row_num, yield_strings, settings, *ostr); ++field_number; - LOG_DEBUG(getLogger("JSONCompactWithProgressRowOutputFormat"), "Field number: {}", field_number); } void JSONCompactWithProgressRowOutputFormat::writeFieldDelimiter() @@ -71,8 +67,6 @@ void JSONCompactWithProgressRowOutputFormat::writeExtremesElement(const char * t void JSONCompactWithProgressRowOutputFormat::onProgress(const Progress & value) { - LOG_DEBUG(getLogger("JSONCompactWithProgressRowOutputFormat"), "onProgress: {}", value.read_rows); - progress.incrementPiecewiseAtomically(value); String progress_line; WriteBufferFromString buf(progress_line); From 384aa9feb90bbf95c5bc0e5498af4aca769c2531 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jun 2024 13:51:20 +0200 Subject: [PATCH 0166/1722] Move setting to 24.7 changes --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 1ab7dc69f60..deaeba2a7de 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,6 +86,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static const std::map settings_changes_history = { + {"24.7", {{"input_format_try_infer_variants", 0, 0, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, + }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, @@ -97,7 +99,6 @@ static const std::map Date: Tue, 25 Jun 2024 02:49:40 +0200 Subject: [PATCH 0167/1722] Update submodule --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 1c2946bfcb7..d5450d76abd 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 1c2946bfcb7f1e3ae0a858de0b59d4f1a7b4ccaf +Subproject commit d5450d76abda556ce145ddabe7e0cc6a7644ec59 From 49634db3ba2961dadbdc1689f0a4ef1ecdb8bea1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Jun 2024 02:51:57 +0200 Subject: [PATCH 0168/1722] Update submodule --- contrib/aws-crt-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index f532d6abc0d..e5aa45cacfd 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit f532d6abc0d2b0d8b5d6fe9e7c51eaedbe4afbd0 +Subproject commit e5aa45cacfdcda7719ead38760e7c61076f5745f From 5e4d3b2e43b04eb0bc0e44c082c86612127726e6 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jan 2024 00:05:57 +0000 Subject: [PATCH 0169/1722] Refreshable materialized views improvements --- .../system-tables/view_refreshes.md | 3 +- .../sql-reference/statements/create/view.md | 28 +++- docs/en/sql-reference/statements/system.md | 6 +- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 4 + src/Parsers/ASTRefreshStrategy.cpp | 3 +- src/Parsers/ASTRefreshStrategy.h | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserRefreshStrategy.cpp | 9 ++ src/Parsers/ParserSystemQuery.cpp | 1 + src/Storages/MaterializedView/RefreshSet.h | 9 +- .../MaterializedView/RefreshSettings.h | 6 +- src/Storages/MaterializedView/RefreshTask.cpp | 144 +++++++++++------- src/Storages/MaterializedView/RefreshTask.h | 28 ++-- .../MaterializedView/RefreshTask_fwd.h | 2 - src/Storages/StorageMaterializedView.cpp | 75 +++++---- src/Storages/StorageMaterializedView.h | 4 +- .../System/StorageSystemViewRefreshes.cpp | 4 +- ...2_refreshable_materialized_views.reference | 33 +++- .../02932_refreshable_materialized_views.sh | 110 ++++++++++--- 20 files changed, 327 insertions(+), 145 deletions(-) diff --git a/docs/en/operations/system-tables/view_refreshes.md b/docs/en/operations/system-tables/view_refreshes.md index 12377507b39..e792e0d095d 100644 --- a/docs/en/operations/system-tables/view_refreshes.md +++ b/docs/en/operations/system-tables/view_refreshes.md @@ -17,7 +17,8 @@ Columns: - `duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — How long the last refresh attempt took. - `next_refresh_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time at which the next refresh is scheduled to start. - `remaining_dependencies` ([Array(String)](../../sql-reference/data-types/array.md)) — If the view has [refresh dependencies](../../sql-reference/statements/create/view.md#refresh-dependencies), this array contains the subset of those dependencies that are not satisfied for the current refresh yet. If `status = 'WaitingForDependencies'`, a refresh is ready to start as soon as these dependencies are fulfilled. -- `exception` ([String](../../sql-reference/data-types/string.md)) — if `last_refresh_result = 'Exception'`, i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace. +- `exception` ([String](../../sql-reference/data-types/string.md)) — if `last_refresh_result = 'Error'`, i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace. +- `retry` ([UInt64](../../sql-reference/data-types/int-uint.md)) — If nonzero, the current or next refresh is a retry (see `refresh_retries` refresh setting), and `retry` is the 1-based index of that retry. - `refresh_count` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of successful refreshes since last server restart or table creation. - `progress` ([Float64](../../sql-reference/data-types/float.md)) — Progress of the current refresh, between 0 and 1. - `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of rows read by the current refresh so far. diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 1fabb6d8cc7..3e0766794f1 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -159,6 +159,8 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name REFRESH EVERY|AFTER interval [OFFSET interval] RANDOMIZE FOR interval DEPENDS ON [db.]name [, [db.]name [, ...]] +SETTINGS name = value [, name = value [, ...]] +[APPEND] [TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY] AS SELECT ... ``` @@ -167,18 +169,23 @@ where `interval` is a sequence of simple intervals: number SECOND|MINUTE|HOUR|DAY|WEEK|MONTH|YEAR ``` -Periodically runs the corresponding query and stores its result in a table, atomically replacing the table's previous contents. +Periodically runs the corresponding query and stores its result in a table. + * If the query says `APPEND`, each refresh inserts rows into the table without deleting existing rows. The insert is not atomic, just like a regular INSERT SELECT. + * Otherwise each refresh atomically replaces the table's previous contents. Differences from regular non-refreshable materialized views: - * No insert trigger. I.e. when new data is inserted into the table specified in SELECT, it's *not* automatically pushed to the refreshable materialized view. The periodic refresh runs the entire query and replaces the entire table. + * No insert trigger. I.e. when new data is inserted into the table specified in SELECT, it's *not* automatically pushed to the refreshable materialized view. The periodic refresh runs the entire query. * No restrictions on the SELECT query. Table functions (e.g. `url()`), views, UNION, JOIN, are all allowed. +:::note +The settings in the The settings in `REFRESH ... SETTINGS` part of the query are refresh settings (e.g. `refresh_retries`), distinct from regular settings (e.g. `max_threads`). Regular settings can be specified using `SETTINGS` at the end of the query. +::: + :::note Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations: * not compatible with Replicated database or table engines * It is not supported in ClickHouse Cloud * require [Atomic database engine](../../../engines/database-engines/atomic.md), - * no retries for failed refresh - we just skip to the next scheduled refresh time, * no limit on number of concurrent refreshes. ::: @@ -243,15 +250,22 @@ A few more examples: `DEPENDS ON` only works between refreshable materialized views. Listing a regular table in the `DEPENDS ON` list will prevent the view from ever refreshing (dependencies can be removed with `ALTER`, see below). ::: +### Settings + +Available refresh settings: + * `refresh_retries` - How many times to retry if refresh query fails with an exception. If all retries fail, skip to the next scheduled refresh time. 0 means no retries, -1 means infinite retries. Default: 0. + * `refresh_retry_initial_backoff_ms` - Delay before the first retry, if `refresh_retries` is not zero. Each subsequent retry doubles the delay, up to `refresh_retry_max_backoff_ms`. Default: 100 ms. + * `refresh_retry_max_backoff_ms` - Limit on the exponential growth of delay between refresh attempts. Default: 60000 ms (1 minute). + ### Changing Refresh Parameters {#changing-refresh-parameters} To change refresh parameters: ``` -ALTER TABLE [db.]name MODIFY REFRESH EVERY|AFTER ... [RANDOMIZE FOR ...] [DEPENDS ON ...] +ALTER TABLE [db.]name MODIFY REFRESH EVERY|AFTER ... [RANDOMIZE FOR ...] [DEPENDS ON ...] [SETTINGS ...] ``` :::note -This replaces refresh schedule *and* dependencies. If the table had a `DEPENDS ON`, doing a `MODIFY REFRESH` without `DEPENDS ON` will remove the dependencies. +This replaces *all* refresh parameters at once: schedule, dependencies, settings, and APPEND-ness. E.g. if the table had a `DEPENDS ON`, doing a `MODIFY REFRESH` without `DEPENDS ON` will remove the dependencies. ::: ### Other operations @@ -260,6 +274,10 @@ The status of all refreshable materialized views is available in table [`system. To manually stop, start, trigger, or cancel refreshes use [`SYSTEM STOP|START|REFRESH|CANCEL VIEW`](../system.md#refreshable-materialized-views). +:::note +Fun fact: the refresh query is allowed to read from the view that's being refreshed, seeing pre-refresh version of the data. This means you can implement Conway's game of life: https://pastila.nl/?00021a4b/d6156ff819c83d490ad2dcec05676865#O0LGWTO7maUQIA4AcGUtlA== +::: + ## Window View [Experimental] :::info diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index e6d3439d2b9..7b8d2339516 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -389,7 +389,7 @@ SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_ After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. The following modifiers are supported: - If a `STRICT` modifier was specified then the query waits for the replication queue to become empty. The `STRICT` version may never succeed if new entries constantly appear in the replication queue. - - If a `LIGHTWEIGHT` modifier was specified then the query waits only for `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` and `DROP_PART` entries to be processed. + - If a `LIGHTWEIGHT` modifier was specified then the query waits only for `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` and `DROP_PART` entries to be processed. Additionally, the LIGHTWEIGHT modifier supports an optional FROM 'srcReplicas' clause, where 'srcReplicas' is a comma-separated list of source replica names. This extension allows for more targeted synchronization by focusing only on replication tasks originating from the specified source replicas. - If a `PULL` modifier was specified then the query pulls new replication queue entries from ZooKeeper, but does not wait for anything to be processed. @@ -515,6 +515,10 @@ Trigger an immediate out-of-schedule refresh of a given view. SYSTEM REFRESH VIEW [db.]name ``` +### REFRESH VIEW + +Wait for the currently running refresh to complete. If the refresh fails, throws an exception. If no refresh is running, completes immediately, throwing an exception if previous refresh failed. + ### STOP VIEW, STOP VIEWS Disable periodic refreshing of the given view or all refreshable views. If a refresh is in progress, cancel it too. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b1b8e2367a4..38fed9f3198 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -604,6 +604,7 @@ M(723, PARQUET_EXCEPTION) \ M(724, TOO_MANY_TABLES) \ M(725, TOO_MANY_DATABASES) \ + M(726, REFRESH_FAILED) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d3526941b33..111817729e2 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -664,6 +664,9 @@ BlockIO InterpreterSystemQuery::execute() case Type::REFRESH_VIEW: getRefreshTask()->run(); break; + case Type::WAIT_VIEW: + getRefreshTask()->wait(); + break; case Type::CANCEL_VIEW: getRefreshTask()->cancel(); break; @@ -1411,6 +1414,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() break; } case Type::REFRESH_VIEW: + case Type::WAIT_VIEW: case Type::START_VIEW: case Type::START_VIEWS: case Type::STOP_VIEW: diff --git a/src/Parsers/ASTRefreshStrategy.cpp b/src/Parsers/ASTRefreshStrategy.cpp index 2e0c6ee4638..d10c1b4e7f5 100644 --- a/src/Parsers/ASTRefreshStrategy.cpp +++ b/src/Parsers/ASTRefreshStrategy.cpp @@ -20,7 +20,6 @@ ASTPtr ASTRefreshStrategy::clone() const res->set(res->settings, settings->clone()); if (dependencies) res->set(res->dependencies, dependencies->clone()); - res->schedule_kind = schedule_kind; return res; } @@ -66,6 +65,8 @@ void ASTRefreshStrategy::formatImpl( f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " SETTINGS " << (f_settings.hilite ? hilite_none : ""); settings->formatImpl(f_settings, state, frame); } + if (append) + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " APPEND" << (f_settings.hilite ? hilite_none : ""); } } diff --git a/src/Parsers/ASTRefreshStrategy.h b/src/Parsers/ASTRefreshStrategy.h index ca248b76b40..bb5ac97c054 100644 --- a/src/Parsers/ASTRefreshStrategy.h +++ b/src/Parsers/ASTRefreshStrategy.h @@ -24,6 +24,7 @@ public: ASTTimeInterval * offset = nullptr; ASTTimeInterval * spread = nullptr; RefreshScheduleKind schedule_kind{RefreshScheduleKind::UNKNOWN}; + bool append = false; String getID(char) const override { return "Refresh strategy definition"; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 167e724dcee..59de90b1d8e 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -95,6 +95,7 @@ public: START_CLEANUP, RESET_COVERAGE, REFRESH_VIEW, + WAIT_VIEW, START_VIEW, START_VIEWS, STOP_VIEW, diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index e7912293d85..8d19312996a 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -95,7 +96,15 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!ParserSetQuery{true}.parse(pos, settings, expected)) return false; refresh->set(refresh->settings, settings); + + /// Validate. + RefreshSettings parsed_settings; + parsed_settings.applyChanges(refresh->settings->changes); } + + if (ParserKeyword{"APPEND"}.ignore(pos, expected)) + refresh->append = true; + node = refresh; return true; } diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 0545c3e5568..4b89b3817f0 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -421,6 +421,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & break; case Type::REFRESH_VIEW: + case Type::WAIT_VIEW: case Type::START_VIEW: case Type::STOP_VIEW: case Type::CANCEL_VIEW: diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index eff445023a6..c5ec227bd92 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -11,7 +11,7 @@ namespace DB using DatabaseAndTableNameSet = std::unordered_set; -enum class RefreshState : RefreshTaskStateUnderlying +enum class RefreshState { Disabled = 0, Scheduled, @@ -19,11 +19,11 @@ enum class RefreshState : RefreshTaskStateUnderlying Running, }; -enum class LastRefreshResult : RefreshTaskStateUnderlying +enum class LastRefreshResult { Unknown = 0, Cancelled, - Exception, + Error, Finished }; @@ -37,7 +37,8 @@ struct RefreshInfo UInt64 last_attempt_duration_ms = 0; UInt32 next_refresh_time = 0; UInt64 refresh_count = 0; - String exception_message; // if last_refresh_result is Exception + UInt64 retry = 0; + String exception_message; // if last_refresh_result is Error std::vector remaining_dependencies; ProgressValues progress; }; diff --git a/src/Storages/MaterializedView/RefreshSettings.h b/src/Storages/MaterializedView/RefreshSettings.h index 814c7e52b32..23676538788 100644 --- a/src/Storages/MaterializedView/RefreshSettings.h +++ b/src/Storages/MaterializedView/RefreshSettings.h @@ -6,8 +6,10 @@ namespace DB { #define LIST_OF_REFRESH_SETTINGS(M, ALIAS) \ - /// TODO: Add settings - /// M(UInt64, name, 42, "...", 0) + M(Int64, refresh_retries, 0, "How many times to retry refresh query if it fails. If all attempts fail, wait for the next refresh time according to schedule. 0 to disable retries. -1 for infinite retries.", 0) \ + M(UInt64, refresh_retry_initial_backoff_ms, 100, "Delay before the first retry if refresh query fails (if refresh_retries setting is not zero). Each subsequent retry doubles the delay, up to refresh_retry_max_backoff_ms.", 0) \ + M(UInt64, refresh_retry_max_backoff_ms, 60'000, "Limit on the exponential growth of delay between refresh attempts, if they keep failing and refresh_retries is positive.", 0) \ + DECLARE_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index bc8cb0ce69a..83930ed7be9 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -23,41 +23,42 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int QUERY_WAS_CANCELLED; + extern const int REFRESH_FAILED; } RefreshTask::RefreshTask( - const ASTRefreshStrategy & strategy) + StorageMaterializedView * view_, const DB::ASTRefreshStrategy & strategy) : log(getLogger("RefreshTask")) + , view(view_) , refresh_schedule(strategy) -{} + , refresh_append(strategy.append) +{ + if (strategy.settings != nullptr) + refresh_settings.applyChanges(strategy.settings->changes); +} RefreshTaskHolder RefreshTask::create( - const StorageMaterializedView & view, + StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) { - auto task = std::make_shared(strategy); + auto task = std::make_shared(view, strategy); - task->refresh_task = context->getSchedulePool().createTask("MaterializedViewRefresherTask", - [self = task->weak_from_this()] - { - if (auto t = self.lock()) - t->refreshTask(); - }); + task->refresh_task = context->getSchedulePool().createTask("RefreshTask", + [self = task.get()] { self->refreshTask(); }); std::vector deps; if (strategy.dependencies) for (auto && dependency : strategy.dependencies->children) deps.emplace_back(dependency->as()); - context->getRefreshSet().emplace(view.getStorageID(), deps, task); + context->getRefreshSet().emplace(view->getStorageID(), deps, task); return task; } -void RefreshTask::initializeAndStart(std::shared_ptr view) +void RefreshTask::initializeAndStart() { - view_to_refresh = view; if (view->getContext()->getSettingsRef().stop_refreshable_materialized_views_on_startup) stop_requested = true; populateDependencies(); @@ -103,7 +104,11 @@ void RefreshTask::alterRefreshParams(const DB::ASTRefreshStrategy & new_strategy if (arriveDependency(id) && !std::exchange(refresh_immediately, true)) refresh_task->schedule(); - /// TODO: Update settings once we have them. + refresh_settings = {}; + if (new_strategy.settings != nullptr) + refresh_settings.applyChanges(new_strategy.settings->changes); + + refresh_append = new_strategy.append; } RefreshInfo RefreshTask::getInfo() const @@ -112,7 +117,7 @@ RefreshInfo RefreshTask::getInfo() const auto res = info; res.view_id = set_handle.getID(); res.remaining_dependencies.assign(remaining_dependencies.begin(), remaining_dependencies.end()); - if (res.last_refresh_result != LastRefreshResult::Exception) + if (res.last_refresh_result != LastRefreshResult::Error) res.exception_message.clear(); res.progress = progress.getValues(); return res; @@ -140,6 +145,8 @@ void RefreshTask::run() std::lock_guard guard(mutex); if (std::exchange(refresh_immediately, true)) return; + next_refresh_prescribed = std::chrono::floor(currentTime()); + next_refresh_actual = currentTime(); refresh_task->schedule(); } @@ -150,6 +157,14 @@ void RefreshTask::cancel() refresh_task->schedule(); } +void RefreshTask::wait() +{ + std::unique_lock lock(mutex); + refresh_cv.wait(lock, [&] { return info.state != RefreshState::Running; }); + if (info.last_refresh_result == LastRefreshResult::Error) + throw Exception(ErrorCodes::REFRESH_FAILED, "Refresh failed: {}", info.exception_message); +} + void RefreshTask::shutdown() { { @@ -167,6 +182,8 @@ void RefreshTask::shutdown() /// (Also, RefreshSet holds a shared_ptr to us.) std::lock_guard guard(mutex); set_handle.reset(); + + view = nullptr; } void RefreshTask::notify(const StorageID & parent_id, std::chrono::sys_seconds parent_next_prescribed_time) @@ -233,6 +250,7 @@ void RefreshTask::refreshTask() chassert(lock.owns_lock()); interrupt_execution.store(false); + refresh_cv.notify_all(); // we'll assign info.state before unlocking the mutex if (stop_requested) { @@ -244,7 +262,7 @@ void RefreshTask::refreshTask() if (!refresh_immediately) { auto now = currentTime(); - if (now >= next_refresh_with_spread) + if (now >= next_refresh_actual) { if (arriveTime()) refresh_immediately = true; @@ -257,7 +275,7 @@ void RefreshTask::refreshTask() else { size_t delay_ms = std::chrono::duration_cast( - next_refresh_with_spread - now).count(); + next_refresh_actual - now).count(); /// If we're in a test that fakes the clock, poll every 100ms. if (fake_clock.load(std::memory_order_relaxed) != INT64_MIN) @@ -271,19 +289,9 @@ void RefreshTask::refreshTask() /// Perform a refresh. + bool append = refresh_append; refresh_immediately = false; - - auto view = lockView(); - if (!view) - { - /// The view was dropped. This RefreshTask should be destroyed soon too. - /// (Maybe this is unreachable.) - info.state = RefreshState::Disabled; - break; - } - info.state = RefreshState::Running; - CurrentMetrics::Increment metric_inc(CurrentMetrics::RefreshingViews); lock.unlock(); @@ -294,7 +302,7 @@ void RefreshTask::refreshTask() try { - executeRefreshUnlocked(view); + executeRefreshUnlocked(append); refreshed = true; } catch (...) @@ -318,18 +326,16 @@ void RefreshTask::refreshTask() if (exception) { - info.last_refresh_result = LastRefreshResult::Exception; + info.last_refresh_result = LastRefreshResult::Error; info.exception_message = *exception; - - /// TODO: Do a few retries with exponential backoff. - advanceNextRefreshTime(now); + scheduleRetryOrSkipToNextRefresh(now); } else if (!refreshed) { info.last_refresh_result = LastRefreshResult::Cancelled; /// Make sure we don't just start another refresh immediately. - if (!stop_requested && now >= next_refresh_with_spread) + if (!stop_requested) advanceNextRefreshTime(now); } else @@ -362,17 +368,18 @@ void RefreshTask::refreshTask() } } -void RefreshTask::executeRefreshUnlocked(std::shared_ptr view) +void RefreshTask::executeRefreshUnlocked(bool append) { LOG_DEBUG(log, "Refreshing view {}", view->getStorageID().getFullTableName()); progress.reset(); - /// Create a table. - auto [refresh_context, refresh_query] = view->prepareRefresh(); - - StorageID stale_table = StorageID::createEmpty(); + ContextMutablePtr refresh_context; + std::optional table_to_drop; try { + /// Create a table. + auto refresh_query = view->prepareRefresh(append, refresh_context, table_to_drop); + /// Run the query. { CurrentThread::QueryScope query_scope(refresh_context); // create a thread group for the query @@ -424,37 +431,67 @@ void RefreshTask::executeRefreshUnlocked(std::shared_ptrexchangeTargetTable(refresh_query->table_id, refresh_context); + if (!append) + table_to_drop = view->exchangeTargetTable(refresh_query->table_id, refresh_context); } catch (...) { - try + if (table_to_drop.has_value()) { - InterpreterDropQuery::executeDropQuery( - ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, refresh_query->table_id, /*sync*/ false, /*ignore_sync_setting*/ true); - } - catch (...) - { - tryLogCurrentException(log, "Failed to drop temporary table after a failed refresh"); - /// Let's ignore this and keep going, at risk of accumulating many trash tables if this keeps happening. + try + { + InterpreterDropQuery::executeDropQuery( + ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ false, /*ignore_sync_setting*/ true); + } + catch (...) + { + tryLogCurrentException(log, "Failed to drop temporary table after a failed refresh"); + /// Let's ignore this and keep going, at risk of accumulating many trash tables if this keeps happening. + } } throw; } /// Drop the old table (outside the try-catch so we don't try to drop the other table if this fails). - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, stale_table, /*sync*/ true, /*ignore_sync_setting*/ true); + if (table_to_drop.has_value()) + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ true, /*ignore_sync_setting*/ true); } void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point now) { std::chrono::sys_seconds next = refresh_schedule.prescribeNext(next_refresh_prescribed, now); next_refresh_prescribed = next; - next_refresh_with_spread = refresh_schedule.addRandomSpread(next); + next_refresh_actual = refresh_schedule.addRandomSpread(next); - auto secs = std::chrono::floor(next_refresh_with_spread); + num_retries = 0; + info.retry = num_retries; + + auto secs = std::chrono::floor(next_refresh_actual); info.next_refresh_time = UInt32(secs.time_since_epoch().count()); } +void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) +{ + if (refresh_settings.refresh_retries >= 0 && num_retries >= refresh_settings.refresh_retries) + { + advanceNextRefreshTime(now); + return; + } + + num_retries += 1; + info.retry = num_retries; + + UInt64 delay_ms; + UInt64 multiplier = UInt64(1) << std::min(num_retries - 1, Int64(62)); + /// Overflow check: a*b <= c iff a <= c/b iff a <= floor(c/b). + if (refresh_settings.refresh_retry_initial_backoff_ms <= refresh_settings.refresh_retry_max_backoff_ms / multiplier) + delay_ms = refresh_settings.refresh_retry_initial_backoff_ms * multiplier; + else + delay_ms = refresh_settings.refresh_retry_max_backoff_ms; + + next_refresh_actual = now + std::chrono::milliseconds(delay_ms); +} + bool RefreshTask::arriveDependency(const StorageID & parent) { remaining_dependencies.erase(parent); @@ -495,11 +532,6 @@ void RefreshTask::interruptExecution() } } -std::shared_ptr RefreshTask::lockView() -{ - return std::static_pointer_cast(view_to_refresh.lock()); -} - std::chrono::system_clock::time_point RefreshTask::currentTime() const { Int64 fake = fake_clock.load(std::memory_order::relaxed); diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 1f050a97cd9..9281c0469a3 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -22,15 +22,15 @@ class RefreshTask : public std::enable_shared_from_this { public: /// Never call it manually, public for shared_ptr construction only - explicit RefreshTask(const ASTRefreshStrategy & strategy); + RefreshTask(StorageMaterializedView * view_, const ASTRefreshStrategy & strategy); /// The only proper way to construct task static RefreshTaskHolder create( - const StorageMaterializedView & view, + StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); - void initializeAndStart(std::shared_ptr view); + void initializeAndStart(); /// Call when renaming the materialized view. void rename(StorageID new_id); @@ -52,6 +52,11 @@ public: /// Cancel task execution void cancel(); + /// Waits for the currently running refresh attempt to complete. + /// If the refresh fails, throws an exception. + /// If no refresh is running, completes immediately, throwing an exception if previous refresh failed. + void wait(); + /// Permanently disable task scheduling and remove this table from RefreshSet. void shutdown(); @@ -66,7 +71,7 @@ public: private: LoggerPtr log = nullptr; - std::weak_ptr view_to_refresh; + StorageMaterializedView * view; /// Protects interrupt_execution and running_executor. /// Can be locked while holding `mutex`. @@ -83,7 +88,9 @@ private: mutable std::mutex mutex; RefreshSchedule refresh_schedule; - RefreshSettings refresh_settings; // TODO: populate, use, update on alter + RefreshSettings refresh_settings; + bool refresh_append; + RefreshSet::Handle set_handle; /// StorageIDs of our dependencies that we're waiting for. @@ -111,7 +118,8 @@ private: /// E.g. for REFRESH EVERY 1 DAY, yesterday's refresh of the dependency shouldn't trigger today's /// refresh of the dependent even if it happened today (e.g. it was slow or had random spread > 1 day). std::chrono::sys_seconds next_refresh_prescribed; - std::chrono::system_clock::time_point next_refresh_with_spread; + std::chrono::system_clock::time_point next_refresh_actual; + Int64 num_retries = 0; /// Calls refreshTask() from background thread. BackgroundSchedulePool::TaskHolder refresh_task; @@ -122,6 +130,7 @@ private: /// Just for observability. RefreshInfo info; Progress progress; + std::condition_variable refresh_cv; // notified when info.state changes /// The main loop of the refresh task. It examines the state, sees what needs to be /// done and does it. If there's nothing to do at the moment, returns; it's then scheduled again, @@ -133,11 +142,14 @@ private: /// Perform an actual refresh: create new table, run INSERT SELECT, exchange tables, drop old table. /// Mutex must be unlocked. Called only from refresh_task. - void executeRefreshUnlocked(std::shared_ptr view); + void executeRefreshUnlocked(bool append); /// Assigns next_refresh_* void advanceNextRefreshTime(std::chrono::system_clock::time_point now); + /// Either advances next_refresh_actual using exponential backoff or does advanceNextRefreshTime(). + void scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); + /// Returns true if all dependencies are fulfilled now. Refills remaining_dependencies in this case. bool arriveDependency(const StorageID & parent); bool arriveTime(); @@ -145,8 +157,6 @@ private: void interruptExecution(); - std::shared_ptr lockView(); - std::chrono::system_clock::time_point currentTime() const; }; diff --git a/src/Storages/MaterializedView/RefreshTask_fwd.h b/src/Storages/MaterializedView/RefreshTask_fwd.h index 1f366962eb6..9e389316c64 100644 --- a/src/Storages/MaterializedView/RefreshTask_fwd.h +++ b/src/Storages/MaterializedView/RefreshTask_fwd.h @@ -8,8 +8,6 @@ namespace DB class RefreshTask; -using RefreshTaskStateUnderlying = UInt8; using RefreshTaskHolder = std::shared_ptr; -using RefreshTaskObserver = std::weak_ptr; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 316f398b476..500312fd245 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -3,6 +3,8 @@ #include #include +#include +#include #include #include @@ -14,6 +16,7 @@ #include #include #include +#include #include #include @@ -141,6 +144,13 @@ StorageMaterializedView::StorageMaterializedView( if (point_to_itself_by_uuid || point_to_itself_by_name) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Materialized view {} cannot point to itself", table_id_.getFullTableName()); + if (query.refresh_strategy) + { + fixed_uuid = false; + refresher = RefreshTask::create(this, getContext(), *query.refresh_strategy); + refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; + } + if (!has_inner_table) { target_table_id = query.to_table_id; @@ -190,16 +200,6 @@ StorageMaterializedView::StorageMaterializedView( target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->getDatabase(), manual_create_query->getTable()}, getContext())->getStorageID(); } - - if (query.refresh_strategy) - { - fixed_uuid = false; - refresher = RefreshTask::create( - *this, - getContext(), - *query.refresh_strategy); - refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; - } } QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( @@ -370,40 +370,57 @@ bool StorageMaterializedView::optimize( return storage_ptr->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); } -std::tuple> StorageMaterializedView::prepareRefresh() const +std::shared_ptr StorageMaterializedView::prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const { auto refresh_context = getInMemoryMetadataPtr()->getSQLSecurityOverriddenContext(getContext()); + out_context = refresh_context; /// Generate a random query id. refresh_context->setCurrentQueryId(""); CurrentThread::QueryScope query_scope(refresh_context); auto inner_table_id = getTargetTableId(); - auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); - auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); + StorageID target_table = inner_table_id; - auto create_table_query = db->getCreateTableQuery(inner_table_id.table_name, getContext()); - auto & create_query = create_table_query->as(); - create_query.setTable(new_table_name); - create_query.setDatabase(db->getDatabaseName()); - create_query.create_or_replace = true; - create_query.replace_table = true; - create_query.uuid = UUIDHelpers::Nil; + if (!append) + { + auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); - InterpreterCreateQuery create_interpreter(create_table_query, refresh_context); - create_interpreter.setInternal(true); - create_interpreter.execute(); + auto create_table_query = db->getCreateTableQuery(inner_table_id.table_name, getContext()); + auto & create_query = create_table_query->as(); + create_query.setTable(new_table_name); + create_query.setDatabase(db->getDatabaseName()); + create_query.create_or_replace = true; + create_query.replace_table = true; + create_query.uuid = UUIDHelpers::Nil; - StorageID fresh_table = DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); + InterpreterCreateQuery create_interpreter(create_table_query, refresh_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); + + target_table = DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); + out_temp_table_id = target_table; + } auto insert_query = std::make_shared(); insert_query->select = getInMemoryMetadataPtr()->getSelectQuery().select_query; - insert_query->setTable(fresh_table.table_name); - insert_query->setDatabase(fresh_table.database_name); - insert_query->table_id = fresh_table; + insert_query->setTable(target_table.table_name); + insert_query->setDatabase(target_table.database_name); + insert_query->table_id = target_table; - return {refresh_context, insert_query}; + Block header; + if (refresh_context->getSettingsRef().allow_experimental_analyzer) + header = InterpreterSelectQueryAnalyzer::getSampleBlock(insert_query->select, refresh_context); + else + header = InterpreterSelectWithUnionQuery(insert_query->select, refresh_context, SelectQueryOptions()).getSampleBlock(); + + auto columns = std::make_shared(','); + for (const String & name : header.getNames()) + columns->children.push_back(std::make_shared(name)); + insert_query->columns = std::move(columns); + + return insert_query; } StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context) @@ -569,7 +586,7 @@ void StorageMaterializedView::startup() if (refresher) { - refresher->initializeAndStart(std::static_pointer_cast(shared_from_this())); + refresher->initializeAndStart(); if (refresh_on_start) refresher->run(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 5ecd2ec3819..12c0dc651e9 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -121,7 +121,9 @@ private: /// Prepare to refresh a refreshable materialized view: create query context, create temporary /// table, form the insert-select query. - std::tuple> prepareRefresh() const; + /// The output arguments may be assigned before throwing an exception, in which case the caller + /// must drop the temp table before rethrowing. + std::shared_ptr prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const; StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); void setTargetTableId(StorageID id); diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 30539ed6b6a..95c89bd0214 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -32,8 +32,9 @@ ColumnsDescription StorageSystemViewRefreshes::getColumnsDescription() "If status = 'WaitingForDependencies', a refresh is ready to start as soon as these dependencies are fulfilled." }, {"exception", std::make_shared(), - "if last_refresh_result = 'Exception', i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace." + "if last_refresh_result = 'Error', i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace." }, + {"retry", std::make_shared(), "How many failed attempts there were so far, for the current refresh."}, {"refresh_count", std::make_shared(), "Number of successful refreshes since last server restart or table creation."}, {"progress", std::make_shared(), "Progress of the current refresh, between 0 and 1."}, {"elapsed", std::make_shared(), "The amount of nanoseconds the current refresh took."}, @@ -85,6 +86,7 @@ void StorageSystemViewRefreshes::fillData( res_columns[i++]->insert(Array(deps)); res_columns[i++]->insert(refresh.exception_message); + res_columns[i++]->insert(refresh.retry); res_columns[i++]->insert(refresh.refresh_count); res_columns[i++]->insert(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read); res_columns[i++]->insert(refresh.progress.elapsed_ns / 1e9); diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index 2eb41590af1..8d4878f4e3f 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -3,21 +3,25 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n) <2: refreshed> 3 1 1 <3: time difference at least> 1000 <4: next refresh in> 2 +<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 <4.5: altered> Scheduled Finished 2052-01-01 00:00:00 CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 <6: refreshed> 2 <7: refreshed> Scheduled Finished 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a +<7.5: created dependent> 2052-11-11 11:11:11 <8: refreshed> 20 <9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 <9: refreshed> b Scheduled Finished 2054-01-01 00:00:00 -<10: waiting> a Scheduled [] 2054-01-01 00:00:00 -<10: waiting> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 +<9.2: dropping> 0 2 +<9.4: dropped> 0 2 +<10: creating> a Scheduled [] 2054-01-01 00:00:00 +<10: creating> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 <11: chain-refreshed a> 4 <12: chain-refreshed b> 40 -<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 -<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 +<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 1 +<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 1 <14: waiting for next cycle> a Scheduled [] 2058-01-01 00:00:00 <14: waiting for next cycle> b WaitingForDependencies ['default.a'] 2060-01-01 00:00:00 <15: chain-refreshed a> 6 @@ -34,11 +38,24 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nE <24: rename during refresh> 1 <25: rename during refresh> f Running <27: cancelled> f Scheduled -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 +<28: drop during refresh> 0 0 +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 <29: randomize> 1 1 CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 <31: to existing table> 10 <31: to existing table> 20 -<32: empty> i Scheduled Unknown -<32: empty> j Scheduled Finished +<31.5: will retry> Error 1 +<31.6: did retry> 10 +<32: empty> i Scheduled Unknown 0 +<32: empty> j Scheduled Finished 0 +<34: append> 10 +<35: append> 10 +<35: append> 20 +<35: append> 30 +<36: not append> 20 +<36: not append> 30 +<37: append chain> 100 +<38: append chain> 100 +<38: append chain> 100 +<38: append chain> 200 diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 89942e25b67..ad6fa3043a1 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -21,9 +21,9 @@ $CLICKHOUSE_CLIENT -nq " refresh after 2 second engine Memory empty - as select number as x from numbers(2) union all select rand64() as x" -$CLICKHOUSE_CLIENT -nq "select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes"; -$CLICKHOUSE_CLIENT -nq "show create a" + as select number as x from numbers(2) union all select rand64() as x; + select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes; + show create a;" # Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] do @@ -56,16 +56,15 @@ $CLICKHOUSE_CLIENT -nq " # Create a source table from which views will read. $CLICKHOUSE_CLIENT -nq " - create table src (x Int8) engine Memory as select 1" + create table src (x Int8) engine Memory as select 1;" # Switch to fake clock, change refresh schedule, change query. $CLICKHOUSE_CLIENT -nq " - system test view a set fake time '2050-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " + system test view a set fake time '2050-01-01 00:00:01'; + system wait view a; + system refresh view a; + system wait view a; + select '<4.1: fake clock>', status, last_refresh_time, next_refresh_time from refreshes; alter table a modify refresh every 2 year; alter table a modify query select x*2 as x from src; select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; @@ -87,11 +86,9 @@ $CLICKHOUSE_CLIENT -nq " create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; show create b; system test view b set fake time '2052-11-11 11:11:11'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2052-11-11 11:11:11' ] -do - sleep 0.1 -done + system refresh view b; + system wait view b; + select '<7.5: created dependent>', last_refresh_time from refreshes where view = 'b';" # Next refresh shouldn't start until the dependency refreshes. $CLICKHOUSE_CLIENT -nq " select '<8: refreshed>', * from b; @@ -101,11 +98,20 @@ while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshe do sleep 0.1 done -# Update source table (by dropping and re-creating it - to test that tables are looked up by name -# rather than uuid), kick off refresh of the dependency. + +# Drop the source table, check that refresh fails and doesn't leave a temp table behind. $CLICKHOUSE_CLIENT -nq " - select '<10: waiting>', view, status, remaining_dependencies, next_refresh_time from refreshes; + select '<9.2: dropping>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase(); drop table src; + system refresh view a;" +$CLICKHOUSE_CLIENT -nq "system wait view a;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" +$CLICKHOUSE_CLIENT -nq " + select '<9.4: dropped>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase();" + +# Create the source table again, check that refresh succeeds (in particular that tables are looked +# up by name rather than uuid). +$CLICKHOUSE_CLIENT -nq " + select '<10: creating>', view, status, remaining_dependencies, next_refresh_time from refreshes; create table src (x Int16) engine Memory as select 2; system test view a set fake time '2054-01-01 00:00:01';" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] @@ -116,7 +122,7 @@ done $CLICKHOUSE_CLIENT -nq " select '<11: chain-refreshed a>', * from a; select '<12: chain-refreshed b>', * from b; - select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception from refreshes;" + select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception == '' from refreshes;" # Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to # catch up to the same cycle. @@ -172,7 +178,7 @@ $CLICKHOUSE_CLIENT -nq " drop table b; create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; drop table src;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Exception' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Error' ] do sleep 0.1 done @@ -246,7 +252,8 @@ done # Drop. $CLICKHOUSE_CLIENT -nq " drop table f; - select '<28: drop during refresh>', view, status from refreshes;" + select '<28: drop during refresh>', view, status from refreshes; + select '<28: drop during refresh>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase()" # Try OFFSET and RANDOMIZE FOR. $CLICKHOUSE_CLIENT -nq " @@ -283,21 +290,74 @@ done $CLICKHOUSE_CLIENT -nq " select '<31: to existing table>', * from dest; drop table dest; - drop table src; drop table h;" +# Retries. +$CLICKHOUSE_CLIENT -nq " + create materialized view h2 refresh after 1 year settings refresh_retries = 10 (x Int64) engine Memory as select x*10 + throwIf(x % 2 == 0) as x from src;" +$CLICKHOUSE_CLIENT -nq "system wait view h2;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" +$CLICKHOUSE_CLIENT -nq " + select '<31.5: will retry>', last_refresh_result, retry > 0 from refreshes; + truncate table src; + insert into src values (1);" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result, retry from refreshes -- $LINENO" | xargs`" != 'Finished 0' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<31.6: did retry>', x from h2; + drop table h2" + # EMPTY $CLICKHOUSE_CLIENT -nq " create materialized view i refresh after 1 year engine Memory empty as select number as x from numbers(2); - create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2)" + create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2);" while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] do sleep 0.1 done $CLICKHOUSE_CLIENT -nq " - select '<32: empty>', view, status, last_refresh_result from refreshes order by view; + select '<32: empty>', view, status, last_refresh_result, retry from refreshes order by view; drop table i; - drop table j" + drop table j;" + +# APPEND +$CLICKHOUSE_CLIENT -nq " + create materialized view k refresh every 10 year append (x Int64) engine Memory empty as select x*10 as x from src; + select '<33: append>', * from k; + system refresh view k; + system wait view k; + select '<34: append>', * from k; + truncate table src; + insert into src values (2), (3); + system refresh view k; + system wait view k; + select '<35: append>', * from k order by x;" +# ALTER to non-APPEND +$CLICKHOUSE_CLIENT -nq " + alter table k modify refresh every 10 year; + system refresh view k; + system wait view k; + select '<36: not append>', * from k order by x; + drop table k; + truncate table src;" + +# APPEND + TO + regular materialized view reading from it. +$CLICKHOUSE_CLIENT -nq " + create table mid (x Int64) engine MergeTree order by x; + create materialized view l refresh every 10 year append to mid empty as select x*10 as x from src; + create materialized view m (x Int64) engine Memory as select x*10 as x from mid; + insert into src values (1); + system refresh view l; + system wait view l; + select '<37: append chain>', * from m; + insert into src values (2); + system refresh view l; + system wait view l; + select '<38: append chain>', * from m order by x; + drop table l; + drop table m; + drop table mid;" $CLICKHOUSE_CLIENT -nq " drop table refreshes;" From 361aa4ffadcad1f68c9157bd44a835ac828de1d4 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 26 Jan 2024 23:03:31 +0000 Subject: [PATCH 0170/1722] Fix build --- src/Parsers/ParserRefreshStrategy.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index 8d19312996a..de7eea93614 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -96,10 +95,6 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!ParserSetQuery{true}.parse(pos, settings, expected)) return false; refresh->set(refresh->settings, settings); - - /// Validate. - RefreshSettings parsed_settings; - parsed_settings.applyChanges(refresh->settings->changes); } if (ParserKeyword{"APPEND"}.ignore(pos, expected)) From dcf783c0edad02c45ede10a981465c67d6991c45 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 1 Feb 2024 22:42:30 +0000 Subject: [PATCH 0171/1722] Check column names when creating table, fix a thing, move code around a little --- src/Interpreters/InterpreterCreateQuery.cpp | 148 ++++++++++++------ src/Interpreters/InterpreterCreateQuery.h | 2 + src/Parsers/ASTCreateQuery.cpp | 2 + src/Parsers/ASTRenameQuery.h | 13 ++ src/Storages/IStorage.h | 2 +- src/Storages/MaterializedView/RefreshTask.cpp | 25 ++- src/Storages/MaterializedView/RefreshTask.h | 24 ++- src/Storages/StorageMaterializedView.cpp | 76 +++++---- src/Storages/StorageMaterializedView.h | 15 +- .../02932_refreshable_materialized_views.sh | 9 +- 10 files changed, 208 insertions(+), 108 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7272e10b801..ee57c376b31 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -120,6 +120,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int TOO_MANY_TABLES; extern const int TOO_MANY_DATABASES; + extern const int THERE_IS_NO_COLUMN; } namespace fs = std::filesystem; @@ -852,6 +853,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); + properties.columns_inferred_from_select_query = true; } else if (create.as_table_function) { @@ -939,6 +941,99 @@ void validateVirtualColumns(const IStorage & storage) } } +void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTCreateQuery & create, const TableProperties & properties, const DatabasePtr & database) +{ + /// This is not strict validation, just catches common errors that would make the view not work. + /// It's possible to circumvent these checks by ALTERing the view or target table after creation. + + String table_engine; + if (create.storage && create.storage->engine) + table_engine = create.storage->engine->name; + + NamesAndTypesList all_output_columns; + bool check_columns = false; + if (create.to_table_id) + { + if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( + {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, + getContext() + )) + { + all_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock().getNamesAndTypesList(); + check_columns = true; + table_engine = to_table->getName(); + } + } + else if (!properties.columns_inferred_from_select_query) + { + all_output_columns = properties.columns.getInsertable(); + check_columns = true; + } + + if (create.refresh_strategy && !create.refresh_strategy->append) + { + if (database && database->getEngineName() != "Atomic") + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Refreshable materialized views (except with APPEND) only support Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName()); + } + + if (check_columns) + { + Block input_block; + + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } + + std::unordered_map output_types; + for (const NameAndTypePair & nt : all_output_columns) + output_types[nt.name] = nt.type; + + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName output_columns; + for (const auto & input_column : input_block) + { + auto it = output_types.find(input_column.name); + if (it != output_types.end()) + { + input_columns.push_back(input_column.cloneEmpty()); + output_columns.push_back(ColumnWithTypeAndName(it->second->createColumn(), it->second, input_column.name)); + } + else if (create.refresh_strategy) + { + /// Unrecognized columns produced by SELECT query are allowed for regular materialized + /// views, but not for refreshable ones. This is in part because it was easier to + /// implement, in part because refreshable views have less concern about ALTERing target + /// tables. + /// + /// The motivating scenario for allowing this in regular MV is ALTERing the table+query. + /// Suppose the user removes a column from target table, then a minute later + /// correspondingly updates the view's query to not produce that column. + /// If MV didn't allow unrecognized columns then during that minute all INSERTs into the + /// source table would fail - unacceptable. + /// For refreshable views, during that minute refreshes will fail - acceptable. + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "SELECT query outputs column with name '{}', which is not found in the target table. Use 'AS' to assign alias that matches a column name.", input_column.name); + } + } + + if (input_columns.empty()) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "None of the columns produced by the SELECT query are present in the target table. Use 'AS' to assign aliases that match column names."); + + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); + } +} + namespace { void checkTemporaryTableEngineName(const String & name) @@ -1059,13 +1154,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data throw Exception(ErrorCodes::LOGICAL_ERROR, "Table UUID is not specified in DDL log"); } - if (create.refresh_strategy && database->getEngineName() != "Atomic") - throw Exception(ErrorCodes::INCORRECT_QUERY, - "Refreshable materialized view requires Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName()); - /// TODO: Support Replicated databases, only with Shared/ReplicatedMergeTree. - /// Figure out how to make the refreshed data appear all at once on other - /// replicas; maybe a replicated SYSTEM SYNC REPLICA query before the rename? - if (database->getUUID() != UUIDHelpers::Nil) { if (create.attach && !from_path && create.uuid == UUIDHelpers::Nil) @@ -1257,54 +1345,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); - /// Check type compatible for materialized dest table and select columns - if (create.select && create.is_materialized_view && create.to_table_id && mode <= LoadingStrictnessLevel::CREATE) - { - if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( - {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, - getContext() - )) - { - Block input_block; - - if (getContext()->getSettingsRef().allow_experimental_analyzer) - { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); - } - else - { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); - } - - Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - - ColumnsWithTypeAndName input_columns; - ColumnsWithTypeAndName output_columns; - for (const auto & input_column : input_block) - { - if (const auto * output_column = output_block.findByName(input_column.name)) - { - input_columns.push_back(input_column.cloneEmpty()); - output_columns.push_back(output_column->cloneEmpty()); - } - } - - ActionsDAG::makeConvertingActions( - input_columns, - output_columns, - ActionsDAG::MatchColumnsMode::Position - ); - } - } - DatabasePtr database; bool need_add_to_database = !create.temporary; // In case of an ON CLUSTER query, the database may not be present on the initiator node if (need_add_to_database) database = DatabaseCatalog::instance().tryGetDatabase(database_name); + /// Check type compatible for materialized dest table and select columns + if (create.select && create.is_materialized_view && mode <= LoadingStrictnessLevel::CREATE) + validateMaterializedViewColumnsAndEngine(create, properties, database); + if (database && database->getEngineName() == "Replicated" && create.select) { bool is_storage_replicated = false; diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 70ef29e6b07..c51c0e757df 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -91,6 +91,7 @@ private: IndicesDescription indices; ConstraintsDescription constraints; ProjectionsDescription projections; + bool columns_inferred_from_select_query = false; }; BlockIO createDatabase(ASTCreateQuery & create); @@ -99,6 +100,7 @@ private: /// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way. TableProperties getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create, LoadingStrictnessLevel mode) const; void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const; + void validateMaterializedViewColumnsAndEngine(const ASTCreateQuery & create, const TableProperties & properties, const DatabasePtr & database); void setEngine(ASTCreateQuery & create) const; AccessRightsElements getRequiredAccess() const; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 3e5c6a9d86e..96a29348326 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -254,6 +254,8 @@ ASTPtr ASTCreateQuery::clone() const res->set(res->dictionary, dictionary->clone()); } + if (refresh_strategy) + res->set(res->refresh_strategy, refresh_strategy->clone()); if (as_table_function) res->set(res->as_table_function, as_table_function->clone()); if (comment) diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index d51c382f374..39fc4f787ec 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -141,6 +141,19 @@ public: QueryKind getQueryKind() const override { return QueryKind::Rename; } + void addElement(const String & from_db, const String & from_table, const String & to_db, const String & to_table) + { + auto identifier = [&](const String & name) -> ASTPtr + { + if (name.empty()) + return nullptr; + ASTPtr ast = std::make_shared(name); + children.push_back(ast); + return ast; + }; + elements.push_back(Element {.from = Table {.database = identifier(from_db), .table = identifier(from_table)}, .to = Table {.database = identifier(to_db), .table = identifier(to_table)}}); + } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 98afd844046..7d854f35029 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -103,7 +103,7 @@ public: IStorage(const IStorage &) = delete; IStorage & operator=(const IStorage &) = delete; - /// The main name of the table type (for example, StorageMergeTree). + /// The main name of the table type (e.g. Memory, MergeTree, CollapsingMergeTree). virtual std::string getName() const = 0; /// The name of the table. diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 83930ed7be9..e593023caee 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -37,7 +37,7 @@ RefreshTask::RefreshTask( refresh_settings.applyChanges(strategy.settings->changes); } -RefreshTaskHolder RefreshTask::create( +OwnedRefreshTask RefreshTask::create( StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) @@ -54,7 +54,7 @@ RefreshTaskHolder RefreshTask::create( context->getRefreshSet().emplace(view->getStorageID(), deps, task); - return task; + return OwnedRefreshTask(task); } void RefreshTask::initializeAndStart() @@ -169,6 +169,10 @@ void RefreshTask::shutdown() { { std::lock_guard guard(mutex); + + if (view == nullptr) + return; // already shut down + stop_requested = true; interruptExecution(); } @@ -373,7 +377,7 @@ void RefreshTask::executeRefreshUnlocked(bool append) LOG_DEBUG(log, "Refreshing view {}", view->getStorageID().getFullTableName()); progress.reset(); - ContextMutablePtr refresh_context; + ContextMutablePtr refresh_context = view->createRefreshContext(); std::optional table_to_drop; try { @@ -437,24 +441,13 @@ void RefreshTask::executeRefreshUnlocked(bool append) catch (...) { if (table_to_drop.has_value()) - { - try - { - InterpreterDropQuery::executeDropQuery( - ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ false, /*ignore_sync_setting*/ true); - } - catch (...) - { - tryLogCurrentException(log, "Failed to drop temporary table after a failed refresh"); - /// Let's ignore this and keep going, at risk of accumulating many trash tables if this keeps happening. - } - } + view->dropTempTable(table_to_drop.value(), refresh_context); throw; } /// Drop the old table (outside the try-catch so we don't try to drop the other table if this fails). if (table_to_drop.has_value()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ true, /*ignore_sync_setting*/ true); + view->dropTempTable(table_to_drop.value(), refresh_context); } void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point now) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 9281c0469a3..411eb6fa7b4 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -17,6 +17,7 @@ class PipelineExecutor; class StorageMaterializedView; class ASTRefreshStrategy; +struct OwnedRefreshTask; class RefreshTask : public std::enable_shared_from_this { @@ -25,12 +26,12 @@ public: RefreshTask(StorageMaterializedView * view_, const ASTRefreshStrategy & strategy); /// The only proper way to construct task - static RefreshTaskHolder create( + static OwnedRefreshTask create( StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); - void initializeAndStart(); + void initializeAndStart(); // called at most once /// Call when renaming the materialized view. void rename(StorageID new_id); @@ -58,6 +59,8 @@ public: void wait(); /// Permanently disable task scheduling and remove this table from RefreshSet. + /// Ok to call multiple times, but not in parallel. + /// Ok to call even if initializeAndStart() wasn't called or failed. void shutdown(); /// Notify dependent task @@ -160,4 +163,21 @@ private: std::chrono::system_clock::time_point currentTime() const; }; +/// Wrapper around shared_ptr, calls shutdown() in destructor. +struct OwnedRefreshTask +{ + RefreshTaskHolder ptr; + + OwnedRefreshTask() = default; + OwnedRefreshTask(RefreshTaskHolder p) : ptr(std::move(p)) {} + OwnedRefreshTask(OwnedRefreshTask &&) = default; + OwnedRefreshTask & operator=(OwnedRefreshTask &&) = default; + + ~OwnedRefreshTask() { if (ptr) ptr->shutdown(); } + + RefreshTask* operator->() const { return ptr.get(); } + RefreshTask& operator*() const { return *ptr; } + operator bool() const { return ptr != nullptr; } +}; + } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 500312fd245..9f793e06c09 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -370,21 +370,28 @@ bool StorageMaterializedView::optimize( return storage_ptr->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); } -std::shared_ptr StorageMaterializedView::prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const +ContextMutablePtr StorageMaterializedView::createRefreshContext() const { auto refresh_context = getInMemoryMetadataPtr()->getSQLSecurityOverriddenContext(getContext()); - out_context = refresh_context; + refresh_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); /// Generate a random query id. refresh_context->setCurrentQueryId(""); + /// TODO: Set view's definer as the current user in refresh_context, so that the correct user's + /// quotas and permissions apply for this query. + return refresh_context; +} - CurrentThread::QueryScope query_scope(refresh_context); - +std::shared_ptr StorageMaterializedView::prepareRefresh(bool append, ContextMutablePtr refresh_context, std::optional & out_temp_table_id) const +{ auto inner_table_id = getTargetTableId(); - auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); StorageID target_table = inner_table_id; if (!append) { + CurrentThread::QueryScope query_scope(refresh_context); + + auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); + String db_name = db->getDatabaseName(); auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); auto create_table_query = db->getCreateTableQuery(inner_table_id.table_name, getContext()); @@ -399,7 +406,7 @@ std::shared_ptr StorageMaterializedView::prepareRefresh(bool app create_interpreter.setInternal(true); create_interpreter.execute(); - target_table = DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); + target_table = DatabaseCatalog::instance().getTable({db_name, new_table_name}, getContext())->getStorageID(); out_temp_table_id = target_table; } @@ -425,6 +432,9 @@ std::shared_ptr StorageMaterializedView::prepareRefresh(bool app StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context) { + /// Known problem: if the target table was ALTERed during refresh, this will effectively revert + /// the ALTER. + auto stale_table_id = getTargetTableId(); auto db = DatabaseCatalog::instance().getDatabase(stale_table_id.database_name); @@ -432,13 +442,34 @@ StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, Co CurrentThread::QueryScope query_scope(refresh_context); - target_db->renameTable( - refresh_context, fresh_table.table_name, *db, stale_table_id.table_name, /*exchange=*/true, /*dictionary=*/false); + auto rename_query = std::make_shared(); + rename_query->exchange = true; + rename_query->addElement(fresh_table.database_name, fresh_table.table_name, stale_table_id.database_name, stale_table_id.table_name); - std::swap(stale_table_id.database_name, fresh_table.database_name); - std::swap(stale_table_id.table_name, fresh_table.table_name); - setTargetTableId(std::move(fresh_table)); - return stale_table_id; + InterpreterRenameQuery(rename_query, refresh_context).execute(); + + return fresh_table; +} + +void StorageMaterializedView::dropTempTable(StorageID table_id, ContextMutablePtr refresh_context) +{ + CurrentThread::QueryScope query_scope(refresh_context); + + try + { + auto drop_query = std::make_shared(); + drop_query->setDatabase(table_id.database_name); + drop_query->setTable(table_id.table_name); + drop_query->kind = ASTDropQuery::Kind::Drop; + drop_query->if_exists = true; + drop_query->sync = false; + + InterpreterDropQuery(drop_query, refresh_context).execute(); + } + catch (...) + { + tryLogCurrentException(&Poco::Logger::get("StorageMaterializedView"), "Failed to drop temporary table after refresh"); + } } void StorageMaterializedView::alter( @@ -543,20 +574,7 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) ASTRenameQuery::Elements rename_elements; assert(inner_table_id.database_name == old_table_id.database_name); - ASTRenameQuery::Element elem - { - ASTRenameQuery::Table - { - inner_table_id.database_name.empty() ? nullptr : std::make_shared(inner_table_id.database_name), - std::make_shared(inner_table_id.table_name) - }, - ASTRenameQuery::Table - { - new_table_id.database_name.empty() ? nullptr : std::make_shared(new_table_id.database_name), - std::make_shared(new_target_table_name) - } - }; - rename_elements.emplace_back(std::move(elem)); + rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_table_id.table_name); auto rename = std::make_shared(std::move(rename_elements)); InterpreterRenameQuery(rename, getContext()).execute(); @@ -715,12 +733,6 @@ StorageID StorageMaterializedView::getTargetTableId() const return id; } -void StorageMaterializedView::setTargetTableId(DB::StorageID id) -{ - std::lock_guard guard(target_table_id_mutex); - target_table_id = std::move(id); -} - void StorageMaterializedView::updateTargetTableId(std::optional database_name, std::optional table_name) { std::lock_guard guard(target_table_id_mutex); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 12c0dc651e9..70487c1fad3 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -5,7 +5,7 @@ #include #include -#include +#include namespace DB { @@ -106,7 +106,7 @@ private: /// Will be initialized in constructor StorageID target_table_id = StorageID::createEmpty(); - RefreshTaskHolder refresher; + OwnedRefreshTask refresher; bool refresh_on_start = false; bool has_inner_table = false; @@ -119,14 +119,15 @@ private: void checkStatementCanBeForwarded() const; - /// Prepare to refresh a refreshable materialized view: create query context, create temporary - /// table, form the insert-select query. - /// The output arguments may be assigned before throwing an exception, in which case the caller + ContextMutablePtr createRefreshContext() const; + /// Prepare to refresh a refreshable materialized view: create temporary table and form the + /// insert-select query. + /// out_temp_table_id may be assigned before throwing an exception, in which case the caller /// must drop the temp table before rethrowing. - std::shared_ptr prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const; + std::shared_ptr prepareRefresh(bool append, ContextMutablePtr refresh_context, std::optional & out_temp_table_id) const; StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); + void dropTempTable(StorageID table, ContextMutablePtr refresh_context); - void setTargetTableId(StorageID id); void updateTargetTableId(std::optional database_name, std::optional table_name); }; diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index ad6fa3043a1..528a97535cc 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -257,7 +257,7 @@ $CLICKHOUSE_CLIENT -nq " # Try OFFSET and RANDOMIZE FOR. $CLICKHOUSE_CLIENT -nq " - create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42; + create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42 as x; show create g; system test view g set fake time '2050-02-03 15:30:13';" while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] @@ -359,5 +359,12 @@ $CLICKHOUSE_CLIENT -nq " drop table m; drop table mid;" +# Failing to create inner table. +$CLICKHOUSE_CLIENT -nq " + create materialized view n refresh every 1 second (x Int64) engine MergeTree as select 1 as x from numbers(2);" 2>/dev/null || echo "creating MergeTree without ORDER BY failed, as expected" +$CLICKHOUSE_CLIENT -nq " + create materialized view n refresh every 1 second (x Int64) engine MergeTree order by x as select 1 as x from numbers(2); + drop table n;" + $CLICKHOUSE_CLIENT -nq " drop table refreshes;" From 09f07838af87cdbf3d1c2a829009ce81f7ddd178 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Feb 2024 00:46:35 +0000 Subject: [PATCH 0172/1722] Add 'ness' (as in 'APPEND-ness') to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f1fcd19ea4a..b1e5c208b1c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2083,6 +2083,7 @@ namequota namespaces natively nats +ness nestjs netloc ngram From be04b4439c8d7cf9b313cc6e1d750820653d95ef Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Feb 2024 05:49:14 +0000 Subject: [PATCH 0173/1722] Unbreak --- src/Storages/StorageMaterializedView.cpp | 12 +++++++++++- src/Storages/StorageMaterializedView.h | 1 + .../02932_refreshable_materialized_views.reference | 3 ++- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9f793e06c09..459fc300159 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -448,7 +448,11 @@ StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, Co InterpreterRenameQuery(rename_query, refresh_context).execute(); - return fresh_table; + std::swap(stale_table_id.database_name, fresh_table.database_name); + std::swap(stale_table_id.table_name, fresh_table.table_name); + + setTargetTableId(std::move(fresh_table)); + return stale_table_id; } void StorageMaterializedView::dropTempTable(StorageID table_id, ContextMutablePtr refresh_context) @@ -733,6 +737,12 @@ StorageID StorageMaterializedView::getTargetTableId() const return id; } +void StorageMaterializedView::setTargetTableId(DB::StorageID id) +{ + std::lock_guard guard(target_table_id_mutex); + target_table_id = std::move(id); +} + void StorageMaterializedView::updateTargetTableId(std::optional database_name, std::optional table_name) { std::lock_guard guard(target_table_id_mutex); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 70487c1fad3..a09ee07b3f6 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -128,6 +128,7 @@ private: StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); void dropTempTable(StorageID table, ContextMutablePtr refresh_context); + void setTargetTableId(StorageID id); void updateTargetTableId(std::optional database_name, std::optional table_name); }; diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index 8d4878f4e3f..3bc0d59a608 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -39,7 +39,7 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nE <25: rename during refresh> f Running <27: cancelled> f Scheduled <28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 AS x <29: randomize> 1 1 CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 @@ -59,3 +59,4 @@ CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n <38: append chain> 100 <38: append chain> 100 <38: append chain> 200 +creating MergeTree without ORDER BY failed, as expected From 285936fe3fa2fd7bd850b2b10588ad7216938ed4 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 8 Feb 2024 02:55:03 +0000 Subject: [PATCH 0174/1722] Fix renaming --- src/Storages/StorageMaterializedView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 459fc300159..3691fd5e212 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -578,7 +578,7 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) ASTRenameQuery::Elements rename_elements; assert(inner_table_id.database_name == old_table_id.database_name); - rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_table_id.table_name); + rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_target_table_name); auto rename = std::make_shared(std::move(rename_elements)); InterpreterRenameQuery(rename, getContext()).execute(); From f1ea73624956e80d63ce59b98e0bbf19bfbacb08 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 8 Feb 2024 03:03:12 +0000 Subject: [PATCH 0175/1722] Fix missing column aliases in other tests --- .../queries/0_stateless/00982_low_cardinality_setting_in_mv.sql | 2 +- tests/queries/0_stateless/01160_table_dependencies.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql b/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql index 7192642bcde..e545dec90b7 100644 --- a/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql +++ b/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS mat_view; CREATE TABLE test1 (a LowCardinality(String)) ENGINE=MergeTree() ORDER BY a; CREATE TABLE test2 (a UInt64) engine=MergeTree() ORDER BY a; -CREATE MATERIALIZED VIEW test_mv TO test2 AS SELECT toUInt64(a = 'test') FROM test1; +CREATE MATERIALIZED VIEW test_mv TO test2 AS SELECT toUInt64(a = 'test') AS a FROM test1; DROP TABLE test_mv; DROP TABLE test1; diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index acb6522e9e2..b72acf62610 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -35,7 +35,7 @@ arraySort(loading_dependencies_table), arraySort(loading_dependent_table) from s $CLICKHOUSE_CLIENT -q "select '====='" $CLICKHOUSE_CLIENT -q "alter table t add column x int default in(1, $CLICKHOUSE_DATABASE.s), drop column y" -$CLICKHOUSE_CLIENT -q "create materialized view mv to s as select n from t where n in (select n from join)" +$CLICKHOUSE_CLIENT -q "create materialized view mv to s as select n as x from t where n in (select n from join)" $CLICKHOUSE_CLIENT -q "select table, arraySort(dependencies_table), arraySort(loading_dependencies_table), arraySort(loading_dependent_table) from system.tables where database=currentDatabase() order by table" From 7575115069b0a0072b0e4af7c849c3eabf55d7ea Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 10 Feb 2024 01:57:28 +0000 Subject: [PATCH 0176/1722] Fix race conditions in the test, more small fixes to other tests --- tests/queries/0_stateless/02345_implicit_transaction.sql | 2 +- .../0_stateless/02932_refreshable_materialized_views.sh | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02345_implicit_transaction.sql b/tests/queries/0_stateless/02345_implicit_transaction.sql index ee2e0a07c3e..9496de71e13 100644 --- a/tests/queries/0_stateless/02345_implicit_transaction.sql +++ b/tests/queries/0_stateless/02345_implicit_transaction.sql @@ -3,7 +3,7 @@ CREATE TABLE landing (n Int64) engine=MergeTree order by n; CREATE TABLE target (n Int64) engine=MergeTree order by n; CREATE MATERIALIZED VIEW landing_to_target TO target AS - SELECT n + throwIf(n == 3333) + SELECT n + throwIf(n == 3333) AS n FROM landing; INSERT INTO landing SELECT * FROM numbers(10000); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 528a97535cc..2324041728e 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -298,8 +298,10 @@ $CLICKHOUSE_CLIENT -nq " $CLICKHOUSE_CLIENT -nq "system wait view h2;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" $CLICKHOUSE_CLIENT -nq " select '<31.5: will retry>', last_refresh_result, retry > 0 from refreshes; - truncate table src; - insert into src values (1);" + create table src2 empty as src; + insert into src2 values (1) + exchange tables src and src2; + drop table src2;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result, retry from refreshes -- $LINENO" | xargs`" != 'Finished 0' ] do sleep 0.1 @@ -336,6 +338,7 @@ $CLICKHOUSE_CLIENT -nq " # ALTER to non-APPEND $CLICKHOUSE_CLIENT -nq " alter table k modify refresh every 10 year; + system wait view k; system refresh view k; system wait view k; select '<36: not append>', * from k order by x; From 279d9c78ee30eafec6a3d2dc711d2c4e0496944a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 19 Feb 2024 14:52:19 +0000 Subject: [PATCH 0177/1722] Fix race in wait --- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index e593023caee..fdcf358cc20 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -160,7 +160,7 @@ void RefreshTask::cancel() void RefreshTask::wait() { std::unique_lock lock(mutex); - refresh_cv.wait(lock, [&] { return info.state != RefreshState::Running; }); + refresh_cv.wait(lock, [&] { return info.state != RefreshState::Running && !refresh_immediately; }); if (info.last_refresh_result == LastRefreshResult::Error) throw Exception(ErrorCodes::REFRESH_FAILED, "Refresh failed: {}", info.exception_message); } From 78a3d0d4f71554c7f07e92f3996b56a28d5acbdf Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 21 Feb 2024 12:51:19 +0000 Subject: [PATCH 0178/1722] Fix parallel_mv perf test --- tests/performance/parallel_mv.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/parallel_mv.xml b/tests/performance/parallel_mv.xml index 5b856740a19..0bf5ed1be09 100644 --- a/tests/performance/parallel_mv.xml +++ b/tests/performance/parallel_mv.xml @@ -11,13 +11,13 @@ create table mt_4 (n UInt64, s String) engine = MergeTree order by tuple() create materialized view mv_1 to mt_1 as - select number, toString(number) from main_table where number % 13 != 0 + select number as n, toString(number) as s from main_table where number % 13 != 0 create materialized view mv_2 to mt_2 as - select number, toString(number) from main_table where number % 13 != 1 + select number as n, toString(number) as s from main_table where number % 13 != 1 create materialized view mv_3 to mt_3 as - select number, toString(number) from main_table where number % 13 != 3 + select number as n, toString(number) as s from main_table where number % 13 != 3 create materialized view mv_4 to mt_4 as - select number, toString(number) from main_table where number % 13 != 4 + select number as n, toString(number) as s from main_table where number % 13 != 4 SYSTEM STOP MERGES main_table SYSTEM STOP MERGES mt_1 From ef5070c1e66e41686eef983272b9a434257c34bf Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 9 Mar 2024 02:12:53 +0000 Subject: [PATCH 0179/1722] Fix conflict --- src/Parsers/ASTSystemQuery.cpp | 1 + .../02932_refreshable_materialized_views.reference | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a730ea0ba3d..a183eb4c0f6 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -375,6 +375,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::START_VIEW: case Type::STOP_VIEW: case Type::CANCEL_VIEW: + case Type::WAIT_VIEW: { settings.ostr << ' '; print_database_table(); diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index 3bc0d59a608..3c6c8b2d778 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -3,13 +3,13 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n) <2: refreshed> 3 1 1 <3: time difference at least> 1000 <4: next refresh in> 2 -<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 +<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03 <4.5: altered> Scheduled Finished 2052-01-01 00:00:00 CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 <6: refreshed> 2 <7: refreshed> Scheduled Finished 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a <7.5: created dependent> 2052-11-11 11:11:11 <8: refreshed> 20 <9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 @@ -39,7 +39,7 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nE <25: rename during refresh> f Running <27: cancelled> f Scheduled <28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 AS x +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x <29: randomize> 1 1 CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 From 5ce3aaf49ef947e4481e8d7934ee84190ef04542 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 23 Mar 2024 01:29:24 +0000 Subject: [PATCH 0180/1722] Fix conflicts --- src/Parsers/ParserRefreshStrategy.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index de7eea93614..4f3b7c66558 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -97,7 +97,7 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec refresh->set(refresh->settings, settings); } - if (ParserKeyword{"APPEND"}.ignore(pos, expected)) + if (ParserKeyword{Keyword::APPEND}.ignore(pos, expected)) refresh->append = true; node = refresh; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 3691fd5e212..a78bb44eb66 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -575,12 +575,11 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto new_target_table_name = generateInnerTableName(new_table_id); - ASTRenameQuery::Elements rename_elements; assert(inner_table_id.database_name == old_table_id.database_name); + auto rename = std::make_shared(); rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_target_table_name); - auto rename = std::make_shared(std::move(rename_elements)); InterpreterRenameQuery(rename, getContext()).execute(); updateTargetTableId(new_table_id.database_name, new_target_table_name); } From 0549d154058169e9f4831923c2c2d2e8fdc295b5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 26 Mar 2024 04:10:33 +0000 Subject: [PATCH 0181/1722] tidy --- src/Storages/MaterializedView/RefreshTask.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 411eb6fa7b4..e8e3e45ace3 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -169,7 +169,7 @@ struct OwnedRefreshTask RefreshTaskHolder ptr; OwnedRefreshTask() = default; - OwnedRefreshTask(RefreshTaskHolder p) : ptr(std::move(p)) {} + explicit OwnedRefreshTask(RefreshTaskHolder p) : ptr(std::move(p)) {} OwnedRefreshTask(OwnedRefreshTask &&) = default; OwnedRefreshTask & operator=(OwnedRefreshTask &&) = default; @@ -177,7 +177,7 @@ struct OwnedRefreshTask RefreshTask* operator->() const { return ptr.get(); } RefreshTask& operator*() const { return *ptr; } - operator bool() const { return ptr != nullptr; } + explicit operator bool() const { return ptr != nullptr; } }; } From 90c2b29ea28afa99f4a2dd7919ffc3274cf04764 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 01:44:32 +0000 Subject: [PATCH 0182/1722] Log retry information, remove outdated comment --- src/Storages/MaterializedView/RefreshTask.cpp | 24 +++++++++---------- src/Storages/MaterializedView/RefreshTask.h | 3 ++- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index fdcf358cc20..71feece13e6 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -1,7 +1,6 @@ #include -#include - +#include #include #include #include @@ -10,6 +9,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -312,13 +312,7 @@ void RefreshTask::refreshTask() catch (...) { if (!interrupt_execution.load()) - { - PreformattedMessage message = getCurrentExceptionMessageAndPattern(true); - auto text = message.text; - message.text = fmt::format("Refresh failed: {}", message.text); - LOG_ERROR(log, message); - exception = text; - } + exception = getCurrentExceptionMessage(true); } lock.lock(); @@ -332,7 +326,10 @@ void RefreshTask::refreshTask() { info.last_refresh_result = LastRefreshResult::Error; info.exception_message = *exception; - scheduleRetryOrSkipToNextRefresh(now); + + String retry_info = scheduleRetryOrSkipToNextRefresh(now); + + LOG_ERROR(log, "Refresh failed ({}): {}", retry_info, *exception); } else if (!refreshed) { @@ -445,7 +442,6 @@ void RefreshTask::executeRefreshUnlocked(bool append) throw; } - /// Drop the old table (outside the try-catch so we don't try to drop the other table if this fails). if (table_to_drop.has_value()) view->dropTempTable(table_to_drop.value(), refresh_context); } @@ -463,12 +459,13 @@ void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point n info.next_refresh_time = UInt32(secs.time_since_epoch().count()); } -void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) +String RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) { + Int64 attempt_number = num_retries + 1; if (refresh_settings.refresh_retries >= 0 && num_retries >= refresh_settings.refresh_retries) { advanceNextRefreshTime(now); - return; + return fmt::format("attempt {}/{}, next refresh at {:%Y.%m.%d %T}", attempt_number, refresh_settings.refresh_retries + 1, next_refresh_actual); } num_retries += 1; @@ -483,6 +480,7 @@ void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::ti delay_ms = refresh_settings.refresh_retry_max_backoff_ms; next_refresh_actual = now + std::chrono::milliseconds(delay_ms); + return fmt::format("attempt {}/{}, retry in {} seconds", attempt_number, refresh_settings.refresh_retries, (delay_ms + 500) / 1000); } bool RefreshTask::arriveDependency(const StorageID & parent) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index e8e3e45ace3..54253548795 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -151,7 +151,8 @@ private: void advanceNextRefreshTime(std::chrono::system_clock::time_point now); /// Either advances next_refresh_actual using exponential backoff or does advanceNextRefreshTime(). - void scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); + /// Returns human-readable information for logging. + String scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); /// Returns true if all dependencies are fulfilled now. Refills remaining_dependencies in this case. bool arriveDependency(const StorageID & parent); From f40b45b78cb5838af70df8eb9611370a14a7e7b9 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 02:20:29 +0000 Subject: [PATCH 0183/1722] Avoid fmt/chrono.h because clang refuses to compile it --- src/Storages/MaterializedView/RefreshTask.cpp | 15 ++++++--------- src/Storages/MaterializedView/RefreshTask.h | 3 +-- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 71feece13e6..781cefbf449 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -326,10 +325,9 @@ void RefreshTask::refreshTask() { info.last_refresh_result = LastRefreshResult::Error; info.exception_message = *exception; - - String retry_info = scheduleRetryOrSkipToNextRefresh(now); - - LOG_ERROR(log, "Refresh failed ({}): {}", retry_info, *exception); + Int64 attempt_number = num_retries + 1; + scheduleRetryOrSkipToNextRefresh(now); + LOG_ERROR(log, "Refresh failed (attempt {}/{}): {}", attempt_number, refresh_settings.refresh_retries + 1, *exception); } else if (!refreshed) { @@ -459,13 +457,12 @@ void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point n info.next_refresh_time = UInt32(secs.time_since_epoch().count()); } -String RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) +void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) { - Int64 attempt_number = num_retries + 1; if (refresh_settings.refresh_retries >= 0 && num_retries >= refresh_settings.refresh_retries) { advanceNextRefreshTime(now); - return fmt::format("attempt {}/{}, next refresh at {:%Y.%m.%d %T}", attempt_number, refresh_settings.refresh_retries + 1, next_refresh_actual); + return; } num_retries += 1; @@ -480,7 +477,7 @@ String RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock:: delay_ms = refresh_settings.refresh_retry_max_backoff_ms; next_refresh_actual = now + std::chrono::milliseconds(delay_ms); - return fmt::format("attempt {}/{}, retry in {} seconds", attempt_number, refresh_settings.refresh_retries, (delay_ms + 500) / 1000); + return; } bool RefreshTask::arriveDependency(const StorageID & parent) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 54253548795..e8e3e45ace3 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -151,8 +151,7 @@ private: void advanceNextRefreshTime(std::chrono::system_clock::time_point now); /// Either advances next_refresh_actual using exponential backoff or does advanceNextRefreshTime(). - /// Returns human-readable information for logging. - String scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); + void scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); /// Returns true if all dependencies are fulfilled now. Refills remaining_dependencies in this case. bool arriveDependency(const StorageID & parent); From e1b9844086f7cd31b1d0bd2fa7de80d521325e5f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 05:29:44 +0000 Subject: [PATCH 0184/1722] Lint --- src/Storages/MaterializedView/RefreshTask.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 781cefbf449..6cc85cc0f82 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -477,7 +477,6 @@ void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::ti delay_ms = refresh_settings.refresh_retry_max_backoff_ms; next_refresh_actual = now + std::chrono::milliseconds(delay_ms); - return; } bool RefreshTask::arriveDependency(const StorageID & parent) From 2e8a7ecc57a103997c5c659960c91d28fab1aa86 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 17:43:02 +0000 Subject: [PATCH 0185/1722] Remove unused variable --- src/Interpreters/InterpreterCreateQuery.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ee57c376b31..6e5b500cd69 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -946,10 +946,6 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC /// This is not strict validation, just catches common errors that would make the view not work. /// It's possible to circumvent these checks by ALTERing the view or target table after creation. - String table_engine; - if (create.storage && create.storage->engine) - table_engine = create.storage->engine->name; - NamesAndTypesList all_output_columns; bool check_columns = false; if (create.to_table_id) @@ -961,7 +957,6 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC { all_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock().getNamesAndTypesList(); check_columns = true; - table_engine = to_table->getName(); } } else if (!properties.columns_inferred_from_select_query) From f4c13a1d9ad82dbc4382729490e3226c81073c1c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 19:13:29 +0000 Subject: [PATCH 0186/1722] Add compatibility setting allow_materialized_view_with_bad_select --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 44 +++++++++---------- .../test_replicated_database/test.py | 1 + .../02932_refreshable_materialized_views.sh | 8 ++++ 5 files changed, 33 insertions(+), 22 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fbab72446a0..954e21dd988 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -635,6 +635,7 @@ class IColumn; M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ + M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns, or outputs columns with unexpected names (which will be ignored). It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index fbc414d4f2f..9d3832e451d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -115,6 +115,7 @@ static const std::mapgetEngineName()); } + Block input_block; + bool relaxed = getContext()->getSettingsRef().allow_materialized_view_with_bad_select; + if (check_columns) { - Block input_block; - - if (getContext()->getSettingsRef().allow_experimental_analyzer) + try { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } } - else + catch (Exception &) { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); + if (!relaxed) + throw; + check_columns = false; } + } + if (check_columns) + { std::unordered_map output_types; for (const NameAndTypePair & nt : all_output_columns) output_types[nt.name] = nt.type; @@ -1001,21 +1014,8 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC input_columns.push_back(input_column.cloneEmpty()); output_columns.push_back(ColumnWithTypeAndName(it->second->createColumn(), it->second, input_column.name)); } - else if (create.refresh_strategy) - { - /// Unrecognized columns produced by SELECT query are allowed for regular materialized - /// views, but not for refreshable ones. This is in part because it was easier to - /// implement, in part because refreshable views have less concern about ALTERing target - /// tables. - /// - /// The motivating scenario for allowing this in regular MV is ALTERing the table+query. - /// Suppose the user removes a column from target table, then a minute later - /// correspondingly updates the view's query to not produce that column. - /// If MV didn't allow unrecognized columns then during that minute all INSERTs into the - /// source table would fail - unacceptable. - /// For refreshable views, during that minute refreshes will fail - acceptable. + else if (!relaxed) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "SELECT query outputs column with name '{}', which is not found in the target table. Use 'AS' to assign alias that matches a column name.", input_column.name); - } } if (input_columns.empty()) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index f23384b5c04..4ecb1aefba7 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -708,6 +708,7 @@ def create_some_tables(db): "distributed_ddl_task_timeout": 0, "allow_experimental_object_type": 1, "allow_suspicious_codecs": 1, + "allow_materialized_view_with_bad_select": 1, } main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 2324041728e..580d1bdc190 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -369,5 +369,13 @@ $CLICKHOUSE_CLIENT -nq " create materialized view n refresh every 1 second (x Int64) engine MergeTree order by x as select 1 as x from numbers(2); drop table n;" +# Reading from table that doesn't exist yet. +$CLICKHOUSE_CLIENT -nq " + create materialized view o refresh every 1 second (x Int64) engine Memory as select x from nonexist; -- { serverError UNKNOWN_TABLE } + create materialized view o (x Int64) engine Memory as select x from nonexist; -- { serverError UNKNOWN_TABLE } + create materialized view o (x Int64) engine Memory as select x from nope.nonexist; -- { serverError UNKNOWN_DATABASE } + create materialized view o refresh every 1 second (x Int64) engine Memory as select x from nope.nonexist settings allow_materialized_view_with_bad_select = 1; + drop table o;" + $CLICKHOUSE_CLIENT -nq " drop table refreshes;" From b3b70b037ff1b7f317ac5fe0b5219e1308971581 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 12 Jun 2024 00:25:31 +0000 Subject: [PATCH 0187/1722] Revert some of the strictness, because of comment in 01019_materialized_view_select_extra_columns --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 18 +++++++++++++++--- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 954e21dd988..93e60541352 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -635,7 +635,7 @@ class IColumn; M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns, or outputs columns with unexpected names (which will be ignored). It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ + M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f0c450f2ff3..1f6cf31e3d3 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -973,7 +973,6 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC } Block input_block; - bool relaxed = getContext()->getSettingsRef().allow_materialized_view_with_bad_select; if (check_columns) { @@ -992,7 +991,7 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC } catch (Exception &) { - if (!relaxed) + if (!getContext()->getSettingsRef().allow_materialized_view_with_bad_select) throw; check_columns = false; } @@ -1014,8 +1013,21 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC input_columns.push_back(input_column.cloneEmpty()); output_columns.push_back(ColumnWithTypeAndName(it->second->createColumn(), it->second, input_column.name)); } - else if (!relaxed) + else if (create.refresh_strategy) + { + /// Unrecognized columns produced by SELECT query are allowed by regular materialized + /// views, but not by refreshable ones. This is in part because it was easier to + /// implement, in part because refreshable views have less concern about ALTERing target + /// tables. + /// + /// The motivating scenario for allowing this in regular MV is ALTERing the table+query. + /// Suppose the user removes a column from target table, then a minute later + /// correspondingly updates the view's query to not produce that column. + /// If MV didn't allow unrecognized columns then during that minute all INSERTs into the + /// source table would fail - unacceptable. + /// For refreshable views, during that minute refreshes will fail - acceptable. throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "SELECT query outputs column with name '{}', which is not found in the target table. Use 'AS' to assign alias that matches a column name.", input_column.name); + } } if (input_columns.empty()) From 9957fe1734260cecd383227abc9ab75283e95d9b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 20 Jun 2024 18:38:12 +0000 Subject: [PATCH 0188/1722] Comment fixes --- docs/en/sql-reference/statements/create/view.md | 16 ++++++++-------- src/Interpreters/InterpreterCreateQuery.cpp | 3 ++- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 3e0766794f1..950d8207028 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -13,8 +13,8 @@ Creates a new view. Views can be [normal](#normal-view), [materialized](#materia Syntax: ``` sql -CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] -[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] +CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] +[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... ``` @@ -54,8 +54,8 @@ SELECT * FROM view(column1=value1, column2=value2 ...) ## Materialized View ``` sql -CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] -[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] +CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] +[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... ``` @@ -90,7 +90,7 @@ Given that `POPULATE` works like `CREATE TABLE ... AS SELECT ...` it has limitat - It is not supported with Replicated database - It is not supported in ClickHouse cloud -Instead a separate `INSERT ... SELECT` can be used. +Instead a separate `INSERT ... SELECT` can be used. ::: A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`. @@ -108,7 +108,7 @@ To delete a view, use [DROP VIEW](../../../sql-reference/statements/drop.md#drop `DEFINER` and `SQL SECURITY` allow you to specify which ClickHouse user to use when executing the view's underlying query. `SQL SECURITY` has three legal values: `DEFINER`, `INVOKER`, or `NONE`. You can specify any existing user or `CURRENT_USER` in the `DEFINER` clause. -The following table will explain which rights are required for which user in order to select from view. +The following table will explain which rights are required for which user in order to select from view. Note that regardless of the SQL security option, in every case it is still required to have `GRANT SELECT ON ` in order to read from it. | SQL security option | View | Materialized View | @@ -128,7 +128,7 @@ If `DEFINER`/`SQL SECURITY` aren't specified, the default values are used: If a view is attached without `DEFINER`/`SQL SECURITY` specified, the default value is `SQL SECURITY NONE` for the materialized view and `SQL SECURITY INVOKER` for the normal view. -To change SQL security for an existing view, use +To change SQL security for an existing view, use ```sql ALTER TABLE MODIFY SQL SECURITY { DEFINER | INVOKER | NONE } [DEFINER = { user | CURRENT_USER }] ``` @@ -178,7 +178,7 @@ Differences from regular non-refreshable materialized views: * No restrictions on the SELECT query. Table functions (e.g. `url()`), views, UNION, JOIN, are all allowed. :::note -The settings in the The settings in `REFRESH ... SETTINGS` part of the query are refresh settings (e.g. `refresh_retries`), distinct from regular settings (e.g. `max_threads`). Regular settings can be specified using `SETTINGS` at the end of the query. +The settings in the `REFRESH ... SETTINGS` part of the query are refresh settings (e.g. `refresh_retries`), distinct from regular settings (e.g. `max_threads`). Regular settings can be specified using `SETTINGS` at the end of the query. ::: :::note diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1f6cf31e3d3..186df8f2108 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -944,7 +944,8 @@ void validateVirtualColumns(const IStorage & storage) void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTCreateQuery & create, const TableProperties & properties, const DatabasePtr & database) { /// This is not strict validation, just catches common errors that would make the view not work. - /// It's possible to circumvent these checks by ALTERing the view or target table after creation. + /// It's possible to circumvent these checks by ALTERing the view or target table after creation; + /// we should probably do some of these checks on ALTER as well. NamesAndTypesList all_output_columns; bool check_columns = false; From f72b0335cdc1d76b5aeb6d130900a68a8f4c8e3b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 22 Jun 2024 02:27:06 +0000 Subject: [PATCH 0189/1722] Test workaround --- .../queries/0_stateless/02932_refreshable_materialized_views.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 580d1bdc190..d5b07287936 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -298,7 +298,7 @@ $CLICKHOUSE_CLIENT -nq " $CLICKHOUSE_CLIENT -nq "system wait view h2;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" $CLICKHOUSE_CLIENT -nq " select '<31.5: will retry>', last_refresh_result, retry > 0 from refreshes; - create table src2 empty as src; + create table src2 (x Int8) engine Memory; insert into src2 values (1) exchange tables src and src2; drop table src2;" From aadf1536a40bd53c6a1b6359cf652854f134599b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 25 Jun 2024 22:28:01 +0000 Subject: [PATCH 0190/1722] fix protocol --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 30 ++++++++------------- 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index 324f8ae3a02..a83b976ae09 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -24,6 +24,8 @@ void WriteBufferFromPocoSocketChunked::enableChunked() /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); pos += std::min(available(), sizeof(*chunk_size_ptr)); + /// Pretend finishChunk() was just called to prevent sending empty chunk if finishChunk() called immediately + last_finish_chunk = chunk_size_ptr; } void WriteBufferFromPocoSocketChunked::finishChunk() @@ -33,7 +35,8 @@ void WriteBufferFromPocoSocketChunked::finishChunk() if (pos <= reinterpret_cast(chunk_size_ptr) + sizeof(*chunk_size_ptr)) { - if (chunk_size_ptr == last_finish_chunk) // prevent duplicate finish chunk + /// Prevent duplicate finish chunk (and finish chunk right after enableChunked()) + if (chunk_size_ptr == last_finish_chunk) return; /// If current chunk is empty it means we are finishing a chunk previously sent by next(), @@ -85,7 +88,7 @@ void WriteBufferFromPocoSocketChunked::finishChunk() } /// Buffer end-of-chunk - *reinterpret_cast(pos) = 0; + setValue(reinterpret_cast(pos), 0); pos += sizeof(*chunk_size_ptr); /// Initialize next chunk chunk_size_ptr = reinterpret_cast(pos); @@ -114,7 +117,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl() return; } - /// next() after finishChunk ar the end of the buffer + /// next() after finishChunk at the end of the buffer if (finishing < sizeof(*chunk_size_ptr)) { pos -= finishing; @@ -135,21 +138,6 @@ void WriteBufferFromPocoSocketChunked::nextImpl() return; } - /// Send end-of-chunk buffered by finishChunk - if (offset() == 2 * sizeof(*chunk_size_ptr) && last_finish_chunk == chunk_size_ptr) - { - pos -= sizeof(*chunk_size_ptr); - /// Send end-of-chunk - WriteBufferFromPocoSocket::nextImpl(); - /// Initialize next chunk - chunk_size_ptr = reinterpret_cast(working_buffer.begin()); - nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - - last_finish_chunk = chunk_size_ptr; - - return; - } - /// Prevent sending empty chunk if (offset() == sizeof(*chunk_size_ptr)) { @@ -172,8 +160,12 @@ void WriteBufferFromPocoSocketChunked::nextImpl() return; } + bool initialize_last_finish_chunk = false; if (pos - reinterpret_cast(chunk_size_ptr) == sizeof(*chunk_size_ptr)) // next() after finishChunk + { pos -= sizeof(*chunk_size_ptr); + initialize_last_finish_chunk = true; + } else // fill up current chunk size { setValue(chunk_size_ptr, toLittleEndian(static_cast(pos - reinterpret_cast(chunk_size_ptr) - sizeof(*chunk_size_ptr)))); @@ -194,7 +186,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl() chunk_size_ptr = reinterpret_cast(working_buffer.begin()); nextimpl_working_buffer_offset = sizeof(*chunk_size_ptr); - last_finish_chunk = nullptr; + last_finish_chunk = initialize_last_finish_chunk ? chunk_size_ptr : nullptr; } void WriteBufferFromPocoSocketChunked::finalizeImpl() From 9eec8344279082a3d02583c092f3c90b85a76fa3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 03:19:16 +0000 Subject: [PATCH 0191/1722] better chunked protocol negotiation, comments, review suggestions --- src/Client/Connection.cpp | 37 +++++++++++++++------- src/IO/ReadBufferFromPocoSocketChunked.cpp | 11 ++++--- src/IO/ReadBufferFromPocoSocketChunked.h | 4 +-- src/IO/WriteBufferFromPocoSocketChunked.h | 10 ++++-- src/Server/TCPHandler.cpp | 17 +++++++--- src/Server/TCPHandler.h | 4 +-- 6 files changed, 56 insertions(+), 27 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 9f727b974ee..c41229c7226 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -208,11 +208,20 @@ void Connection::connect(const ConnectionTimeouts & timeouts) sendHello(); receiveHello(timeouts.handshake_timeout); - bool out_chunked = false; - bool in_chunked = false; - if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { + /// Client side of chunked protocol negotiation. + /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. + /// Not optional types are strict meaning that server only supports this type, optional means that + /// server prefer this type but capable to work in opposite. + /// Client selects which type it is going to communicate based on the settings from config or arguments, + /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. + /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example + /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case + /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// will send appropriate exception and disconnect client. + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) { bool chunked_srv = chunked_srv_str.starts_with("chunked"); @@ -235,20 +244,24 @@ void Connection::connect(const ConnectionTimeouts & timeouts) return chunked_srv; }; - out_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send"); - in_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv"); + proto_send_chunked = is_chunked(proto_recv_chunked_srv, proto_send_chunked, "send") ? "chunked" : "notchunked"; + proto_recv_chunked = is_chunked(proto_send_chunked_srv, proto_recv_chunked, "recv") ? "chunked" : "notchunked"; + } + else + { + if (proto_send_chunked == "chunked" || proto_recv_chunked == "chunked") + throw NetException( + ErrorCodes::NETWORK_ERROR, + "Incompatible protocol: server's version is too old and doesn't support chunked protocol while client settings require it."); } if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); - if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) - { - if (out_chunked) - out->enableChunked(); - if (in_chunked) - in->enableChunked(); - } + if (proto_send_chunked == "chunked") + out->enableChunked(); + if (proto_recv_chunked == "chunked") + in->enableChunked(); LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", server_name, server_version_major, server_version_minor, server_version_patch); diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 798be547e99..07598f2adf4 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -28,6 +28,7 @@ void ReadBufferFromPocoSocketChunked::enableChunked() return; chunked = 1; data_end = buffer().end(); + /// Resize working buffer so any next read will call nextImpl working_buffer.resize(offset()); chunk_left = 0; next_chunk = 0; @@ -51,7 +52,7 @@ bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const } -bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) +bool ReadBufferFromPocoSocketChunked::loadNextChunk(Position c_pos, bool cont) { auto buffered = std::min(static_cast(data_end - c_pos), sizeof(next_chunk)); @@ -73,7 +74,7 @@ bool ReadBufferFromPocoSocketChunked::load_next_chunk(Position c_pos, bool cont) return true; } -bool ReadBufferFromPocoSocketChunked::process_chunk_left(Position c_pos) +bool ReadBufferFromPocoSocketChunked::processChunkLeft(Position c_pos) { if (data_end - c_pos < chunk_left) { @@ -88,7 +89,7 @@ bool ReadBufferFromPocoSocketChunked::process_chunk_left(Position c_pos) c_pos += chunk_left; - if (!load_next_chunk(c_pos, true)) + if (!loadNextChunk(c_pos, true)) return false; chunk_left = 0; @@ -115,7 +116,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() if (c_pos > data_end) c_pos = data_end; - if (!load_next_chunk(c_pos)) + if (!loadNextChunk(c_pos)) return false; chunk_left = next_chunk; @@ -159,7 +160,7 @@ bool ReadBufferFromPocoSocketChunked::nextImpl() c_pos = buffer().begin(); } - return process_chunk_left(c_pos); + return processChunkLeft(c_pos); } } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index acf0edafe0a..943a50f5d08 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -92,8 +92,8 @@ public: Poco::Net::SocketAddress ourAddress() { return our_address; } protected: - bool load_next_chunk(Position c_pos, bool cont = false); - bool process_chunk_left(Position c_pos); + bool loadNextChunk(Position c_pos, bool cont = false); + bool processChunkLeft(Position c_pos); bool nextImpl() override; Poco::Net::SocketAddress our_address; diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 269c6d66dda..8270ca445c9 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -10,8 +10,14 @@ namespace DB class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) {} - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) {} + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) + { + chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); + } + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) + { + chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); + } void enableChunked(); void finishChunk(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 960860a3c13..3093c508c22 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -277,19 +277,28 @@ void TCPHandler::runImpl() if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { + /// Server side of chunked protocol negotiation. + /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. + /// Not optional types are strict meaning that server only supports this type, optional means that + /// server prefer this type but capable to work in opposite. + /// Client selects which type it is going to communicate based on the settings from config or arguments, + /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. + /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example + /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case + /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// will send appropriate exception and disconnect client. + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) { bool chunked_srv = chunked_srv_str.starts_with("chunked"); bool optional_srv = chunked_srv_str.ends_with("_optional"); bool chunked_cl = chunked_cl_str.starts_with("chunked"); - bool optional_cl = chunked_cl_str.ends_with("_optional"); if (optional_srv) return chunked_cl; - if (optional_cl) - return chunked_srv; + if (chunked_cl != chunked_srv) throw NetException( ErrorCodes::NETWORK_ERROR, diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 88c8fc6d52c..f6400161041 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -187,8 +187,8 @@ private: UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; UInt32 client_tcp_protocol_version = 0; - String proto_send_chunked_cl; - String proto_recv_chunked_cl; + String proto_send_chunked_cl = "notchunked"; + String proto_recv_chunked_cl = "notchunked"; String quota_key; /// Connection settings, which are extracted from a context. From 6112ef710c2d949c3c8824fcf0e7c148f5deaea4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 03:43:28 +0000 Subject: [PATCH 0192/1722] fix style --- src/Client/Connection.cpp | 8 ++++---- src/Server/TCPHandler.cpp | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index c41229c7226..14ffff10081 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -211,17 +211,17 @@ void Connection::connect(const ConnectionTimeouts & timeouts) if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { /// Client side of chunked protocol negotiation. - /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// Server advertises its protocol capabilities (separate for send and receive channels) by sending /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. /// Not optional types are strict meaning that server only supports this type, optional means that /// server prefer this type but capable to work in opposite. /// Client selects which type it is going to communicate based on the settings from config or arguments, /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example - /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case - /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case + /// client should interrupt this connection. However if client continues with incompatible protocol type request, server /// will send appropriate exception and disconnect client. - + auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) { bool chunked_srv = chunked_srv_str.starts_with("chunked"); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3093c508c22..d5afb624e77 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -279,15 +279,15 @@ void TCPHandler::runImpl() { /// Server side of chunked protocol negotiation. - /// Server advertises its protocol capabilities (separate for send and recieve channels) by sending + /// Server advertises its protocol capabilities (separate for send and receive channels) by sending /// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional. /// Not optional types are strict meaning that server only supports this type, optional means that /// server prefer this type but capable to work in opposite. /// Client selects which type it is going to communicate based on the settings from config or arguments, /// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake. /// Client can detect if server's protocol capabilities are not compatible with client's settings (for example - /// server strictly requires chunked protocol but client's settings only allowes notchunked protocol) - in such case - /// client should interrup this connection. However if client continues with incompatible protocol type request, server + /// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case + /// client should interrupt this connection. However if client continues with incompatible protocol type request, server /// will send appropriate exception and disconnect client. auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction) From 3f3305a63a1218dc944ac7b3a8540f084a57a039 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 04:33:52 +0000 Subject: [PATCH 0193/1722] fix server settings --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d5afb624e77..40fd3848455 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -310,8 +310,8 @@ void TCPHandler::runImpl() return chunked_srv; }; - bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked"), proto_recv_chunked_cl, "send"); - bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked"), proto_send_chunked_cl, "recv"); + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked_optional"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked_optional"), proto_send_chunked_cl, "recv"); if (out_chunked) out->enableChunked(); From 32e6bed4ee8aecf97ddd289ca869f8da096d58af Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 14:04:33 +0000 Subject: [PATCH 0194/1722] bug fix, ubsan paranoia fix --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index a83b976ae09..b6d9efda815 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -7,10 +7,9 @@ namespace { template -const T & setValue(T * typed_ptr, std::type_identity_t val) +void setValue(T * typed_ptr, std::type_identity_t val) { - memcpy(typed_ptr, &val, sizeof(T)); - return *typed_ptr; + memcpy(static_cast(typed_ptr), &val, sizeof(T)); } } @@ -84,6 +83,7 @@ void WriteBufferFromPocoSocketChunked::finishChunk() finishing = available(); pos += available(); chunk_size_ptr = reinterpret_cast(pos); + last_finish_chunk = chunk_size_ptr; return; } From 30a9c38c9596b40555c8ec041257b53cd10b9abc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 26 Jun 2024 20:43:13 +0000 Subject: [PATCH 0195/1722] fix buffer size check --- src/IO/ReadBufferFromPocoSocketChunked.cpp | 10 +++++----- src/IO/WriteBufferFromPocoSocketChunked.cpp | 11 +++++++++++ src/IO/WriteBufferFromPocoSocketChunked.h | 11 +++-------- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 07598f2adf4..93afeadba60 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -16,11 +16,11 @@ ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Sock {} ReadBufferFromPocoSocketChunked::ReadBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & read_event_, size_t buf_size) - : ReadBufferFromPocoSocketBase(socket_, read_event_, buf_size), our_address(socket_.address()), log(getLogger("Protocol")) - -{ - chassert(buf_size <= std::numeric_limits::max()); -} + : ReadBufferFromPocoSocketBase( + socket_, read_event_, + std::min(buf_size, static_cast(std::numeric_limits::max()))), + our_address(socket_.address()), log(getLogger("Protocol")) +{} void ReadBufferFromPocoSocketChunked::enableChunked() { diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index b6d9efda815..98c5126c24b 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -17,6 +17,17 @@ void setValue(T * typed_ptr, std::type_identity_t val) namespace DB { +WriteBufferFromPocoSocketChunked::WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size) + : WriteBufferFromPocoSocketChunked(socket_, ProfileEvents::end(), buf_size) +{} + +WriteBufferFromPocoSocketChunked::WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size) + : WriteBufferFromPocoSocket( + socket_, write_event_, + std::clamp(buf_size, sizeof(*chunk_size_ptr) + 1, static_cast(std::numeric_limits>::max()))), + log(getLogger("Protocol")) +{} + void WriteBufferFromPocoSocketChunked::enableChunked() { chunked = true; diff --git a/src/IO/WriteBufferFromPocoSocketChunked.h b/src/IO/WriteBufferFromPocoSocketChunked.h index 8270ca445c9..13a277e3bfb 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.h +++ b/src/IO/WriteBufferFromPocoSocketChunked.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -10,14 +11,8 @@ namespace DB class WriteBufferFromPocoSocketChunked: public WriteBufferFromPocoSocket { public: - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, buf_size), log(getLogger("Protocol")) - { - chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); - } - explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : WriteBufferFromPocoSocket(socket_, write_event_, buf_size), log(getLogger("Protocol")) - { - chassert(buf_size <= std::numeric_limits>::max() && buf_size > sizeof(*chunk_size_ptr)); - } + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + explicit WriteBufferFromPocoSocketChunked(Poco::Net::Socket & socket_, const ProfileEvents::Event & write_event_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); void enableChunked(); void finishChunk(); From 71d71bd5fe8884b5f2b11e06302dce8e511f5b7c Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 27 Jun 2024 20:06:14 +0800 Subject: [PATCH 0196/1722] Add rows_before_group_by_counter --- src/Client/ClientBase.cpp | 2 + src/Core/Settings.h | 1 + src/Processors/Formats/IOutputFormat.cpp | 3 +- src/Processors/Formats/IOutputFormat.h | 8 ++++ src/Processors/Formats/LazyOutputFormat.cpp | 4 ++ src/Processors/Formats/LazyOutputFormat.h | 1 + .../Formats/PullingOutputFormat.cpp | 5 ++- src/Processors/Formats/PullingOutputFormat.h | 1 + src/Processors/IProcessor.h | 6 +++ src/Processors/Sources/DelayedSource.h | 2 + src/Processors/Sources/RemoteSource.cpp | 30 ++++++++++----- src/Processors/Sources/RemoteSource.h | 3 ++ .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/AggregatingTransform.h | 11 ++++-- src/QueryPipeline/ProfileInfo.cpp | 17 +++++++++ src/QueryPipeline/ProfileInfo.h | 13 +++++++ src/QueryPipeline/QueryPipeline.cpp | 37 +++++++++++++------ src/Server/GRPCServer.cpp | 1 + src/Server/grpc_protos/clickhouse_grpc.proto | 2 + 19 files changed, 123 insertions(+), 28 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 15a4836ef7a..dbb67d230d5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -522,6 +522,8 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) { if (profile_info.hasAppliedLimit() && output_format) output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); + if (profile_info.hasAppliedGroupBy() && output_format) + output_format->setRowsBeforeGroupBy(profile_info.getRowsBeforeGroupBy()); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e3c122467bd..09291d4300d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1200,6 +1200,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ + M(Bool, exact_rows_before_group_by, false, "When enabled, ClickHouse will provide exact value for rows_before_group_by_at_least statistic, but with the cost that the data before group by will have to be read completely", 0) \ M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 88a6fb1e92f..4191bf9f0fe 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -71,7 +71,8 @@ void IOutputFormat::work() { if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) setRowsBeforeLimit(rows_before_limit_counter->get()); - + if (rows_before_group_by_counter && rows_before_group_by_counter->hasAppliedLimit()) + setRowsBeforeGroupBy(rows_before_group_by_counter->get()); finalize(); if (auto_flush) flush(); diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index cae2ab7691e..9bb7cccb612 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -41,6 +41,12 @@ public: /// Counter to calculate rows_before_limit_at_least in processors pipeline. void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } + /// Value for rows_before_group_by_at_least field. + virtual void setRowsBeforeGroupBy(size_t /*rows_before_limit*/) { } + + /// Counter to calculate rows_before_group_by_at_least in processors pipeline. + void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by_counter.swap(counter); } + /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. virtual void onProgress(const Progress & /*progress*/) {} @@ -151,6 +157,7 @@ protected: Progress progress; bool applied_limit = false; size_t rows_before_limit = 0; + size_t rows_before_group_by = 0; Chunk totals; Chunk extremes; }; @@ -185,6 +192,7 @@ protected: bool need_write_suffix = true; RowsBeforeLimitCounterPtr rows_before_limit_counter; + RowsBeforeGroupByCounterPtr rows_before_group_by_counter; Statistics statistics; private: diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 4f6b10dd068..63423628e57 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -45,4 +45,8 @@ void LazyOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) info.setRowsBeforeLimit(rows_before_limit); } +void LazyOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +{ + info.setRowsBeforeGroupBy(rows_before_group_by); +} } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 9cf609ed2d7..83abb2ff1a1 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -28,6 +28,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; + void setRowsBeforeGroupBy(size_t rows_before_group_by) override; void onCancel() override { diff --git a/src/Processors/Formats/PullingOutputFormat.cpp b/src/Processors/Formats/PullingOutputFormat.cpp index b2378e62d34..646755deb6b 100644 --- a/src/Processors/Formats/PullingOutputFormat.cpp +++ b/src/Processors/Formats/PullingOutputFormat.cpp @@ -42,5 +42,8 @@ void PullingOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) { info.setRowsBeforeLimit(rows_before_limit); } - +void PullingOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +{ + info.setRowsBeforeGroupBy(rows_before_group_by); +} } diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index a8efb8dd962..c4d8cf4aab2 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -22,6 +22,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; + void setRowsBeforeGroupBy(size_t rows_before_group_by) override; bool expectMaterializedColumns() const override { return false; } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 63f32d8deb7..0df4b3168e3 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -24,6 +24,8 @@ using StorageLimitsList = std::list; class RowsBeforeLimitCounter; using RowsBeforeLimitCounterPtr = std::shared_ptr; +using RowsBeforeGroupByCounterPtr = std::shared_ptr; + class IProcessor; using ProcessorPtr = std::shared_ptr; using Processors = std::vector; @@ -366,6 +368,10 @@ public: /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} + /// Set rows_before_group_by counter for current processor. + /// This counter is used to calculate the number of rows right before AggregatingTransform. + virtual void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr /* counter */) { } + protected: virtual void onCancel() {} diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 0b2751e18a6..bd100f29a47 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -31,12 +31,14 @@ public: OutputPort * getExtremesPort() { return extremes; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } private: QueryPlanResourceHolder resources; Creator creator; Processors processors; RowsBeforeLimitCounterPtr rows_before_limit; + RowsBeforeLimitCounterPtr rows_before_group_by; /// Outputs for DelayedSource. OutputPort * main = nullptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..a78db630786 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -35,16 +35,25 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation progress(value.read_rows, value.read_bytes); }); - query_executor->setProfileInfoCallback([this](const ProfileInfo & info) - { - if (rows_before_limit) + query_executor->setProfileInfoCallback( + [this](const ProfileInfo & info) { - if (info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); - else - manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit - } - }); + if (rows_before_limit) + { + if (info.hasAppliedLimit()) + rows_before_limit->add(info.getRowsBeforeLimit()); + else + manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit + } + + if (rows_before_group_by) + { + if (info.hasAppliedGroupBy()) + rows_before_group_by->add(info.getRowsBeforeGroupBy()); + else + manually_add_rows_before_group_by_counter = true; /// Remote subquery doesn't contain a group by + } + }); } RemoteSource::~RemoteSource() = default; @@ -162,7 +171,8 @@ std::optional RemoteSource::tryGenerate() { if (manually_add_rows_before_limit_counter) rows_before_limit->add(rows); - + if (manually_add_rows_before_group_by_counter) + rows_before_group_by->add(rows); query_executor->finish(); return {}; } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 052567bc261..b2ea6d50e01 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -26,6 +26,7 @@ public: String getName() const override { return "Remote"; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -45,6 +46,7 @@ private: bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; RowsBeforeLimitCounterPtr rows_before_limit; + RowsBeforeLimitCounterPtr rows_before_group_by; const bool async_read; const bool async_query_sending; @@ -52,6 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; + bool manually_add_rows_before_group_by_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 65f0612d738..7c0e222f89b 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -8,7 +8,6 @@ #include #include #include - #include @@ -684,7 +683,8 @@ void AggregatingTransform::consume(Chunk chunk) LOG_TRACE(log, "Aggregating"); is_consume_started = true; } - + if (rows_before_group_by_at_least) + rows_before_group_by_at_least->add(num_rows); src_rows += num_rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index e167acde067..6e7b04f9191 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -3,11 +3,13 @@ #include #include #include -#include -#include -#include +#include #include #include +#include +#include +#include + namespace CurrentMetrics { @@ -167,6 +169,7 @@ public: Status prepare() override; void work() override; Processors expandPipeline() override; + void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr counter) override { rows_before_group_by_at_least.swap(counter); } protected: void consume(Chunk chunk); @@ -210,6 +213,8 @@ private: bool is_consume_started = false; + RowsBeforeGroupByCounterPtr rows_before_group_by_at_least; + void initGenerate(); }; diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index ee0ff8c69bf..cec179ecfad 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -16,6 +16,8 @@ void ProfileInfo::read(ReadBuffer & in) readBinary(applied_limit, in); readVarUInt(rows_before_limit, in); readBinary(calculated_rows_before_limit, in); + readBinary(applied_group_by, in); + readVarUInt(rows_before_group_by, in); } @@ -27,6 +29,8 @@ void ProfileInfo::write(WriteBuffer & out) const writeBinary(hasAppliedLimit(), out); writeVarUInt(getRowsBeforeLimit(), out); writeBinary(calculated_rows_before_limit, out); + writeBinary(hasAppliedGroupBy(), out); + writeVarUInt(getRowsBeforeGroupBy(), out); } @@ -41,6 +45,8 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) applied_limit = rhs.applied_limit; rows_before_limit = rhs.rows_before_limit; calculated_rows_before_limit = rhs.calculated_rows_before_limit; + applied_group_by = rhs.applied_group_by; + rows_before_group_by = rhs.rows_before_group_by; } @@ -57,6 +63,17 @@ bool ProfileInfo::hasAppliedLimit() const return applied_limit; } +size_t ProfileInfo::getRowsBeforeGroupBy() const +{ + return rows_before_group_by; +} + + +bool ProfileInfo::hasAppliedGroupBy() const +{ + return applied_group_by; +} + void ProfileInfo::update(Block & block) { diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index 7a0a0c304e2..141adc7430d 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -32,6 +32,9 @@ struct ProfileInfo size_t getRowsBeforeLimit() const; bool hasAppliedLimit() const; + size_t getRowsBeforeGroupBy() const; + bool hasAppliedGroupBy() const; + void update(Block & block); void update(size_t num_rows, size_t num_bytes); @@ -51,11 +54,21 @@ struct ProfileInfo rows_before_limit = rows_before_limit_; } + /// Only for Processors. + void setRowsBeforeGroupBy(size_t rows_before_group_by_) + { + applied_group_by = true; + rows_before_group_by = rows_before_group_by_; + } + private: /// For these fields we make accessors, because they must be calculated beforehand. mutable bool applied_limit = false; /// Whether LIMIT was applied mutable size_t rows_before_limit = 0; mutable bool calculated_rows_before_limit = false; /// Whether the field rows_before_limit was calculated + + mutable bool applied_group_by = false; /// Whether GROUP BY was applied + mutable size_t rows_before_group_by = 0; }; } diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 935c006c217..5e0885ed4e8 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -1,15 +1,13 @@ #include #include -#include -#include -#include -#include #include #include -#include -#include -#include +#include +#include +#include +#include +#include #include #include #include @@ -17,15 +15,18 @@ #include #include #include -#include +#include #include +#include #include #include #include #include -#include #include -#include +#include +#include +#include +#include namespace DB @@ -273,7 +274,20 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } } - +static void initRowsBeforeGroupBy(std::shared_ptr processors, IOutputFormat * output_format) +{ + if (!processors->empty()) + { + RowsBeforeGroupByCounterPtr rows_before_group_by_at_least = std::make_shared(); + for (auto & processor : *processors) + { + if (auto transform = std::dynamic_pointer_cast(processor)) + transform->setRowsBeforeGroupByCounter(rows_before_group_by_at_least); + } + rows_before_group_by_at_least->add(0); + output_format->setRowsBeforeLimitCounter(rows_before_group_by_at_least); + } +} QueryPipeline::QueryPipeline( QueryPlanResourceHolder resources_, @@ -521,6 +535,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); + initRowsBeforeGroupBy(processors, format.get()); output_format = format.get(); processors->emplace_back(std::move(format)); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 10b59751b22..37e4342f3b0 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1577,6 +1577,7 @@ namespace stats.set_allocated_bytes(info.bytes); stats.set_applied_limit(info.hasAppliedLimit()); stats.set_rows_before_limit(info.getRowsBeforeLimit()); + stats.set_rows_before_group_by(info.getRowsBeforeGroupBy()); } void Call::addLogsToResult() diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c9ba6f28506..02b6988b8c0 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -179,6 +179,8 @@ message Stats { uint64 allocated_bytes = 3; bool applied_limit = 4; uint64 rows_before_limit = 5; + bool applied_group_by = 6; + uint64 rows_before_group_by = 7; } message Exception { From 096616bd1f5f105a760243dfaec5f4493bccabeb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 Jun 2024 02:28:02 +0000 Subject: [PATCH 0197/1722] introduce mutations snapshot --- src/Interpreters/MutationsInterpreter.cpp | 17 ++- src/Interpreters/MutationsInterpreter.h | 6 +- .../optimizeUseAggregateProjection.cpp | 2 +- .../optimizeUseNormalProjection.cpp | 2 +- .../Optimizations/projectionsCommon.cpp | 9 +- src/Processors/QueryPlan/PartsSplitter.cpp | 1 - .../QueryPlan/ReadFromMergeTree.cpp | 24 ++-- src/Processors/QueryPlan/ReadFromMergeTree.h | 10 +- src/Storages/MergeTree/AlterConversions.cpp | 9 +- src/Storages/MergeTree/AlterConversions.h | 3 +- src/Storages/MergeTree/MergeTask.cpp | 14 ++- src/Storages/MergeTree/MergeTask.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 77 ++++++++---- src/Storages/MergeTree/MergeTreeData.h | 46 +++++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 51 ++------ .../MergeTree/MergeTreeDataSelectExecutor.h | 8 +- .../MergeTree/MergeTreeMutationEntry.cpp | 12 +- .../MergeTree/MergeTreeMutationEntry.h | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 + .../MergeTree/MergeTreePrefetchedReadPool.h | 1 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 + src/Storages/MergeTree/MergeTreeReadPool.h | 1 + .../MergeTree/MergeTreeReadPoolBase.cpp | 6 +- .../MergeTree/MergeTreeReadPoolBase.h | 4 + .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 + .../MergeTree/MergeTreeReadPoolInOrder.h | 1 + .../MergeTreeReadPoolParallelReplicas.cpp | 2 + .../MergeTreeReadPoolParallelReplicas.h | 1 + ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 + ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 21 +++- .../MergeTree/MergeTreeSequentialSource.h | 2 + src/Storages/MergeTree/MutateTask.cpp | 27 ++-- src/Storages/MergeTree/RangesInDataPart.h | 6 +- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 82 ++++++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 24 +++- .../MergeTree/StorageFromMergeTreeDataPart.h | 11 +- src/Storages/StorageMergeTree.cpp | 117 ++++++++++-------- src/Storages/StorageMergeTree.h | 24 +++- src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 42 files changed, 413 insertions(+), 230 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6d3a4f30b34..fc15a20f992 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -145,6 +145,7 @@ ColumnDependencies getAllColumnDependencies( bool isStorageTouchedByMutations( MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextPtr context) @@ -181,7 +182,7 @@ bool isStorageTouchedByMutations( if (all_commands_can_be_skipped) return false; - auto storage_from_part = std::make_shared(source_part); + auto storage_from_part = std::make_shared(source_part, mutations_snapshot); std::optional interpreter_select_query; BlockIO io; @@ -283,8 +284,13 @@ MutationsInterpreter::Source::Source(StoragePtr storage_) : storage(std::move(st { } -MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_) - : data(&storage_), part(std::move(source_part_)) +MutationsInterpreter::Source::Source( + MergeTreeData & storage_, + MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_) + : data(&storage_) + , part(std::move(source_part_)) + , alter_conversions(std::move(alter_conversions_)) { } @@ -384,13 +390,14 @@ MutationsInterpreter::MutationsInterpreter( MutationsInterpreter::MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_, StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, Names available_columns_, ContextPtr context_, Settings settings_) : MutationsInterpreter( - Source(storage_, std::move(source_part_)), + Source(storage_, std::move(source_part_), std::move(alter_conversions_)), std::move(metadata_snapshot_), std::move(commands_), std::move(available_columns_), std::move(context_), std::move(settings_)) { @@ -1210,7 +1217,7 @@ void MutationsInterpreter::Source::read( createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, plan, *data, storage_snapshot, - part, required_columns, + part, alter_conversions, required_columns, apply_deleted_mask_, filter, context_, getLogger("MutationsInterpreter")); } diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6aaa233cda3..8ae438efc19 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -6,6 +6,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" namespace DB @@ -21,6 +22,7 @@ using QueryPipelineBuilderPtr = std::unique_ptr; bool isStorageTouchedByMutations( MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextPtr context @@ -71,6 +73,7 @@ public: MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_, StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, Names available_columns_, @@ -138,7 +141,7 @@ public: bool can_execute_) const; explicit Source(StoragePtr storage_); - Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_); + Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, AlterConversionsPtr alter_conversions_); private: StoragePtr storage; @@ -146,6 +149,7 @@ public: /// Special case for *MergeTree. MergeTreeData * data = nullptr; MergeTreeData::DataPartPtr part; + AlterConversionsPtr alter_conversions; }; private: diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 70327bc95b4..7e69734a7e5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -764,7 +764,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu projection_reading = reader.readFromParts( /* parts = */ {}, - /* alter_conversions = */ {}, + reading->getMutationsSnapshot()->cloneEmpty(), best_candidate->dag->getRequiredColumnsNames(), proj_snapshot, projection_query_info, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 0af3869ccf1..43e60318004 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -199,7 +199,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod auto projection_reading = reader.readFromParts( /*parts=*/ {}, - /*alter_conversions=*/ {}, + reading->getMutationsSnapshot()->cloneEmpty(), required_columns, proj_snapshot, query_info_copy, diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index af1578d6af8..e6939581b9e 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -217,20 +217,15 @@ bool analyzeProjectionCandidate( { MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; - std::vector alter_conversions; + for (const auto & part_with_ranges : parts_with_ranges) { const auto & created_projections = part_with_ranges.data_part->getProjectionParts(); auto it = created_projections.find(candidate.projection->name); if (it != created_projections.end() && !it->second->is_broken) - { projection_parts.push_back(it->second); - } else - { normal_parts.push_back(part_with_ranges.data_part); - alter_conversions.push_back(part_with_ranges.alter_conversions); - } } if (projection_parts.empty()) @@ -255,7 +250,7 @@ bool analyzeProjectionCandidate( if (!normal_parts.empty()) { /// TODO: We can reuse existing analysis_result by filtering out projection parts - auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts), std::move(alter_conversions)); + auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); if (normal_result_ptr->selected_marks != 0) { diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index ed4b1906635..65f1c89f990 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -229,7 +229,6 @@ public: { ranges_in_data_parts.emplace_back( initial_ranges_in_data_parts[part_index].data_part, - initial_ranges_in_data_parts[part_index].alter_conversions, initial_ranges_in_data_parts[part_index].part_index_in_query, MarkRanges{mark_range}); part_index_to_initial_ranges_in_data_parts_index[it->second] = part_index; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index aba3f6ff2da..e958430ff16 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -262,7 +262,7 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c ReadFromMergeTree::ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, - std::vector alter_conversions_, + MergeTreeData::MutationsSnapshotPtr mutations_, Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, @@ -279,7 +279,7 @@ ReadFromMergeTree::ReadFromMergeTree( query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) , prepared_parts(std::move(parts_)) - , alter_conversions_for_parts(std::move(alter_conversions_)) + , mutations_snapshot(std::move(mutations_)) , all_column_names(std::move(all_column_names_)) , data(data_) , actions_settings(ExpressionActionsSettings::fromContext(context_)) @@ -361,6 +361,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( auto pool = std::make_shared( std::move(extension), std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -442,6 +443,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -455,6 +457,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -535,6 +538,7 @@ Pipe ReadFromMergeTree::readInOrder( std::move(extension), mode, parts_with_ranges, + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -550,6 +554,7 @@ Pipe ReadFromMergeTree::readInOrder( has_limit_below_one_block, read_type, parts_with_ranges, + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -1016,7 +1021,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( } ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part)); + new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); } splitted_parts_and_ranges.emplace_back(std::move(new_parts)); @@ -1243,7 +1248,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( RangesInDataParts new_parts; for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) - new_parts.emplace_back(part_it->data_part, part_it->alter_conversions, part_it->part_index_in_query, part_it->ranges); + new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); if (new_parts.empty()) continue; @@ -1356,15 +1361,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(bool find_exact_ranges) const { - return selectRangesToRead(prepared_parts, alter_conversions_for_parts, find_exact_ranges); + return selectRangesToRead(prepared_parts, find_exact_ranges); } -ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( - MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges) const +ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges) const { return selectRangesToRead( std::move(parts), - std::move(alter_conversions), metadata_for_reading, query_info, context, @@ -1534,7 +1537,6 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, ContextPtr context_, @@ -1596,10 +1598,9 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( { MergeTreeDataSelectExecutor::filterPartsByPartition( + parts, indexes->partition_pruner, indexes->minmax_idx_condition, - parts, - alter_conversions, indexes->part_values, metadata_snapshot, data, @@ -1628,7 +1629,6 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( auto reader_settings = getMergeTreeReaderSettings(context_, query_info_); result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), - std::move(alter_conversions), metadata_snapshot, context_, indexes->key_condition, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index caa8aa2e1bd..57e19441b82 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -110,7 +110,7 @@ public: ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, - std::vector alter_conversions_, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot_, Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, @@ -154,7 +154,6 @@ public: static AnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, @@ -166,8 +165,7 @@ public: std::optional & indexes, bool find_exact_ranges); - AnalysisResultPtr selectRangesToRead( - MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges = false) const; + AnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges = false) const; AnalysisResultPtr selectRangesToRead(bool find_exact_ranges = false) const; @@ -188,7 +186,7 @@ public: void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } - const std::vector & getAlterConvertionsForParts() const { return alter_conversions_for_parts; } + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot() const { return mutations_snapshot; } const MergeTreeData & getMergeTreeData() const { return data; } size_t getMaxBlockSize() const { return block_size.max_block_size_rows; } @@ -209,7 +207,7 @@ private: MergeTreeReaderSettings reader_settings; MergeTreeData::DataPartsVector prepared_parts; - std::vector alter_conversions_for_parts; + MergeTreeData::MutationsSnapshotPtr mutations_snapshot; Names all_column_names; diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp index 31f8f17e2c1..82bef500b34 100644 --- a/src/Storages/MergeTree/AlterConversions.cpp +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -9,9 +9,14 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -bool AlterConversions::supportsMutationCommandType(MutationCommand::Type t) +bool AlterConversions::isSupportedDataMutation(MutationCommand::Type) { - return t == MutationCommand::Type::RENAME_COLUMN; + return false; +} + +bool AlterConversions::isSupportedMetadataMutation(MutationCommand::Type type) +{ + return type == MutationCommand::Type::RENAME_COLUMN; } void AlterConversions::addMutationCommand(const MutationCommand & command) diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index 0f857d351dd..68966f88f84 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -35,7 +35,8 @@ public: /// Get column old name before rename (lookup by key in rename_map) std::string getColumnOldName(const std::string & new_name) const; - static bool supportsMutationCommandType(MutationCommand::Type); + static bool isSupportedDataMutation(MutationCommand::Type type); + static bool isSupportedMetadataMutation(MutationCommand::Type type); private: /// Rename map new_name -> old_name. diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7ab8fa2430a..5dab0cd0c08 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -257,6 +257,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (enabledBlockOffsetColumn(global_ctx)) addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); + auto mutations_snapshot = global_ctx->data->getMutationsSnapshot( + global_ctx->metadata_snapshot->getMetadataVersion(), + /*need_data_mutations=*/ false); + SerializationInfo::Settings info_settings = { .ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, @@ -264,10 +268,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() }; SerializationInfoByName infos(global_ctx->storage_columns, info_settings); + global_ctx->alter_conversions.reserve(global_ctx->future_part->parts.size()); for (const auto & part : global_ctx->future_part->parts) { global_ctx->new_data_part->ttl_infos.update(part->ttl_infos); + if (global_ctx->metadata_snapshot->hasAnyTTL() && !part->checkAllTTLCalculated(global_ctx->metadata_snapshot)) { LOG_INFO(ctx->log, "Some TTL values were not calculated for part {}. Will calculate them forcefully during merge.", part->name); @@ -288,6 +294,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part_infos); } + + global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot)); } const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; @@ -604,6 +612,7 @@ Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & *global_ctx->data, global_ctx->storage_snapshot, global_ctx->future_part->parts[part_num], + global_ctx->alter_conversions[part_num], Names{column_name}, /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, @@ -996,13 +1005,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); - for (const auto & part : global_ctx->future_part->parts) + for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) { Pipe pipe = createMergeTreeSequentialSource( MergeTreeSequentialSourceType::Merge, *global_ctx->data, global_ctx->storage_snapshot, - part, + global_ctx->future_part->parts[i], + global_ctx->alter_conversions[i], global_ctx->merging_columns.getNames(), /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 56909d1b7a0..c394a47aff0 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -4,6 +4,7 @@ #include #include +#include "Storages/MergeTree/AlterConversions.h" #include #include @@ -154,6 +155,7 @@ private: StorageSnapshotPtr storage_snapshot{nullptr}; StorageMetadataPtr metadata_snapshot{nullptr}; FutureMergedMutatedPartPtr future_part{nullptr}; + std::vector alter_conversions; /// This will be either nullptr or new_data_part, so raw pointer is ok. IMergeTreeDataPart * parent_part{nullptr}; ContextPtr context{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f9cc65871fe..98c308f5fd1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8115,11 +8115,13 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S return true; } -AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartPtr part) const +AlterConversionsPtr MergeTreeData::getAlterConversionsForPart( + const MergeTreeDataPartPtr & part, + const MutationsSnapshotPtr & snapshot) { - auto commands = getAlterMutationCommandsForPart(part); - + auto commands = snapshot->getAlterMutationCommandsForPart(part); auto result = std::make_shared(); + for (const auto & command : commands | std::views::reverse) result->addMutationCommand(command); @@ -8427,9 +8429,9 @@ StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & object_columns_copy = object_columns; } - snapshot_data->alter_conversions.reserve(snapshot_data->parts.size()); - for (const auto & part : snapshot_data->parts) - snapshot_data->alter_conversions.push_back(getAlterConversionsForPart(part)); + snapshot_data->mutations_snapshot = getMutationsSnapshot( + metadata_snapshot->getMetadataVersion(), + query_context->getSettingsRef().apply_mutations_on_fly); return std::make_shared(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data)); } @@ -8616,28 +8618,59 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic & alter_conversions_mutations, bool remove) +static void updateMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + Int64 increment) { + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + + bool has_data_mutation = false; + bool has_metadata_mutation = false; + for (const auto & command : commands) { - if (AlterConversions::supportsMutationCommandType(command.type)) + if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - if (remove) - { - --alter_conversions_mutations; - if (alter_conversions_mutations < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations); - } - else - { - if (alter_conversions_mutations < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations); - ++alter_conversions_mutations; - } - return true; + data_mutations_to_apply += increment; + has_data_mutation = true; + + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + } + + if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) + { + metadata_mutations_to_apply += increment; + has_metadata_mutation = true; + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); } } - return false; +} + +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) +{ + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); +} + +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) +{ + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c6f736a4afd..765b68b7559 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -35,6 +35,7 @@ #include #include #include +#include "Storages/ProjectionsDescription.h" #include #include @@ -445,12 +446,27 @@ public: bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; + struct IMutationsSnapshot + { + /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly + /// (i.e. when reading from the part). Mutations not supported by AlterConversions + /// (supportsMutationCommandType()) can be omitted. + /// + /// @return list of mutations, in *reverse* order (newest to oldest) + virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; + virtual std::shared_ptr cloneEmpty() const = 0; + + virtual ~IMutationsSnapshot() = default; + }; + + using MutationsSnapshotPtr = std::shared_ptr; + /// Snapshot for MergeTree contains the current set of data parts - /// at the moment of the start of query. + /// and mutations required to be applied at the moment of the start of query. struct SnapshotData : public StorageSnapshot::Data { DataPartsVector parts; - std::vector alter_conversions; + MutationsSnapshotPtr mutations_snapshot; }; StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; @@ -934,8 +950,13 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } + /// TODO: comment + virtual MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const = 0; + /// Return alter conversions for part which must be applied on fly. - AlterConversionsPtr getAlterConversionsForPart(MergeTreeDataPartPtr part) const; + static AlterConversionsPtr getAlterConversionsForPart( + const MergeTreeDataPartPtr & part, + const MutationsSnapshotPtr & snapshot); /// Returns destination disk or volume for the TTL rule according to current storage policy. SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const; @@ -1448,13 +1469,6 @@ protected: /// mechanisms for parts locking virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0; - /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly - /// (i.e. when reading from the part). Mutations not supported by AlterConversions - /// (supportsMutationCommandType()) can be omitted. - /// - /// @return list of mutations, in *reverse* order (newest to oldest) - virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; - struct PartBackupEntries { String part_name; @@ -1738,6 +1752,16 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated -bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic & alter_conversions_mutations, bool remove); +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); + +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2e287ff3042..0ad7bd47648 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -130,12 +130,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( bool enable_parallel_reading) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; - const auto & alter_conversions = snapshot_data.alter_conversions; auto step = readFromParts( - parts, - alter_conversions, + snapshot_data.parts, + snapshot_data.mutations_snapshot, column_names_to_return, storage_snapshot, query_info, @@ -491,10 +489,9 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, const std::optional & partition_pruner, const std::optional & minmax_idx_condition, - MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -503,8 +500,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( LoggerPtr log, ReadFromMergeTree::IndexStats & index_stats) { - chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); - const Settings & settings = context->getSettingsRef(); DataTypes minmax_columns_types; @@ -528,7 +523,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( parts, - alter_conversions, part_values, data.getPinnedPartUUIDs(), minmax_idx_condition, @@ -541,7 +535,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( else selectPartsToRead( parts, - alter_conversions, part_values, minmax_idx_condition, minmax_columns_types, @@ -580,7 +573,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, - std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const ContextPtr & context, const KeyCondition & key_condition, @@ -593,8 +585,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd bool use_skip_indexes, bool find_exact_ranges) { - chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); - RangesInDataParts parts_with_ranges; parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); @@ -653,11 +643,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd auto process_part = [&](size_t part_index) { auto & part = parts[part_index]; - auto alter_conversions_for_part = !alter_conversions.empty() - ? alter_conversions[part_index] - : std::make_shared(); - RangesInDataPart ranges(part, alter_conversions_for_part, part_index); + RangesInDataPart ranges(part, part_index); size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); if (metadata_snapshot->hasPrimaryKey() || part_offset_condition) @@ -907,11 +894,11 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return std::make_shared(); std::optional indexes; - /// NOTE: We don't need alter_conversions because the returned analysis_result is only used for: - /// 1. estimate the number of rows to read; 2. projection reading, which doesn't have alter_conversions. + /// NOTE: We don't need mutations snapshot because the returned analysis_result is only used for: + /// 1. estimate the number of rows to read; + /// 2. projection reading, which doesn't have alter conversions. return ReadFromMergeTree::selectRangesToRead( std::move(parts), - /*alter_conversions=*/{}, metadata_snapshot, query_info, context, @@ -926,7 +913,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -948,7 +935,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( return std::make_unique( std::move(parts), - std::move(alter_conversions), + std::move(mutations_snapshot), column_names_to_return, data, query_info, @@ -1546,7 +1533,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -1555,10 +1541,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( PartFilterCounters & counters) { MergeTreeData::DataPartsVector prev_parts; - std::vector prev_conversions; - std::swap(prev_parts, parts); - std::swap(prev_conversions, alter_conversions); for (size_t i = 0; i < prev_parts.size(); ++i) { @@ -1600,14 +1583,11 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( counters.num_granules_after_partition_pruner += num_granules; parts.push_back(prev_parts[i]); - if (!prev_conversions.empty()) - alter_conversions.push_back(prev_conversions[i]); } } void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -1620,18 +1600,13 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( { /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met - auto select_parts = [&] ( - MergeTreeData::DataPartsVector & selected_parts, - std::vector & selected_conversions) -> bool + auto select_parts = [&](MergeTreeData::DataPartsVector & selected_parts) -> bool { auto ignored_part_uuids = query_context->getIgnoredPartUUIDs(); std::unordered_set temp_part_uuids; MergeTreeData::DataPartsVector prev_parts; - std::vector prev_conversions; - std::swap(prev_parts, selected_parts); - std::swap(prev_conversions, selected_conversions); for (size_t i = 0; i < prev_parts.size(); ++i) { @@ -1686,8 +1661,6 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( } selected_parts.push_back(prev_parts[i]); - if (!prev_conversions.empty()) - selected_conversions.push_back(prev_conversions[i]); } if (!temp_part_uuids.empty()) @@ -1706,7 +1679,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( }; /// Process parts that have to be read for a query. - auto needs_retry = !select_parts(parts, alter_conversions); + auto needs_retry = !select_parts(parts); /// If any duplicated part UUIDs met during the first step, try to ignore them in second pass. /// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage. @@ -1717,7 +1690,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( counters = PartFilterCounters(); /// Second attempt didn't help, throw an exception - if (!select_parts(parts, alter_conversions)) + if (!select_parts(parts)) throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate UUIDs while processing query."); } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 788355c1e59..0d02456e480 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -40,7 +40,7 @@ public: /// The same as read, but with specified set of parts. QueryPlanStepPtr readFromParts( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -120,7 +120,6 @@ private: /// as well as `max_block_number_to_read`. static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -131,7 +130,6 @@ private: /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. static void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -175,10 +173,9 @@ public: /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, const std::optional & partition_pruner, const std::optional & minmax_idx_condition, - MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -192,7 +189,6 @@ public: /// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings. static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, - std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const ContextPtr & context, const KeyCondition & key_condition, diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 4dbccb91620..6f06b921031 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "Storages/MutationCommands.h" #include @@ -50,7 +51,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings) : create_time(time(nullptr)) - , commands(std::move(commands_)) + , commands(std::make_shared(std::move(commands_))) , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") @@ -63,7 +64,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n"; *out << "commands: "; - commands.writeText(*out, /* with_pure_metadata_commands = */ false); + commands->writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; if (tid.isPrehistoric()) { @@ -116,7 +117,8 @@ void MergeTreeMutationEntry::writeCSN(CSN csn_) } MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_) - : disk(std::move(disk_)) + : commands(std::make_shared()) + , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name(file_name_) , is_temp(false) @@ -133,7 +135,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); *buf >> "commands: "; - commands.readText(*buf); + commands->readText(*buf); *buf >> "\n"; if (buf->eof()) @@ -177,7 +179,7 @@ std::shared_ptr MergeTreeMutationEntry::backup() const out << "block number: " << block_number << "\n"; out << "commands: "; - commands.writeText(out, /* with_pure_metadata_commands = */ false); + commands->writeText(out, /* with_pure_metadata_commands = */ false); out << "\n"; return std::make_shared(out.str()); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 04297f2852a..f41ad2a17f8 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -16,7 +16,7 @@ class IBackupEntry; struct MergeTreeMutationEntry { time_t create_time = 0; - MutationCommands commands; + std::shared_ptr commands; DiskPtr disk; String path_prefix; diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 2c249f7b63b..2aaf06fde7f 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -84,6 +84,7 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get() MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -94,6 +95,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a3a57227630..65a7d62ad2d 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -19,6 +19,7 @@ class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithCo public: MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index dc1ba030f45..d45e2e9c578 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -35,6 +35,7 @@ size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & column MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -45,6 +46,7 @@ MergeTreeReadPool::MergeTreeReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index cb0e8a9657f..d7b354a2799 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -26,6 +26,7 @@ public: MergeTreeReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 0ea19370d45..2935890cba5 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes MergeTreeReadPoolBase::MergeTreeReadPoolBase( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const PoolSettings & pool_settings_, const ContextPtr & context_) : parts_ranges(std::move(parts_)) + , mutations_snapshot(std::move(mutations_snapshot_)) , shared_virtual_fields(std::move(shared_virtual_fields_)) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) @@ -67,9 +69,9 @@ void MergeTreeReadPoolBase::fillPerPartInfos() } read_task_info.part_index_in_query = part_with_ranges.part_index_in_query; - read_task_info.alter_conversions = part_with_ranges.alter_conversions; + read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot); - LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions); + LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, read_task_info.alter_conversions); read_task_info.task_columns = getReadTaskColumns( part_info, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 1b5bfec5898..8286ff52a5c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -9,6 +9,8 @@ namespace DB class MergeTreeReadPoolBase : public IMergeTreeReadPool { public: + using MutationsSnapshotPtr = MergeTreeData::MutationsSnapshotPtr; + struct PoolSettings { size_t threads = 0; @@ -23,6 +25,7 @@ public: MergeTreeReadPoolBase( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -37,6 +40,7 @@ public: protected: /// Initialized in constructor const RangesInDataParts parts_ranges; + const MutationsSnapshotPtr mutations_snapshot; const VirtualFields shared_virtual_fields; const StorageSnapshotPtr storage_snapshot; const PrewhereInfoPtr prewhere_info; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 4c0391ffa57..60f127acdae 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index 9fedf396a6b..a3668acb170 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -11,6 +11,7 @@ public: bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 38035d97f56..0d615fae443 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -23,6 +24,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index ca159edb91c..d9d628b8be2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -11,6 +11,7 @@ public: MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 01c0a9f91be..b1f9ffc8ea4 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 4fe3f7a699c..7c549ed3c4a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -12,6 +12,7 @@ public: ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 02f8d6f4f6a..bd4aa066dfd 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" #include namespace DB @@ -38,6 +39,7 @@ public: const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, std::optional mark_ranges_, bool apply_deleted_mask, @@ -62,6 +64,9 @@ private: /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; + /// TODO: comment. + AlterConversionsPtr alter_conversions; + /// Columns we have to read (each Block from read will contain them) Names columns_to_read; @@ -91,6 +96,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, std::optional mark_ranges_, bool apply_deleted_mask, @@ -100,6 +106,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( , storage(storage_) , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) , read_with_direct_io(read_with_direct_io_) , mark_ranges(std::move(mark_ranges_)) @@ -113,8 +120,6 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( LOG_DEBUG(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part", data_part->getMarksCount(), data_part->name, data_part->rows_count); - auto alter_conversions = storage.getAlterConversionsForPart(data_part); - /// Note, that we don't check setting collaborate_with_coordinator presence, because this source /// is only used in background merges. addTotalRowsApprox(data_part->rows_count); @@ -300,6 +305,7 @@ Pipe createMergeTreeSequentialSource( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, std::optional mark_ranges, std::shared_ptr> filtered_rows_count, @@ -316,7 +322,8 @@ Pipe createMergeTreeSequentialSource( columns_to_read.emplace_back(RowExistsColumn::name); auto column_part_source = std::make_shared(type, - storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), + storage, storage_snapshot, data_part, alter_conversions, + columns_to_read, std::move(mark_ranges), /*apply_deleted_mask=*/ false, read_with_direct_io, prefetch); Pipe pipe(std::move(column_part_source)); @@ -347,6 +354,7 @@ public: const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, bool apply_deleted_mask_, ActionsDAGPtr filter_, @@ -357,6 +365,7 @@ public: , storage(storage_) , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) , apply_deleted_mask(apply_deleted_mask_) , filter(std::move(filter_)) @@ -400,6 +409,7 @@ public: storage, storage_snapshot, data_part, + alter_conversions, columns_to_read, std::move(mark_ranges), /*filtered_rows_count=*/ nullptr, @@ -415,6 +425,7 @@ private: const MergeTreeData & storage; StorageSnapshotPtr storage_snapshot; MergeTreeData::DataPartPtr data_part; + AlterConversionsPtr alter_conversions; Names columns_to_read; bool apply_deleted_mask; ActionsDAGPtr filter; @@ -428,6 +439,7 @@ void createReadFromPartStep( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, bool apply_deleted_mask, ActionsDAGPtr filter, @@ -435,7 +447,8 @@ void createReadFromPartStep( LoggerPtr log) { auto reading = std::make_unique(type, - storage, storage_snapshot, std::move(data_part), + storage, storage_snapshot, + std::move(data_part), std::move(alter_conversions), std::move(columns_to_read), apply_deleted_mask, filter, std::move(context), log); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index e6f055f776c..1ecc721d4a8 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -21,6 +21,7 @@ Pipe createMergeTreeSequentialSource( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, std::optional mark_ranges, std::shared_ptr> filtered_rows_count, @@ -36,6 +37,7 @@ void createReadFromPartStep( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, bool apply_deleted_mask, ActionsDAGPtr filter, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a552ee89aee..3c4ef44dbd8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -29,6 +29,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" #include @@ -104,6 +105,7 @@ static UInt64 getExistingRowsCount(const Block & block) static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, StorageMetadataPtr metadata_snapshot, + AlterConversionsPtr alter_conversions, const MutationCommands & commands, MutationCommands & for_interpreter, MutationCommands & for_file_renames, @@ -169,8 +171,6 @@ static void splitAndModifyMutationCommands( } - auto alter_conversions = part->storage.getAlterConversionsForPart(part); - /// We don't add renames from commands, instead we take them from rename_map. /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter @@ -286,7 +286,6 @@ static void splitAndModifyMutationCommands( } } - auto alter_conversions = part->storage.getAlterConversionsForPart(part); /// We don't add renames from commands, instead we take them from rename_map. /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter @@ -2119,6 +2118,14 @@ bool MutateTask::prepare() ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); + auto mutations_snapshot = ctx->data->getMutationsSnapshot( + ctx->metadata_snapshot->getMetadataVersion(), + /*need_data_mutations=*/ false); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + ctx->source_part, + mutations_snapshot); + auto context_for_reading = Context::createCopy(ctx->context); /// Allow mutations to work when force_index_by_date or force_primary_key is on. @@ -2133,7 +2140,7 @@ bool MutateTask::prepare() ctx->commands_for_part.emplace_back(command); if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) + *ctx->data, ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) { NameSet files_to_copy_instead_of_hardlinks; auto settings_ptr = ctx->data->getSettings(); @@ -2192,8 +2199,13 @@ bool MutateTask::prepare() context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); MutationHelpers::splitAndModifyMutationCommands( - ctx->source_part, ctx->metadata_snapshot, - ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames, ctx->log); + ctx->source_part, + ctx->metadata_snapshot, + alter_conversions, + ctx->commands_for_part, + ctx->for_interpreter, + ctx->for_file_renames, + ctx->log); ctx->stage_progress = std::make_unique(1.0); @@ -2205,7 +2217,8 @@ bool MutateTask::prepare() settings.apply_deleted_mask = false; ctx->interpreter = std::make_unique( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, + *ctx->data, ctx->source_part, alter_conversions, + ctx->metadata_snapshot, ctx->for_interpreter, ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); diff --git a/src/Storages/MergeTree/RangesInDataPart.h b/src/Storages/MergeTree/RangesInDataPart.h index bf9e4c7dfb2..966637d0812 100644 --- a/src/Storages/MergeTree/RangesInDataPart.h +++ b/src/Storages/MergeTree/RangesInDataPart.h @@ -42,7 +42,6 @@ struct RangesInDataPartsDescription: public std::deque #include #include +#include +#include #include #include @@ -949,7 +951,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, lock); } else it = mutations_by_znode.erase(it); @@ -1001,7 +1003,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)) .first->second; - updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ false); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) { @@ -1075,7 +1077,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// updateAlterConversionsMutations() will be called in updateMutations() + /// decrementMutationsCounters() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1899,25 +1901,15 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk return ReplicatedMergeTreeMergePredicate(*this, zookeeper, std::move(partition_ids_hint)); } - -MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const +MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { - int32_t part_metadata_version = part->getMetadataVersion(); - int32_t metadata_version = storage.getInMemoryMetadataPtr()->getMetadataVersion(); - - chassert(alter_conversions_mutations >= 0); - /// NOTE: that just checking part_metadata_version is not enough, since we - /// need to check for non-metadata mutations as well. - if (alter_conversions_mutations == 0 && metadata_version == part_metadata_version) - return {}; - - std::unique_lock lock(state_mutex); - auto in_partition = mutations_by_partition.find(part->info.partition_id); if (in_partition == mutations_by_partition.end()) return {}; Int64 part_data_version = part->info.getDataVersion(); + int32_t part_metadata_version = part->getMetadataVersion(); + MutationCommands result; bool seen_all_data_mutations = false; @@ -1926,20 +1918,22 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const auto add_to_result = [&](const ReplicatedMergeTreeMutationEntryPtr & entry) { for (const auto & command : entry->commands | std::views::reverse) - if (AlterConversions::supportsMutationCommandType(command.type)) - result.emplace_back(command); + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + result.push_back(command); + else if (AlterConversions::isSupportedMetadataMutation(command.type)) + result.push_back(command); + } }; /// Here we return mutation commands for part which has bigger alter version than part metadata version. /// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions /// of part's metadata. - for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse) + for (const auto & [mutation_version, entry] : in_partition->second | std::views::reverse) { if (seen_all_data_mutations && seen_all_metadata_mutations) break; - auto & entry = mutation_status->entry; - auto alter_version = entry->alter_version; if (alter_version != -1) { @@ -1964,6 +1958,48 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const return result; } +MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +{ + auto res = std::make_shared(); + res->metadata_version = metadata_version; + res->need_data_mutations = need_data_mutations; + + std::lock_guard lock(state_mutex); + + bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; + bool have_metadata_mutations = metadata_mutations_to_apply > 0; + + if (!have_data_mutations && !have_metadata_mutations) + return res; + + for (const auto & [partition_id, mutations] : mutations_by_partition) + { + auto & in_partition = res->mutations_by_partition[partition_id]; + + for (const auto & [version, status] : mutations | std::views::reverse) + { + if (status->is_done) + break; + + bool has_required_command = std::ranges::any_of(status->entry->commands, [&](const auto & command) + { + if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }); + + if (has_required_command) + in_partition.emplace(version, status->entry); + } + } + + return res; +} + MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const { @@ -2044,7 +2080,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - updateAlterConversionsMutations(mutation.entry->commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2101,7 +2137,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 89ef6240558..f9d5487ee3f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -151,8 +152,11 @@ private: /// Mapping from znode path to Mutations Status std::map mutations_by_znode; - /// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart()) - std::atomic alter_conversions_mutations = 0; + + /// Unfinished mutations that are required for AlterConversions. + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; + /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; /// Znode ID of the latest mutation that is done. @@ -409,10 +413,24 @@ public: MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const; + struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot + { + MutationsSnapshot() = default; + + Int64 metadata_version = -1; + bool need_data_mutations = false; + + using MutationsByPartititon = std::unordered_map>; + MutationsByPartititon mutations_by_partition; + + MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; + std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + }; + /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, /// without actual data modification on disk. - MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const; + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const; /// Mark finished mutations as done. If the function needs to be called again at some later time /// (because some mutations are probably done but we are not sure yet), returns true. diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index a94508ad41f..f871157c2c9 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -26,10 +26,12 @@ class StorageFromMergeTreeDataPart final : public IStorage { public: /// Used in part mutation. - explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) + explicit StorageFromMergeTreeDataPart( + const MergeTreeData::DataPartPtr & part_, + const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot_) : IStorage(getIDFromPart(part_)) , parts({part_}) - , alter_conversions({part_->storage.getAlterConversionsForPart(part_)}) + , mutations_snapshot(mutations_snapshot_) , storage(part_->storage) , partition_id(part_->info.partition_id) { @@ -71,10 +73,11 @@ public: size_t max_block_size, size_t num_streams) override { + /// TODO: fix query_plan.addStep(MergeTreeDataSelectExecutor(storage) .readFromParts( parts, - alter_conversions, + mutations_snapshot, column_names, storage_snapshot, query_info, @@ -121,7 +124,7 @@ public: private: const MergeTreeData::DataPartsVector parts; - const std::vector alter_conversions; + const MergeTreeData::MutationsSnapshotPtr mutations_snapshot; const MergeTreeData & storage; const String partition_id; const ReadFromMergeTree::AnalysisResultPtr analysis_result_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9255ee00340..636d2ba5d53 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -498,18 +498,11 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (txn) txn->addMutation(shared_from_this(), mutation_id); - bool alter_conversions_mutations_updated = updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); - bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; + auto [it, inserted] = current_mutations_by_version.try_emplace(version, std::move(entry)); if (!inserted) - { - if (alter_conversions_mutations_updated) - { - --alter_conversions_mutations; - chassert(alter_conversions_mutations >= 0); - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - } + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -545,7 +538,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - updateAlterConversionsMutations(it->second.commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); } } else @@ -744,17 +737,15 @@ std::map StorageMergeTree::getUnfinishedMutationC std::map result; - for (const auto & kv : current_mutations_by_version) + for (const auto & [mutation_version, entry] : current_mutations_by_version) { - Int64 mutation_version = kv.first; - const MergeTreeMutationEntry & entry = kv.second; - const PartVersionWithName needle{mutation_version, ""}; + const PartVersionWithName needle{static_cast(mutation_version), ""}; auto versions_it = std::lower_bound( part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator); size_t parts_to_do = versions_it - part_versions_with_names.begin(); if (parts_to_do > 0) - result.emplace(entry.file_name, entry.commands); + result.emplace(entry.file_name, *entry.commands); } return result; } @@ -787,7 +778,7 @@ std::vector StorageMergeTree::getMutationsStatus() cons std::map block_numbers_map({{"", entry.block_number}}); - for (const MutationCommand & command : entry.commands) + for (const MutationCommand & command : *entry.commands) { WriteBufferFromOwnString buf; formatAST(*command.ast, buf, false, true); @@ -824,20 +815,15 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) auto it = current_mutations_by_version.find(mutation_version); if (it != current_mutations_by_version.end()) { - bool mutation_finished = true; if (std::optional min_version = getMinPartDataVersion()) - mutation_finished = *min_version > static_cast(mutation_version); + { + bool mutation_finished = *min_version > static_cast(mutation_version); + if (!mutation_finished) + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + } to_kill.emplace(std::move(it->second)); - - if (!mutation_finished) - { - const auto commands = it->second.commands; - current_mutations_by_version.erase(it); - updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true); - } - else - current_mutations_by_version.erase(it); + current_mutations_by_version.erase(it); } } @@ -885,6 +871,8 @@ void StorageMergeTree::loadDeduplicationLog() void StorageMergeTree::loadMutations() { + std::lock_guard lock(currently_processing_in_background_mutex); + for (const auto & disk : getDisks()) { for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) @@ -893,7 +881,7 @@ void StorageMergeTree::loadMutations() { MergeTreeMutationEntry entry(disk, relative_data_path, it->name()); UInt64 block_number = entry.block_number; - LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); + LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands->size()); if (!entry.tid.isPrehistoric() && !entry.csn) { @@ -912,10 +900,11 @@ void StorageMergeTree::loadMutations() } } - auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; + auto [entry_it, inserted] = current_mutations_by_version.try_emplace(block_number, std::move(entry)); if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); + + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -1264,7 +1253,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( size_t commands_size = 0; MutationCommands commands_for_size_validation; - for (const auto & command : it->second.commands) + for (const auto & command : *it->second.commands) { if (command.type != MutationCommand::Type::DROP_COLUMN && command.type != MutationCommand::Type::DROP_INDEX @@ -1308,11 +1297,11 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( const auto & single_mutation_commands = it->second.commands; - if (single_mutation_commands.containBarrierCommand()) + if (single_mutation_commands->containBarrierCommand()) { if (commands->empty()) { - commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end()); last_mutation_to_apply = it; } break; @@ -1320,7 +1309,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( else { current_ast_elements += commands_size; - commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end()); last_mutation_to_apply = it; } @@ -2431,34 +2420,62 @@ void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts) } } - -MutationCommands StorageMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const +MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsForPart(const DataPartPtr & part) const { - /// NOTE: there is no need to check part metadata_version, since - /// ALTER_METADATA cannot be done asynchronously, like in - /// ReplicatedMergeTree. - chassert(alter_conversions_mutations >= 0); - if (alter_conversions_mutations == 0) - return {}; - - std::lock_guard lock(currently_processing_in_background_mutex); - - UInt64 part_data_version = part->info.getDataVersion(); MutationCommands result; + UInt64 part_data_version = part->info.getDataVersion(); - for (const auto & [mutation_version, entry] : current_mutations_by_version | std::views::reverse) + for (const auto & [mutation_version, commands] : mutations_by_version | std::views::reverse) { if (mutation_version <= part_data_version) break; - for (const auto & command : entry.commands | std::views::reverse) - if (AlterConversions::supportsMutationCommandType(command.type)) - result.emplace_back(command); + for (const auto & command : *commands | std::views::reverse) + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + result.push_back(command); + else if (AlterConversions::isSupportedMetadataMutation(command.type)) + result.push_back(command); + } } return result; } +MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +{ + auto res = std::make_shared(); + res->metadata_version = metadata_version; + res->need_data_mutations = need_data_mutations; + + std::lock_guard lock(currently_processing_in_background_mutex); + + bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; + bool have_metadata_mutations = metadata_mutations_to_apply > 0; + + if (!have_data_mutations && !have_metadata_mutations) + return res; + + for (const auto & [version, entry] : current_mutations_by_version) + { + bool has_required_command = std::ranges::any_of(*entry.commands, [&](const auto & command) + { + if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }); + + if (has_required_command) + res->mutations_by_version.emplace(version, entry.commands); + } + + return res; +} + void StorageMergeTree::startBackgroundMovesIfNeeded() { if (areBackgroundMovesNeeded()) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 4d819508934..1f2af8b9571 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,6 +17,7 @@ #include #include +#include "Storages/MutationCommands.h" namespace DB @@ -147,8 +148,10 @@ private: DataParts currently_merging_mutating_parts; std::map current_mutations_by_version; - /// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart()) - std::atomic alter_conversions_mutations = 0; + + /// Unfinished mutations that are required for AlterConversions. + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -309,8 +312,21 @@ private: ContextPtr context; }; -protected: - MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; + struct MutationsSnapshot : public IMutationsSnapshot + { + MutationsSnapshot() = default; + + Int64 metadata_version = -1; + bool need_data_mutations = false; + + using MutationsByVersion = std::map>; + MutationsByVersion mutations_by_version; + + MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; + std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + }; + + MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a127384c03c..e2bba1e8068 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9149,13 +9149,11 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } - -MutationCommands StorageReplicatedMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const +MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const { - return queue.getAlterMutationCommandsForPart(part); + return queue.getMutationsSnapshot(metadata_version, need_data_mutations); } - void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { if (areBackgroundMovesNeeded()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index f96206ce657..3ef367d09ce 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -932,7 +932,7 @@ private: void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; + MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; void startBackgroundMovesIfNeeded() override; From d7b3c3e8a97835c9f7987a5852ef9469770f8560 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Jun 2024 21:59:04 +0200 Subject: [PATCH 0198/1722] Add review suggestion --- src/Interpreters/InterpreterSetQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 15d4ba56d8d..2ae35c4313b 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -46,7 +46,7 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel // It is flattened later, when we process UNION ALL/DISTINCT. const auto * last_select = children.back()->as(); if (last_select && last_select->settings()) - InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(false); + InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(/* ignore_setting_constraints= */ false); } void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_) @@ -58,7 +58,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (const auto * query_with_output = dynamic_cast(ast.get())) { if (query_with_output->settings_ast) - InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(false); + InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); if (const auto * create_query = ast->as(); create_query && create_query->select) applySettingsFromSelectWithUnion(create_query->select->as(), context_); @@ -67,7 +67,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (const auto * select_query = ast->as()) { if (auto new_settings = select_query->settings()) - InterpreterSetQuery(new_settings, context_).executeForCurrentContext(false); + InterpreterSetQuery(new_settings, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); } else if (const auto * select_with_union_query = ast->as()) { @@ -76,7 +76,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta else if (const auto * explain_query = ast->as()) { if (explain_query->settings_ast) - InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(false); + InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } @@ -84,7 +84,7 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta { context_->setInsertFormat(insert_query->format); if (insert_query->settings_ast) - InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(false); + InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); } } From 9e33e3dd4dd3c1aad3de32c382da15cfe1bb4917 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jun 2024 04:44:52 +0200 Subject: [PATCH 0199/1722] Update TablesLoader.h --- src/Databases/TablesLoader.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 5df52be4e97..bf469d83245 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -14,7 +14,6 @@ #include - namespace Poco { class Logger; // NOLINT(cppcoreguidelines-virtual-class-destructor) From f8a117affc055fc15396374ed8b258b13b5adc3f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 1 Jul 2024 20:23:39 +0000 Subject: [PATCH 0200/1722] Split the test --- ...efreshable_materialized_views_1.reference} | 30 --- .../02932_refreshable_materialized_views_1.sh | 179 +++++++++++++++++ ...refreshable_materialized_views_2.reference | 30 +++ ...02932_refreshable_materialized_views_2.sh} | 184 ++---------------- 4 files changed, 222 insertions(+), 201 deletions(-) rename tests/queries/0_stateless/{02932_refreshable_materialized_views.reference => 02932_refreshable_materialized_views_1.reference} (66%) create mode 100755 tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh create mode 100644 tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference rename tests/queries/0_stateless/{02932_refreshable_materialized_views.sh => 02932_refreshable_materialized_views_2.sh} (54%) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference similarity index 66% rename from tests/queries/0_stateless/02932_refreshable_materialized_views.reference rename to tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference index 3c6c8b2d778..bfc6add90a7 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference @@ -30,33 +30,3 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n( <17: chain-refreshed> b Scheduled 2062-01-01 00:00:00 <18: removed dependency> b Scheduled [] 2062-03-03 03:03:03 2064-01-01 00:00:00 5 CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a -<19: exception> 1 -<20: unexception> 1 -<21: rename> 1 -<22: rename> d Finished -<23: simple refresh> 1 -<24: rename during refresh> 1 -<25: rename during refresh> f Running -<27: cancelled> f Scheduled -<28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x -<29: randomize> 1 1 -CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src -<30: to existing table> 10 -<31: to existing table> 10 -<31: to existing table> 20 -<31.5: will retry> Error 1 -<31.6: did retry> 10 -<32: empty> i Scheduled Unknown 0 -<32: empty> j Scheduled Finished 0 -<34: append> 10 -<35: append> 10 -<35: append> 20 -<35: append> 30 -<36: not append> 20 -<36: not append> 30 -<37: append chain> 100 -<38: append chain> 100 -<38: append chain> 100 -<38: append chain> 200 -creating MergeTree without ORDER BY failed, as expected diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh new file mode 100755 index 00000000000..03d752b995d --- /dev/null +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh @@ -0,0 +1,179 @@ +#!/usr/bin/env bash +# Tags: atomic-database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh +# scheduling is done in UTC. +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" + +$CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" + + +# Basic refreshing. +$CLICKHOUSE_CLIENT -nq " + create materialized view a + refresh after 2 second + engine Memory + empty + as select number as x from numbers(2) union all select rand64() as x; + select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes; + show create a;" +# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] +do + sleep 0.5 +done +start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" +# Check table contents. +$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" +# Wait for table contents to change. +res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" +while : +do + res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" + [ "$res2" == "$res1" ] || break + sleep 0.5 +done +# Wait for another change. +while : +do + res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" + [ "$res3" == "$res2" ] || break + sleep 0.5 +done +# Check that the two changes were at least 1 second apart, in particular that we're not refreshing +# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer +# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. +$CLICKHOUSE_CLIENT -nq " + select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); + select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" + +# Create a source table from which views will read. +$CLICKHOUSE_CLIENT -nq " + create table src (x Int8) engine Memory as select 1;" + +# Switch to fake clock, change refresh schedule, change query. +$CLICKHOUSE_CLIENT -nq " + system test view a set fake time '2050-01-01 00:00:01'; + system wait view a; + system refresh view a; + system wait view a; + select '<4.1: fake clock>', status, last_refresh_time, next_refresh_time from refreshes; + alter table a modify refresh every 2 year; + alter table a modify query select x*2 as x from src; + select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; + show create a;" +# Advance time to trigger the refresh. +$CLICKHOUSE_CLIENT -nq " + select '<5: no refresh>', count() from a; + system test view a set fake time '2052-02-03 04:05:06';" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + select '<6: refreshed>', * from a; + select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" + +# Create a dependent view, refresh it once. +$CLICKHOUSE_CLIENT -nq " + create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; + show create b; + system test view b set fake time '2052-11-11 11:11:11'; + system refresh view b; + system wait view b; + select '<7.5: created dependent>', last_refresh_time from refreshes where view = 'b';" +# Next refresh shouldn't start until the dependency refreshes. +$CLICKHOUSE_CLIENT -nq " + select '<8: refreshed>', * from b; + select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; + system test view b set fake time '2054-01-24 23:22:21';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] +do + sleep 0.5 +done + +# Drop the source table, check that refresh fails and doesn't leave a temp table behind. +$CLICKHOUSE_CLIENT -nq " + select '<9.2: dropping>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase(); + drop table src; + system refresh view a;" +$CLICKHOUSE_CLIENT -nq "system wait view a;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" +$CLICKHOUSE_CLIENT -nq " + select '<9.4: dropped>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase();" + +# Create the source table again, check that refresh succeeds (in particular that tables are looked +# up by name rather than uuid). +$CLICKHOUSE_CLIENT -nq " + select '<10: creating>', view, status, remaining_dependencies, next_refresh_time from refreshes; + create table src (x Int16) engine Memory as select 2; + system test view a set fake time '2054-01-01 00:00:01';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] +do + sleep 0.5 +done +# Both tables should've refreshed. +$CLICKHOUSE_CLIENT -nq " + select '<11: chain-refreshed a>', * from a; + select '<12: chain-refreshed b>', * from b; + select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception == '' from refreshes;" + +# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to +# catch up to the same cycle. +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2059-01-01 00:00:00'; + system refresh view b;" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2061-01-01 00:00:00'; + system test view a set fake time '2057-01-01 00:00:00';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] +do + sleep 0.5 +done +sleep 1 +$CLICKHOUSE_CLIENT -nq " + select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; + truncate src; + insert into src values (3); + system test view a set fake time '2060-02-02 02:02:02';" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + select '<15: chain-refreshed a>', * from a; + select '<16: chain-refreshed b>', * from b; + select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" + +# Get to WaitingForDependencies state and remove the depencency. +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2062-03-03 03:03:03'" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + alter table b modify refresh every 2 year" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; + show create b;" + +$CLICKHOUSE_CLIENT -nq " + drop table src; + drop table a; + drop table b; + drop table refreshes;" diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference new file mode 100644 index 00000000000..eb4a0498260 --- /dev/null +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference @@ -0,0 +1,30 @@ +<19: exception> 1 +<20: unexception> 1 +<21: rename> 1 +<22: rename> d Finished +<23: simple refresh> 1 +<24: rename during refresh> 1 +<25: rename during refresh> f Running +<27: cancelled> f Scheduled +<28: drop during refresh> 0 0 +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x +<29: randomize> 1 1 +CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src +<30: to existing table> 10 +<31: to existing table> 10 +<31: to existing table> 20 +<31.5: will retry> Error 1 +<31.6: did retry> 10 +<32: empty> i Scheduled Unknown 0 +<32: empty> j Scheduled Finished 0 +<34: append> 10 +<35: append> 10 +<35: append> 20 +<35: append> 30 +<36: not append> 20 +<36: not append> 30 +<37: append chain> 100 +<38: append chain> 100 +<38: append chain> 100 +<38: append chain> 200 +creating MergeTree without ORDER BY failed, as expected diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh similarity index 54% rename from tests/queries/0_stateless/02932_refreshable_materialized_views.sh rename to tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh index d5b07287936..b62ea9fb638 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh @@ -15,172 +15,14 @@ CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_ma $CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" -# Basic refreshing. -$CLICKHOUSE_CLIENT -nq " - create materialized view a - refresh after 2 second - engine Memory - empty - as select number as x from numbers(2) union all select rand64() as x; - select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes; - show create a;" -# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] -do - sleep 0.1 -done -start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" -# Check table contents. -$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" -# Wait for table contents to change. -res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" -while : -do - res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res2" == "$res1" ] || break - sleep 0.1 -done -# Wait for another change. -while : -do - res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res3" == "$res2" ] || break - sleep 0.1 -done -# Check that the two changes were at least 1 second apart, in particular that we're not refreshing -# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer -# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. -$CLICKHOUSE_CLIENT -nq " - select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); - select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" - -# Create a source table from which views will read. -$CLICKHOUSE_CLIENT -nq " - create table src (x Int8) engine Memory as select 1;" - -# Switch to fake clock, change refresh schedule, change query. -$CLICKHOUSE_CLIENT -nq " - system test view a set fake time '2050-01-01 00:00:01'; - system wait view a; - system refresh view a; - system wait view a; - select '<4.1: fake clock>', status, last_refresh_time, next_refresh_time from refreshes; - alter table a modify refresh every 2 year; - alter table a modify query select x*2 as x from src; - select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; - show create a;" -# Advance time to trigger the refresh. -$CLICKHOUSE_CLIENT -nq " - select '<5: no refresh>', count() from a; - system test view a set fake time '2052-02-03 04:05:06';" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<6: refreshed>', * from a; - select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" - -# Create a dependent view, refresh it once. -$CLICKHOUSE_CLIENT -nq " - create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; - show create b; - system test view b set fake time '2052-11-11 11:11:11'; - system refresh view b; - system wait view b; - select '<7.5: created dependent>', last_refresh_time from refreshes where view = 'b';" -# Next refresh shouldn't start until the dependency refreshes. -$CLICKHOUSE_CLIENT -nq " - select '<8: refreshed>', * from b; - select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; - system test view b set fake time '2054-01-24 23:22:21';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] -do - sleep 0.1 -done - -# Drop the source table, check that refresh fails and doesn't leave a temp table behind. -$CLICKHOUSE_CLIENT -nq " - select '<9.2: dropping>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase(); - drop table src; - system refresh view a;" -$CLICKHOUSE_CLIENT -nq "system wait view a;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" -$CLICKHOUSE_CLIENT -nq " - select '<9.4: dropped>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase();" - -# Create the source table again, check that refresh succeeds (in particular that tables are looked -# up by name rather than uuid). -$CLICKHOUSE_CLIENT -nq " - select '<10: creating>', view, status, remaining_dependencies, next_refresh_time from refreshes; - create table src (x Int16) engine Memory as select 2; - system test view a set fake time '2054-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] -do - sleep 0.1 -done -# Both tables should've refreshed. -$CLICKHOUSE_CLIENT -nq " - select '<11: chain-refreshed a>', * from a; - select '<12: chain-refreshed b>', * from b; - select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception == '' from refreshes;" - -# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to -# catch up to the same cycle. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2059-01-01 00:00:00'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2061-01-01 00:00:00'; - system test view a set fake time '2057-01-01 00:00:00';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] -do - sleep 0.1 -done -sleep 1 -$CLICKHOUSE_CLIENT -nq " - select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; - truncate src; - insert into src values (3); - system test view a set fake time '2060-02-02 02:02:02';" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<15: chain-refreshed a>', * from a; - select '<16: chain-refreshed b>', * from b; - select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" - -# Get to WaitingForDependencies state and remove the depencency. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2062-03-03 03:03:03'" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - alter table b modify refresh every 2 year" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; - show create b;" - # Select from a table that doesn't exist, get an exception. $CLICKHOUSE_CLIENT -nq " - drop table a; - drop table b; + create table src (x Int8) engine Memory as select 1; create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; drop table src;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Error' ] do - sleep 0.1 + sleep 0.5 done # Check exception, create src, expect successful refresh. $CLICKHOUSE_CLIENT -nq " @@ -189,7 +31,7 @@ $CLICKHOUSE_CLIENT -nq " system refresh view c;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do - sleep 0.1 + sleep 0.5 done # Rename table. $CLICKHOUSE_CLIENT -nq " @@ -207,7 +49,7 @@ $CLICKHOUSE_CLIENT -nq " create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do - sleep 0.1 + sleep 0.5 done # Stop refreshes. $CLICKHOUSE_CLIENT -nq " @@ -215,7 +57,7 @@ $CLICKHOUSE_CLIENT -nq " system stop view e;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Disabled' ] do - sleep 0.1 + sleep 0.5 done # Make refreshes slow, wait for a slow refresh to start. (We stopped refreshes first to make sure # we wait for a slow refresh, not a previous fast one.) @@ -224,7 +66,7 @@ $CLICKHOUSE_CLIENT -nq " system start view e;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] do - sleep 0.1 + sleep 0.5 done # Rename. $CLICKHOUSE_CLIENT -nq " @@ -238,7 +80,7 @@ $CLICKHOUSE_CLIENT -nq " system cancel view f;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Cancelled' ] do - sleep 0.1 + sleep 0.5 done # Check that another refresh doesn't immediately start after the cancelled one. sleep 1 @@ -247,7 +89,7 @@ $CLICKHOUSE_CLIENT -nq " system refresh view f;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] do - sleep 0.1 + sleep 0.5 done # Drop. $CLICKHOUSE_CLIENT -nq " @@ -262,7 +104,7 @@ $CLICKHOUSE_CLIENT -nq " system test view g set fake time '2050-02-03 15:30:13';" while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " with '2050-02-10 04:00:00'::DateTime as expected @@ -278,14 +120,14 @@ $CLICKHOUSE_CLIENT -nq " show create h;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<30: to existing table>', * from dest; insert into src values (2);" while [ "`$CLICKHOUSE_CLIENT -nq "select count() from dest -- $LINENO" | xargs`" != '2' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<31: to existing table>', * from dest; @@ -304,7 +146,7 @@ $CLICKHOUSE_CLIENT -nq " drop table src2;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result, retry from refreshes -- $LINENO" | xargs`" != 'Finished 0' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<31.6: did retry>', x from h2; @@ -316,7 +158,7 @@ $CLICKHOUSE_CLIENT -nq " create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2);" while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<32: empty>', view, status, last_refresh_result, retry from refreshes order by view; From fcb01277e143011154c659b9461891e6a28017c7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 1 Jul 2024 20:25:16 +0000 Subject: [PATCH 0201/1722] July --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b0c4f256e45..c20a1dee3c2 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -87,6 +87,7 @@ namespace SettingsChangesHistory static const std::map settings_changes_history = { {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, + {"allow_materialized_view_with_bad_select", true, false, "Stricter validation in CREATE MATERIALIZED VIEW"}, }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, @@ -117,7 +118,6 @@ static const std::map Date: Tue, 2 Jul 2024 03:10:26 +0000 Subject: [PATCH 0202/1722] Fix unrelated setting --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2806ccf98dc..c17e6642ac2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list Date: Tue, 2 Jul 2024 14:01:19 +0000 Subject: [PATCH 0203/1722] initial commit for Hive-style partitioning --- src/Core/Settings.h | 5 + src/Core/SettingsChangesHistory.h | 5 + .../ObjectStorage/StorageObjectStorage.cpp | 32 ++- .../StorageObjectStorageSource.cpp | 14 +- src/Storages/StorageFile.cpp | 39 +++- src/Storages/StorageURL.cpp | 16 +- src/Storages/VirtualColumnUtils.cpp | 52 ++++- src/Storages/VirtualColumnUtils.h | 7 +- .../__init__.py | 0 .../configs/cluster_azure.xml | 39 ++++ .../configs/cluster_hdfs.xml | 33 +++ .../configs/disable_profilers_azure.xml | 9 + .../configs/macro_hdfs.xml | 5 + .../configs/named_collections_azure.xml | 14 ++ .../configs/schema_cache_azure.xml | 3 + .../configs/schema_cache_hdfs.xml | 3 + .../configs/users_azure.xml | 9 + .../test_azure.py | 204 ++++++++++++++++++ .../test_hdfs.py | 81 +++++++ .../03203_hive_style_partitioning.reference | 96 +++++++++ .../03203_hive_style_partitioning.sh | 93 ++++++++ .../column1=Gordon/sample.parquet | Bin 0 -> 1308 bytes .../column1=Schmidt/sample.parquet | Bin 0 -> 1308 bytes .../column0=Elizabeth/sample.parquet | Bin 0 -> 1308 bytes .../sample.parquet | Bin 0 -> 1308 bytes .../column1=Gordon/sample.parquet | Bin 0 -> 1308 bytes .../column1=Schmidt/sample.parquet | Bin 0 -> 1308 bytes .../coumn0=Elizabeth/sample.parquet | Bin 0 -> 1308 bytes .../sample.parquet | Bin 0 -> 1308 bytes 29 files changed, 749 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py create mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py create mode 100644 tests/queries/0_stateless/03203_hive_style_partitioning.reference create mode 100755 tests/queries/0_stateless/03203_hive_style_partitioning.sh create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_minio/hive_partitioning/non_existing_column=Elizabeth/sample.parquet diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 14fe0924b40..738c0129d2d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1106,6 +1106,11 @@ class IColumn; M(Bool, input_format_tsv_skip_trailing_empty_lines, false, "Skip trailing empty lines in TSV format", 0) \ M(Bool, input_format_custom_skip_trailing_empty_lines, false, "Skip trailing empty lines in CustomSeparated format", 0) \ M(Bool, input_format_tsv_crlf_end_of_line, false, "If it is set true, file function will read TSV format with \\r\\n instead of \\n.", 0) \ + M(Bool, file_hive_partitioning, false, "Allows to use hive partitioning for file format", 0)\ + M(Bool, url_hive_partitioning, false, "Allows to use hive partitioning for url format", 0)\ + M(Bool, s3_hive_partitioning, false, "Allows to use hive partitioning for s3 format", 0)\ + M(Bool, azure_blob_storage_hive_partitioning, false, "Allows to use hive partitioning for AzureBlobStorage format", 0)\ + M(Bool, hdfs_hive_partitioning, false, "Allows to use hive partitioning for hdfs format", 0)\ \ M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4ac25a649b7..dd778149674 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -118,6 +118,11 @@ static const std::map +#include #include #include @@ -32,6 +33,19 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +bool checkIfHiveSettingEnabled(const ContextPtr & context, const std::string & storage_type_name) +{ + if (storage_type_name == "s3") + return context->getSettings().s3_hive_partitioning; + else if (storage_type_name == "hdfs") + return context->getSettings().hdfs_hive_partitioning; + else if (storage_type_name == "azure") + return context->getSettings().azure_blob_storage_hive_partitioning; + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); +} + StorageObjectStorage::StorageObjectStorage( ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, @@ -60,7 +74,23 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + object_storage, + distributed_processing_, + context, + {}, // predicate + metadata.getColumns().getAll(), // virtual_columns + nullptr, // read_keys + {} // file_progress_callback + ); + + Strings paths; + + if (checkIfHiveSettingEnabled(context, configuration->getTypeName())) + if (auto file = file_iterator->next(0)) + paths = {file->getPath()}; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), paths)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index aef783fc3c4..2741cfecf6b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -195,13 +196,24 @@ Chunk StorageObjectStorageSource::generate() const auto & object_info = reader.getObjectInfo(); const auto & filename = object_info->getFileName(); chassert(object_info->metadata); + + auto hive_map = VirtualColumnUtils::parsePartitionMapFromPath(object_info->getPath()); + bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), + [&](const auto& pair) { + return read_from_format_info.requested_virtual_columns.contains(pair.first); + }); + + if (!contains_virtual_column) + hive_map.clear(); // If we cannot find any virual column in requested, we don't add any of them to chunk + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, { .path = getUniqueStoragePathIdentifier(*configuration, *object_info, false), .size = object_info->metadata->size_bytes, .filename = &filename, - .last_modified = object_info->metadata->last_modified + .last_modified = object_info->metadata->last_modified, + .hive_partitioning_map = hive_map }); return chunk; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7f39ff615f0..0c32f29cb34 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ #include #include #include +#include #include #include @@ -1095,7 +1096,11 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setConstraints(args.constraints); storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + + Strings paths_for_virtuals; + if (args.getContext()->getSettingsRef().file_hive_partitioning) + paths_for_virtuals = paths; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), paths_for_virtuals)); } @@ -1437,6 +1442,15 @@ Chunk StorageFileSource::generate() chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); + std::map hive_map; + if (getContext()->getSettingsRef().file_hive_partitioning) + { + hive_map = VirtualColumnUtils::parsePartitionMapFromPath(current_path); + + for (const auto& item : hive_map) + requested_virtual_columns.push_back(NameAndTypePair(item.first, std::make_shared())); + } + /// Enrich with virtual columns. VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, requested_virtual_columns, @@ -1444,7 +1458,8 @@ Chunk StorageFileSource::generate() .path = current_path, .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), - .last_modified = current_file_last_modified + .last_modified = current_file_last_modified, + .hive_partitioning_map = hive_map }); return chunk; @@ -1621,6 +1636,16 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) storage->distributed_processing); } +void addPartitionColumnsToInfoHeader(Strings paths, ReadFromFormatInfo & info) +{ + for (const auto& path : paths) + { + auto map = VirtualColumnUtils::parsePartitionMapFromPath(path); + for (const auto& item : map) + info.source_header.insertUnique(ColumnWithTypeAndName(std::make_shared(), item.first)); + } +} + void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); @@ -1628,10 +1653,20 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui size_t num_streams = max_num_streams; size_t files_to_read = 0; + Strings paths; if (storage->archive_info) + { files_to_read = storage->archive_info->paths_to_archives.size(); + paths = storage->archive_info->paths_to_archives; + } else + { files_to_read = storage->paths.size(); + paths = storage->paths; + } + + if (getContext()->getSettingsRef().file_hive_partitioning) + addPartitionColumnsToInfoHeader(paths, info); if (max_num_streams > files_to_read) num_streams = files_to_read; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 895da028fc2..f6374701fc2 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include @@ -151,7 +152,11 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + + Strings uri_for_partitioning; + if (context_->getSettingsRef().url_hive_partitioning) + uri_for_partitioning = {uri}; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning)); } @@ -410,12 +415,17 @@ Chunk StorageURLSource::generate() size_t chunk_size = 0; if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); + std::map hive_map; + if (getContext()->getSettingsRef().url_hive_partitioning) + hive_map = VirtualColumnUtils::parsePartitionMapFromPath(curr_uri.getPath()); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, requested_virtual_columns, { .path = curr_uri.getPath(), - .size = current_file_size + .size = current_file_size, + .hive_partitioning_map = hive_map }); return chunk; } @@ -1170,6 +1180,7 @@ void ReadFromURL::createIterator(const ActionsDAG::Node * predicate) void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); + const auto & settings = context->getSettingsRef(); if (is_empty_glob) { @@ -1180,7 +1191,6 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil Pipes pipes; pipes.reserve(num_streams); - const auto & settings = context->getSettingsRef(); const size_t max_parsing_threads = num_streams >= settings.max_parsing_threads ? 1 : (settings.max_parsing_threads / num_streams); for (size_t i = 0; i < num_streams; ++i) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..0b79e3b7a16 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -37,6 +36,7 @@ #include #include +#include #include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" @@ -115,7 +115,22 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) +Strings parseVirtualColumnNameFromPath(const std::string & path) +{ + std::string pattern = "/([^/]+)=([^/]+)"; + // Map to store the key-value pairs + std::map key_values; + + re2::StringPiece input_piece(path); + std::string key; + Strings result; + while (RE2::FindAndConsume(&input_piece, pattern, &key)) + result.push_back(key); + + return result; +} + +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths) { VirtualColumnsDescription desc; @@ -132,6 +147,13 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); + for (const auto& path : paths) + { + auto names = parseVirtualColumnNameFromPath(path); + for (const auto& name : names) + add_virtual("_" + name, std::make_shared(std::make_shared())); + } + return desc; } @@ -178,6 +200,8 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); + if (!getVirtualNamesForFileLikeStorage().contains(column.name)) + block.insert({column.type->createColumn(), column.type, column.name}); } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); @@ -189,6 +213,21 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } +std::map parsePartitionMapFromPath(const std::string & path) +{ + std::string pattern = "/([^/]+)=([^/]+)"; // Regex to capture key=value pairs + // Map to store the key-value pairs + std::map key_values; + + re2::StringPiece input_piece(path); + std::string key; + std::string value; + while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) + key_values["_" + key] = value; + + return key_values; +} + void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, VirtualsForFileLikeStorage virtual_values) @@ -226,6 +265,15 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } + else + { + auto it = virtual_values.hive_partitioning_map.find(virtual_column.getNameInStorage()); + if (it != virtual_values.hive_partitioning_map.end()) + { + chunk.addColumn(virtual_column.getTypeInStorage()->createColumnConst(chunk.getNumRows(), it->second)->convertToFullColumnIfConst()); + virtual_values.hive_partitioning_map.erase(it); + } + } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..a03d4c7447f 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -6,6 +6,8 @@ #include #include +#include +#include #include @@ -47,7 +49,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) } NameSet getVirtualNamesForFileLikeStorage(); -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns); +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths = {}); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); @@ -74,9 +76,12 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; + std::map hive_partitioning_map; }; +std::map parsePartitionMapFromPath(const std::string & path); + void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, VirtualsForFileLikeStorage virtual_values); diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml new file mode 100644 index 00000000000..ffa4673c9ee --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml @@ -0,0 +1,39 @@ + + + + + + node_0 + 9000 + + + node_1 + 9000 + + + node_2 + 9000 + + + + + + + + node_0 + 9000 + + + + + node_1 + 19000 + + + + + + + simple_cluster + + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml new file mode 100644 index 00000000000..b99b21ea40b --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml @@ -0,0 +1,33 @@ + + + + + + node1 + 9000 + + + + + node1 + 19000 + + + + + + + + 127.0.0.1 + 9000 + + + + + 127.0.0.2 + 9000 + + + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml new file mode 100644 index 00000000000..a39badbf8ec --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml @@ -0,0 +1,9 @@ + + + + + 0 + 0 + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml new file mode 100644 index 00000000000..c2e11b47a5e --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml @@ -0,0 +1,5 @@ + + + test_cluster_two_shards + + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml new file mode 100644 index 00000000000..bd7f9ff97f1 --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml @@ -0,0 +1,14 @@ + + + + cont + test_simple_write_named.csv + key UInt64, data String + CSV + + + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml new file mode 100644 index 00000000000..e2168ecd06d --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml @@ -0,0 +1,3 @@ + + 2 + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml new file mode 100644 index 00000000000..37639649b5f --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml @@ -0,0 +1,3 @@ + + 2 + \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml new file mode 100644 index 00000000000..4b6ba057ecb --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py new file mode 100644 index 00000000000..c9b2c9fec2e --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py @@ -0,0 +1,204 @@ +#!/usr/bin/env python3 + +import pytest +import time + +from helpers.cluster import ClickHouseCluster, is_arm +import re + +from azure.storage.blob import BlobServiceClient +from helpers.cluster import ClickHouseCluster, ClickHouseInstance + +if is_arm(): + pytestmark = pytest.mark.skip + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=["configs/named_collections_azure.xml", "configs/schema_cache_azure.xml"], + user_configs=["configs/disable_profilers_azure.xml", "configs/users_azure.xml"], + with_azurite=True, + ) + cluster.start() + container_client = cluster.blob_service_client.get_container_client("cont") + container_client.create_container() + yield cluster + finally: + cluster.shutdown() + + +def azure_query( + node, query, expect_error=False, try_num=10, settings={}, query_on_retry=None +): + for i in range(try_num): + try: + if expect_error: + return node.query_and_get_error(query, settings=settings) + else: + return node.query(query, settings=settings) + except Exception as ex: + retriable_errors = [ + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read", + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + print(f"Try num: {i}. Having retriable error: {ex}") + time.sleep(i) + break + if not retry or i == try_num - 1: + raise Exception(ex) + if query_on_retry is not None: + node.query(query_on_retry) + continue + + +def get_azure_file_content(filename, port): + container_name = "cont" + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string( + str(connection_string) + ) + container_client = blob_service_client.get_container_client(container_name) + blob_client = container_client.get_blob_client(filename) + download_stream = blob_client.download_blob() + return download_stream.readall().decode("utf-8") + + +@pytest.fixture(autouse=True, scope="function") +def delete_all_files(cluster): + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + blob_list = container_client.list_blobs() + for blob in blob_list: + print(blob) + blob_client = container_client.get_blob_client(blob) + blob_client.delete_blob() + + assert len(list(container_client.list_blobs())) == 0 + + yield + + +def test_azure_partitioning_with_one_parameter(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values = f"('Elizabeth', 'Gordon')" + path = "a/column1=Elizabeth/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}')" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Gordon" + ] + +def test_azure_partitioning_with_two_parameters(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth" + ] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" + ) + assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + "Elizabeth" + ] + +def test_azure_partitioning_without_setting(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + + with pytest.raises(Exception, match=pattern): + azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py new file mode 100644 index 00000000000..38641b63960 --- /dev/null +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py @@ -0,0 +1,81 @@ +#!/usr/bin/env python3 + +import pytest + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster, is_arm +import re + +from helpers.cluster import ClickHouseCluster + +if is_arm(): + pytestmark = pytest.mark.skip + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/macro_hdfs.xml", + "configs/schema_cache_hdfs.xml", + "configs/cluster_hdfs.xml", + ], + with_hdfs=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_hdfs_partitioning_with_one_parameter(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") + == f"Elizabeth\tGordon\n" + ) + + r = node1.query( + "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", settings={"hdfs_hive_partitioning": 1} + ) + assert (r == f"Elizabeth\n") + +def test_hdfs_partitioning_with_two_parameters(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + + r = node1.query( + "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 1} + ) + assert (r == f"Gordon\n") + +def test_hdfs_partitioning_without_setting(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + + with pytest.raises(QueryRuntimeException, match=pattern): + node1.query(f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 0}) + +if __name__ == "__main__": + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference new file mode 100644 index 00000000000..6ef1fcdf652 --- /dev/null +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -0,0 +1,96 @@ +TESTING THE FILE HIVE PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +1 +TESTING THE URL PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +1 +TESTING THE S3 PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +Elizabeth Gordon Elizabeth Gordon +Elizabeth Gordon Elizabeth +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +1 diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh new file mode 100755 index 00000000000..a5d4c85a33b --- /dev/null +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -0,0 +1,93 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" + + +$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 1; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" + +$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 0; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" + + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" + + +$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 1; + +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" + +$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 0; + +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" + + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" + + +$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 1; + +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; + +SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" + +$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 0; + +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" + diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/non_existing_column=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/non_existing_column=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 From 83462d743e76dcfa8fd35b8b30335682f86d9374 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 2 Jul 2024 16:13:44 +0200 Subject: [PATCH 0204/1722] enhance SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..607f9b6d858 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,11 @@ static std::initializer_list Date: Tue, 2 Jul 2024 14:43:45 +0000 Subject: [PATCH 0205/1722] style check --- .../ObjectStorage/StorageObjectStorage.cpp | 2 +- .../StorageObjectStorageSource.cpp | 7 +-- .../test_azure.py | 43 +++++++++++++------ .../test_hdfs.py | 32 ++++++++------ .../03203_hive_style_partitioning.sh | 24 +++++++---- 5 files changed, 69 insertions(+), 39 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index b169f02940e..ae7c211330c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -86,7 +86,7 @@ StorageObjectStorage::StorageObjectStorage( ); Strings paths; - + if (checkIfHiveSettingEnabled(context, configuration->getTypeName())) if (auto file = file_iterator->next(0)) paths = {file->getPath()}; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2741cfecf6b..afb23961312 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -198,13 +198,14 @@ Chunk StorageObjectStorageSource::generate() chassert(object_info->metadata); auto hive_map = VirtualColumnUtils::parsePartitionMapFromPath(object_info->getPath()); - bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), - [&](const auto& pair) { + bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), + [&](const auto& pair) + { return read_from_format_info.requested_virtual_columns.contains(pair.first); }); if (!contains_virtual_column) - hive_map.clear(); // If we cannot find any virual column in requested, we don't add any of them to chunk + hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py index c9b2c9fec2e..0be697821f0 100644 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py @@ -12,14 +12,21 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance if is_arm(): pytestmark = pytest.mark.skip + @pytest.fixture(scope="module") def cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( "node", - main_configs=["configs/named_collections_azure.xml", "configs/schema_cache_azure.xml"], - user_configs=["configs/disable_profilers_azure.xml", "configs/users_azure.xml"], + main_configs=[ + "configs/named_collections_azure.xml", + "configs/schema_cache_azure.xml", + ], + user_configs=[ + "configs/disable_profilers_azure.xml", + "configs/users_azure.xml", + ], with_azurite=True, ) cluster.start() @@ -121,7 +128,9 @@ def test_azure_partitioning_with_one_parameter(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}')" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( bucket="cont", max_path=path ) @@ -132,9 +141,10 @@ def test_azure_partitioning_with_one_parameter(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ - "Gordon" - ] + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Gordon"] + def test_azure_partitioning_with_two_parameters(cluster): # type: (ClickHouseCluster) -> None @@ -155,7 +165,9 @@ def test_azure_partitioning_with_two_parameters(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( bucket="cont", max_path=path ) @@ -166,18 +178,19 @@ def test_azure_partitioning_with_two_parameters(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ - "Elizabeth" - ] + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] query = ( f"SELECT column1 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" ) - assert azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 1}).splitlines() == [ - "Elizabeth" - ] + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] + def test_azure_partitioning_without_setting(cluster): # type: (ClickHouseCluster) -> None @@ -198,7 +211,9 @@ def test_azure_partitioning_without_setting(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) - pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) with pytest.raises(Exception, match=pattern): azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py index 38641b63960..4667d18688a 100644 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py +++ b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py @@ -31,20 +31,18 @@ def started_cluster(): finally: cluster.shutdown() + def test_hdfs_partitioning_with_one_parameter(started_cluster): hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") - == f"Elizabeth\tGordon\n" - ) + hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") + assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" r = node1.query( - "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", settings={"hdfs_hive_partitioning": 1} + "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + settings={"hdfs_hive_partitioning": 1}, ) - assert (r == f"Elizabeth\n") + assert r == f"Elizabeth\n" + def test_hdfs_partitioning_with_two_parameters(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -57,9 +55,11 @@ def test_hdfs_partitioning_with_two_parameters(started_cluster): ) r = node1.query( - "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 1} + "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 1}, ) - assert (r == f"Gordon\n") + assert r == f"Gordon\n" + def test_hdfs_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -70,10 +70,16 @@ def test_hdfs_partitioning_without_setting(started_cluster): hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") == f"Elizabeth\tGordon\n" ) - pattern = re.compile(r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) with pytest.raises(QueryRuntimeException, match=pattern): - node1.query(f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"hdfs_hive_partitioning": 0}) + node1.query( + f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 0}, + ) + if __name__ == "__main__": cluster.start() diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index a5d4c85a33b..83a8f87a813 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -8,7 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 1; +$CLICKHOUSE_LOCAL -n -q """ +set file_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -31,13 +32,15 @@ SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.pa $CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 0; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 1; +$CLICKHOUSE_LOCAL -n -q """ +set url_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -60,13 +63,15 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/ $CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 0; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 1; +$CLICKHOUSE_LOCAL -n -q """ +set s3_hive_partitioning = 1; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -85,9 +90,12 @@ SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/c SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +""" -$CLICKHOUSE_LOCAL -n -q """set s3_hive_partitioning = 0; +$CLICKHOUSE_LOCAL -n -q """ +set s3_hive_partitioning = 0; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10;""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" From cd5cdcc124f95204a6f63e8a1ce4d7148d8fec7f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 2 Jul 2024 17:00:59 +0200 Subject: [PATCH 0206/1722] Shellcheck fix --- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 83a8f87a813..98c039f3454 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -30,7 +30,8 @@ SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/c SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" -$CLICKHOUSE_LOCAL -n -q """set file_hive_partitioning = 0; +$CLICKHOUSE_LOCAL -n -q """ +set file_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -61,7 +62,8 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=El SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" -$CLICKHOUSE_LOCAL -n -q """set url_hive_partitioning = 0; +$CLICKHOUSE_LOCAL -n -q """ +set url_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -98,4 +100,3 @@ set s3_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" - From dc9dc1676d8f8af74c20173927c6027623cc788c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 2 Jul 2024 17:37:47 +0200 Subject: [PATCH 0207/1722] add default for map in VirtualsForFileLikeStorage --- src/Storages/VirtualColumnUtils.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index a03d4c7447f..f9b49cc48ed 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -76,7 +76,7 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - std::map hive_partitioning_map; + std::map hive_partitioning_map {}; }; From b14823d07ee1c059d2f278c0cc4068474c79ad2a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 3 Jul 2024 09:33:57 +0200 Subject: [PATCH 0208/1722] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7719fe1e837..828031f4c23 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list Date: Wed, 3 Jul 2024 13:47:18 +0000 Subject: [PATCH 0209/1722] fix ambiguous override of non-virtual --- src/Client/Connection.cpp | 2 +- src/IO/ReadBufferFromPocoSocketChunked.cpp | 8 ++++---- src/IO/ReadBufferFromPocoSocketChunked.h | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 803f68c69d6..198518d6314 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1122,7 +1122,7 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadPendingData() const { - return last_input_packet_type.has_value() || in->hasPendingData(); + return last_input_packet_type.has_value() || in->hasBufferedData(); } diff --git a/src/IO/ReadBufferFromPocoSocketChunked.cpp b/src/IO/ReadBufferFromPocoSocketChunked.cpp index 93afeadba60..4a1e3732a55 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.cpp +++ b/src/IO/ReadBufferFromPocoSocketChunked.cpp @@ -34,12 +34,12 @@ void ReadBufferFromPocoSocketChunked::enableChunked() next_chunk = 0; } -bool ReadBufferFromPocoSocketChunked::hasPendingData() const +bool ReadBufferFromPocoSocketChunked::hasBufferedData() const { - if (chunked) - return available() || static_cast(data_end - working_buffer.end()) > sizeof(next_chunk); + if (available()) + return true; - return ReadBufferFromPocoSocketBase::hasPendingData(); + return chunked && (static_cast(data_end - working_buffer.end()) > sizeof(next_chunk)); } bool ReadBufferFromPocoSocketChunked::poll(size_t timeout_microseconds) const diff --git a/src/IO/ReadBufferFromPocoSocketChunked.h b/src/IO/ReadBufferFromPocoSocketChunked.h index 943a50f5d08..8bc4024b978 100644 --- a/src/IO/ReadBufferFromPocoSocketChunked.h +++ b/src/IO/ReadBufferFromPocoSocketChunked.h @@ -84,7 +84,7 @@ public: void enableChunked(); - bool hasPendingData() const; + bool hasBufferedData() const; bool poll(size_t timeout_microseconds) const; From cb884d0ac7461499badc169380b9136941258693 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jul 2024 15:35:52 +0000 Subject: [PATCH 0210/1722] fix applying of metadata mutations --- src/Storages/MergeTree/AlterConversions.cpp | 1 + src/Storages/MergeTree/MergeTask.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.cpp | 72 ++++++------ src/Storages/MergeTree/MergeTreeData.h | 34 +++--- src/Storages/MergeTree/MutateTask.cpp | 13 ++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 103 ++++++++++++------ .../MergeTree/ReplicatedMergeTreeQueue.h | 11 +- src/Storages/StorageMergeTree.cpp | 26 ++--- src/Storages/StorageMergeTree.h | 9 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 11 files changed, 158 insertions(+), 127 deletions(-) diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp index 82bef500b34..a36611e3d87 100644 --- a/src/Storages/MergeTree/AlterConversions.cpp +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -11,6 +11,7 @@ namespace ErrorCodes bool AlterConversions::isSupportedDataMutation(MutationCommand::Type) { + /// Currently there is no such mutations. See setting 'apply_mutations_on_fly'. return false; } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5dab0cd0c08..08e6f654f15 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -257,9 +257,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (enabledBlockOffsetColumn(global_ctx)) addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); - auto mutations_snapshot = global_ctx->data->getMutationsSnapshot( - global_ctx->metadata_snapshot->getMetadataVersion(), - /*need_data_mutations=*/ false); + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = global_ctx->metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = MergeTreeData::getMinMetadataVersion(global_ctx->future_part->parts), + }; + + auto mutations_snapshot = global_ctx->data->getMutationsSnapshot(params); SerializationInfo::Settings info_settings = { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 98c308f5fd1..d06570c3ed8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8418,6 +8418,18 @@ bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, return !hasLightweightDeletedMask(); } +Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) +{ + Int64 version = -1; + for (const auto & part : parts) + { + Int64 part_version = part->getMetadataVersion(); + if (version == -1 || part_version < version) + version = part_version; + } + return version; +} + StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const { auto snapshot_data = std::make_unique(); @@ -8429,10 +8441,14 @@ StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & object_columns_copy = object_columns; } - snapshot_data->mutations_snapshot = getMutationsSnapshot( - metadata_snapshot->getMetadataVersion(), - query_context->getSettingsRef().apply_mutations_on_fly); + IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = getMinMetadataVersion(snapshot_data->parts), + .need_data_mutations = query_context->getSettingsRef().apply_mutations_on_fly, + }; + snapshot_data->mutations_snapshot = getMutationsSnapshot(params); return std::make_shared(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data)); } @@ -8618,59 +8634,33 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -static void updateMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - Int64 increment) +static void updateAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, Int64 increment) { - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); - - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); - - bool has_data_mutation = false; - bool has_metadata_mutation = false; + if (num_alter_conversions < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data alter conversions counter is negative ({})", num_alter_conversions); for (const auto & command : commands) { - if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) + if (AlterConversions::isSupportedDataMutation(command.type) || AlterConversions::isSupportedMetadataMutation(command.type)) { - data_mutations_to_apply += increment; - has_data_mutation = true; + num_alter_conversions += increment; - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); - } + if (num_alter_conversions < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_alter_conversions); - if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) - { - metadata_mutations_to_apply += increment; - has_metadata_mutation = true; - - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + return; } } } -void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) +void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); + updateAlterConversionsCounter(num_alter_conversions, commands, 1); } -void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) +void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); + updateAlterConversionsCounter(num_alter_conversions, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 765b68b7559..faf55292257 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -448,6 +448,21 @@ public: struct IMutationsSnapshot { + struct Params + { + Int64 metadata_version = -1; + Int64 min_part_metadata_version = -1; + bool need_data_mutations = false; + + bool needAnyMutations() const { return need_data_mutations || needMetadataMutations(); } + bool needMetadataMutations() const { return min_part_metadata_version < metadata_version; } + }; + + Params params; + + IMutationsSnapshot() = default; + explicit IMutationsSnapshot(Params params_) : params(std::move(params_)) {} + /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly /// (i.e. when reading from the part). Mutations not supported by AlterConversions /// (supportsMutationCommandType()) can be omitted. @@ -455,7 +470,6 @@ public: /// @return list of mutations, in *reverse* order (newest to oldest) virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; virtual std::shared_ptr cloneEmpty() const = 0; - virtual ~IMutationsSnapshot() = default; }; @@ -951,7 +965,10 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } /// TODO: comment - virtual MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const = 0; + virtual MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const = 0; + + /// TODO: comment + static Int64 getMinMetadataVersion(const DataPartsVector & parts); /// Return alter conversions for part which must be applied on fly. static AlterConversionsPtr getAlterConversionsForPart( @@ -1752,16 +1769,7 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated -void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); - -void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); +void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); +void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3c4ef44dbd8..4a9138c10da 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2118,13 +2118,14 @@ bool MutateTask::prepare() ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); - auto mutations_snapshot = ctx->data->getMutationsSnapshot( - ctx->metadata_snapshot->getMetadataVersion(), - /*need_data_mutations=*/ false); + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = ctx->metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = ctx->source_part->getMetadataVersion(), + }; - auto alter_conversions = MergeTreeData::getAlterConversionsForPart( - ctx->source_part, - mutations_snapshot); + auto mutations_snapshot = ctx->data->getMutationsSnapshot(params); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot); auto context_for_reading = Context::createCopy(ctx->context); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index fbf949b47f5..807fbeebfc4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -951,7 +951,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, commands, state_lock); } else it = mutations_by_znode.erase(it); @@ -1000,10 +1000,9 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { - auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)) - .first->second; + auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; + incrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) { @@ -1077,7 +1076,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// decrementMutationsCounters() will be called in updateMutations() + /// decrementAlterConversionsCounter() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1901,6 +1900,7 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk return ReplicatedMergeTreeMergePredicate(*this, zookeeper, std::move(partition_ids_hint)); } + MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { auto in_partition = mutations_by_partition.find(part->info.partition_id); @@ -1908,20 +1908,23 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo return {}; Int64 part_data_version = part->info.getDataVersion(); - int32_t part_metadata_version = part->getMetadataVersion(); + Int64 part_metadata_version = part->getMetadataVersion(); MutationCommands result; - bool seen_all_data_mutations = false; - bool seen_all_metadata_mutations = false; + bool seen_all_data_mutations = !params.need_data_mutations; + bool seen_all_metadata_mutations = !params.needMetadataMutations(); + + if (seen_all_data_mutations && seen_all_metadata_mutations) + return {}; auto add_to_result = [&](const ReplicatedMergeTreeMutationEntryPtr & entry) { for (const auto & command : entry->commands | std::views::reverse) { - if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + if (AlterConversions::isSupportedMetadataMutation(command.type)) result.push_back(command); - else if (AlterConversions::isSupportedMetadataMutation(command.type)) + else if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) result.push_back(command); } }; @@ -1935,9 +1938,10 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo break; auto alter_version = entry->alter_version; - if (alter_version != -1) + + if (!seen_all_metadata_mutations && alter_version != -1) { - if (alter_version > metadata_version) + if (alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version @@ -1946,7 +1950,7 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo else seen_all_metadata_mutations = true; } - else + else if (!seen_all_data_mutations) { if (mutation_version > part_data_version) add_to_result(entry); @@ -1958,42 +1962,71 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo return result; } -MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const { - auto res = std::make_shared(); - res->metadata_version = metadata_version; - res->need_data_mutations = need_data_mutations; + auto res = std::make_shared(params); std::lock_guard lock(state_mutex); - bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; - bool have_metadata_mutations = metadata_mutations_to_apply > 0; + bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; + bool need_metatadata_mutations = res->params.needMetadataMutations(); - if (!have_data_mutations && !have_metadata_mutations) + if (!need_data_mutations && !need_metatadata_mutations) return res; + auto is_supported_command = [&](const auto & command) + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }; + for (const auto & [partition_id, mutations] : mutations_by_partition) { auto & in_partition = res->mutations_by_partition[partition_id]; - for (const auto & [version, status] : mutations | std::views::reverse) + bool seen_all_data_mutations = !need_data_mutations; + bool seen_all_metadata_mutations = !need_metatadata_mutations; + + for (const auto & [mutation_version, status] : mutations | std::views::reverse) { - if (status->is_done) + if (seen_all_data_mutations && seen_all_metadata_mutations) break; - bool has_required_command = std::ranges::any_of(status->entry->commands, [&](const auto & command) + auto alter_version = status->entry->alter_version; + + if (!seen_all_metadata_mutations && alter_version != -1) { - if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) - return true; + if (alter_version > params.metadata_version) + continue; - if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) - return true; - - return false; - }); - - if (has_required_command) - in_partition.emplace(version, status->entry); + /// We take commands with bigger metadata version + if (alter_version > params.min_part_metadata_version) + { + if (std::ranges::any_of(status->entry->commands, is_supported_command)) + in_partition.emplace(mutation_version, status->entry); + } + else + { + seen_all_metadata_mutations = true; + } + } + else if (!seen_all_data_mutations) + { + if (!status->is_done) + { + if (std::ranges::any_of(status->entry->commands, is_supported_command)) + in_partition.emplace(mutation_version, status->entry); + } + else + { + seen_all_data_mutations = true; + } + } } } @@ -2080,7 +2113,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2137,7 +2170,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index f9d5487ee3f..954e2fd951e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,8 +154,7 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_alter_conversions; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; @@ -416,11 +415,11 @@ public: struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot { MutationsSnapshot() = default; + explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} - Int64 metadata_version = -1; - bool need_data_mutations = false; - + using Params = MergeTreeData::IMutationsSnapshot::Params; using MutationsByPartititon = std::unordered_map>; + MutationsByPartititon mutations_by_partition; MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; @@ -430,7 +429,7 @@ public: /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, /// without actual data modification on disk. - MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const; + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(const MutationsSnapshot::Params & params) const; /// Mark finished mutations as done. If the function needs to be called again at some later time /// (because some mutations are probably done but we are not sure yet), returns true. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 636d2ba5d53..da90ffddc8d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -502,7 +502,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + incrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -538,7 +538,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, *entry.commands, lock); } } else @@ -819,7 +819,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); } to_kill.emplace(std::move(it->second)); @@ -904,7 +904,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); + incrementAlterConversionsCounter(num_alter_conversions, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2432,7 +2432,7 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo for (const auto & command : *commands | std::views::reverse) { - if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) result.push_back(command); else if (AlterConversions::isSupportedMetadataMutation(command.type)) result.push_back(command); @@ -2442,28 +2442,26 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo return result; } -MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - auto res = std::make_shared(); - res->metadata_version = metadata_version; - res->need_data_mutations = need_data_mutations; + auto res = std::make_shared(params); std::lock_guard lock(currently_processing_in_background_mutex); - bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; - bool have_metadata_mutations = metadata_mutations_to_apply > 0; + bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; + bool need_metatadata_mutations = res->params.needMetadataMutations(); - if (!have_data_mutations && !have_metadata_mutations) + if (!need_data_mutations && !need_metatadata_mutations) return res; for (const auto & [version, entry] : current_mutations_by_version) { bool has_required_command = std::ranges::any_of(*entry.commands, [&](const auto & command) { - if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) return true; - if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) return true; return false; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 1f2af8b9571..d05c6739738 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,8 +150,7 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_alter_conversions; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -315,9 +314,7 @@ private: struct MutationsSnapshot : public IMutationsSnapshot { MutationsSnapshot() = default; - - Int64 metadata_version = -1; - bool need_data_mutations = false; + explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} using MutationsByVersion = std::map>; MutationsByVersion mutations_by_version; @@ -326,7 +323,7 @@ private: std::shared_ptr cloneEmpty() const override { return std::make_shared(); } }; - MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; + MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e2bba1e8068..baca8cb2695 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9149,9 +9149,9 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } -MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - return queue.getMutationsSnapshot(metadata_version, need_data_mutations); + return queue.getMutationsSnapshot(params); } void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3ef367d09ce..cd12cfd3c02 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -932,7 +932,7 @@ private: void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; + MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; void startBackgroundMovesIfNeeded() override; From a99c803ddf137e91f43c8f26f549f74f71eab102 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 17:26:06 +0800 Subject: [PATCH 0211/1722] Rename rows_before_group_by_at_least to rows_before_aggregation_at_least --- src/Client/ClientBase.cpp | 4 +- src/Core/Settings.h | 2 +- src/Formats/JSONUtils.cpp | 9 +- src/Formats/JSONUtils.h | 2 + src/Processors/Formats/IOutputFormat.cpp | 4 +- src/Processors/Formats/IOutputFormat.h | 15 +- ...ONColumnsWithMetadataBlockOutputFormat.cpp | 2 + ...JSONColumnsWithMetadataBlockOutputFormat.h | 5 + .../Formats/Impl/JSONRowOutputFormat.cpp | 2 + .../Formats/Impl/JSONRowOutputFormat.h | 5 + .../Impl/ParallelFormattingOutputFormat.h | 6 + .../Impl/TemplateBlockOutputFormat.cpp | 17 +- .../Formats/Impl/TemplateBlockOutputFormat.h | 8 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 11 + .../Formats/Impl/XMLRowOutputFormat.h | 6 + src/Processors/Formats/LazyOutputFormat.cpp | 4 +- src/Processors/Formats/LazyOutputFormat.h | 2 +- .../Formats/PullingOutputFormat.cpp | 4 +- src/Processors/Formats/PullingOutputFormat.h | 2 +- src/Processors/IProcessor.h | 6 +- src/Processors/RowsBeforeLimitCounter.h | 2 +- src/Processors/Sources/DelayedSource.cpp | 6 + src/Processors/Sources/DelayedSource.h | 4 +- src/Processors/Sources/RemoteSource.cpp | 12 +- src/Processors/Sources/RemoteSource.h | 8 +- .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/AggregatingTransform.h | 7 +- src/QueryPipeline/ProfileInfo.cpp | 20 +- src/QueryPipeline/ProfileInfo.h | 16 +- src/QueryPipeline/QueryPipeline.cpp | 21 +- src/Server/GRPCServer.cpp | 3 +- src/Server/grpc_protos/clickhouse_grpc.proto | 4 +- ...74_exact_rows_before_aggregation.reference | 355 ++++++++++++++++++ .../03174_exact_rows_before_aggregation.sql | 31 ++ 34 files changed, 538 insertions(+), 71 deletions(-) create mode 100644 tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference create mode 100644 tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index dbb67d230d5..4b82f30776f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -522,8 +522,8 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) { if (profile_info.hasAppliedLimit() && output_format) output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); - if (profile_info.hasAppliedGroupBy() && output_format) - output_format->setRowsBeforeGroupBy(profile_info.getRowsBeforeGroupBy()); + if (profile_info.hasAppliedAggregation() && output_format) + output_format->setRowsBeforeAggregation(profile_info.getRowsBeforeAggregation()); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 09291d4300d..2296a880bd6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1200,7 +1200,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ - M(Bool, exact_rows_before_group_by, false, "When enabled, ClickHouse will provide exact value for rows_before_group_by_at_least statistic, but with the cost that the data before group by will have to be read completely", 0) \ + M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation_at_least statistic, represents the number of rows read before aggregation", 0) \ M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index f0985f4a6b7..363e9344770 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -487,6 +487,8 @@ namespace JSONUtils size_t rows, size_t rows_before_limit, bool applied_limit, + size_t rows_before_aggregation, + bool applied_aggregation, const Stopwatch & watch, const Progress & progress, bool write_statistics, @@ -502,7 +504,12 @@ namespace JSONUtils writeTitle("rows_before_limit_at_least", out, 1, " "); writeIntText(rows_before_limit, out); } - + if (applied_aggregation) + { + writeFieldDelimiter(out, 2); + writeTitle("rows_before_aggregation_at_least", out, 1, " "); + writeIntText(rows_before_aggregation, out); + } if (write_statistics) { writeFieldDelimiter(out, 2); diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index 7ee111c1285..e2ac3467971 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -104,6 +104,8 @@ namespace JSONUtils size_t rows, size_t rows_before_limit, bool applied_limit, + size_t rows_before_aggregation, + bool applied_aggregation, const Stopwatch & watch, const Progress & progress, bool write_statistics, diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 4191bf9f0fe..7eaecfab3dc 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -71,8 +71,8 @@ void IOutputFormat::work() { if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) setRowsBeforeLimit(rows_before_limit_counter->get()); - if (rows_before_group_by_counter && rows_before_group_by_counter->hasAppliedLimit()) - setRowsBeforeGroupBy(rows_before_group_by_counter->get()); + if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedLimit()) + setRowsBeforeAggregation(rows_before_aggregation_counter->get()); finalize(); if (auto_flush) flush(); diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 9bb7cccb612..1119797e7ff 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -36,16 +36,16 @@ public: void setAutoFlush() { auto_flush = true; } /// Value for rows_before_limit_at_least field. - virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} + virtual void setRowsBeforeLimit(size_t /*rows*/) { } /// Counter to calculate rows_before_limit_at_least in processors pipeline. void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } - /// Value for rows_before_group_by_at_least field. - virtual void setRowsBeforeGroupBy(size_t /*rows_before_limit*/) { } + /// Value for rows_before_aggregation_at_least field. + virtual void setRowsBeforeAggregation(size_t /*rows*/) { } - /// Counter to calculate rows_before_group_by_at_least in processors pipeline. - void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by_counter.swap(counter); } + /// Counter to calculate rows_before_aggregation_at_least in processors pipeline. + void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. @@ -157,7 +157,8 @@ protected: Progress progress; bool applied_limit = false; size_t rows_before_limit = 0; - size_t rows_before_group_by = 0; + bool applied_aggregation = false; + size_t rows_before_aggregation = 0; Chunk totals; Chunk extremes; }; @@ -192,7 +193,7 @@ protected: bool need_write_suffix = true; RowsBeforeLimitCounterPtr rows_before_limit_counter; - RowsBeforeGroupByCounterPtr rows_before_group_by_counter; + RowsBeforeAggregationCounterPtr rows_before_aggregation_counter; Statistics statistics; private: diff --git a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp index 1e8f57aa9a6..2f285e3d202 100644 --- a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.cpp @@ -81,6 +81,8 @@ void JSONColumnsWithMetadataBlockOutputFormat::finalizeImpl() rows, statistics.rows_before_limit, statistics.applied_limit, + statistics.rows_before_aggregation, + statistics.applied_aggregation, statistics.watch, statistics.progress, format_settings.write_statistics, diff --git a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h index c72b4d87234..e5208440483 100644 --- a/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONColumnsWithMetadataBlockOutputFormat.h @@ -44,6 +44,11 @@ public: String getName() const override { return "JSONCompactColumnsBlockOutputFormat"; } void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.rows_before_aggregation = rows_before_aggregation_; + statistics.applied_aggregation = true; + } void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); } protected: diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 20182d84917..fec24b10c11 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -116,6 +116,8 @@ void JSONRowOutputFormat::finalizeImpl() row_count, statistics.rows_before_limit, statistics.applied_limit, + statistics.rows_before_aggregation, + statistics.applied_aggregation, statistics.watch, statistics.progress, settings.write_statistics && exception_message.empty(), diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index a38cd0e8db9..c36adb5ee3e 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -35,6 +35,11 @@ public: statistics.applied_limit = true; statistics.rows_before_limit = rows_before_limit_; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.applied_aggregation = true; + statistics.rows_before_aggregation = rows_before_aggregation_; + } protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 341141dd633..66f8701161e 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -313,6 +313,12 @@ private: statistics.rows_before_limit = rows_before_limit; statistics.applied_limit = true; } + void setRowsBeforeAggregation(size_t rows_before_aggregation) override + { + std::lock_guard lock(statistics_mutex); + statistics.rows_before_aggregation = rows_before_aggregation; + statistics.applied_aggregation = true; + } }; } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index 1c43a0fa331..4bd6684dff4 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -42,9 +42,11 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ case static_cast(ResultsetPart::TimeElapsed): case static_cast(ResultsetPart::RowsRead): case static_cast(ResultsetPart::BytesRead): + case static_cast(ResultsetPart::RowsBeforeAggregation): if (format.escaping_rules[i] == EscapingRule::None) - format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, " - "rows_read or bytes_read is not specified", i); + format.throwInvalidFormat( + "Serialization type for output part rows, rows, time, " + "rows_read or bytes_read is not specified", i); break; default: format.throwInvalidFormat("Invalid output part", i); @@ -80,7 +82,7 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu return ResultsetPart::ExtremesMax; else if (part == "rows") return ResultsetPart::Rows; - else if (part == "rows_before_limit") + else if (part == "rows") return ResultsetPart::RowsBeforeLimit; else if (part == "time") return ResultsetPart::TimeElapsed; @@ -88,6 +90,8 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu return ResultsetPart::RowsRead; else if (part == "bytes_read") return ResultsetPart::BytesRead; + else if (part == "rows_before_aggregation") + return ResultsetPart::RowsBeforeAggregation; else throw Exception(ErrorCodes::SYNTAX_ERROR, "Unknown output part {}", part); } @@ -161,7 +165,7 @@ void TemplateBlockOutputFormat::finalizeImpl() break; case ResultsetPart::RowsBeforeLimit: if (!statistics.applied_limit) - format.throwInvalidFormat("Cannot print rows_before_limit for this request", i); + format.throwInvalidFormat("Cannot print rows for this request", i); writeValue(statistics.rows_before_limit, format.escaping_rules[i]); break; case ResultsetPart::TimeElapsed: @@ -173,6 +177,11 @@ void TemplateBlockOutputFormat::finalizeImpl() case ResultsetPart::BytesRead: writeValue(statistics.progress.read_bytes.load(), format.escaping_rules[i]); break; + case ResultsetPart::RowsBeforeAggregation: + if (!statistics.applied_aggregation) + format.throwInvalidFormat("Cannot print rows_before_aggregation for this request", i); + writeValue(statistics.rows_before_aggregation, format.escaping_rules[i]); + break; default: break; } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 53d98849482..5e88d79b4a8 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -21,6 +21,11 @@ public: String getName() const override { return "TemplateBlockOutputFormat"; } void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.rows_before_aggregation = rows_before_aggregation_; + statistics.applied_aggregation = true; + } void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); } enum class ResultsetPart : size_t @@ -33,7 +38,8 @@ public: RowsBeforeLimit, TimeElapsed, RowsRead, - BytesRead + BytesRead, + RowsBeforeAggregation }; static ResultsetPart stringToResultsetPart(const String & part); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 52c161c3208..2fd0536ed02 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -191,6 +191,7 @@ void XMLRowOutputFormat::finalizeImpl() writeRowsBeforeLimitAtLeast(); + writeRowsBeforeAggregationAtLeast(); if (!exception_message.empty()) writeException(); @@ -219,6 +220,16 @@ void XMLRowOutputFormat::writeRowsBeforeLimitAtLeast() } } +void XMLRowOutputFormat::writeRowsBeforeAggregationAtLeast() +{ + if (statistics.applied_aggregation) + { + writeCString("\t", *ostr); + writeIntText(statistics.rows_before_aggregation, *ostr); + writeCString("\n", *ostr); + } +} + void XMLRowOutputFormat::writeStatistics() { writeCString("\t\n", *ostr); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index daf03539d0b..792acd118c8 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -48,6 +48,11 @@ private: statistics.rows_before_limit = rows_before_limit_; } + void setRowsBeforeAggregation(size_t rows_before_aggregation_) override + { + statistics.applied_aggregation = true; + statistics.rows_before_aggregation = rows_before_aggregation_; + } void onRowsReadBeforeUpdate() override { row_count = getRowsReadBefore(); } void onProgress(const Progress & value) override; @@ -56,6 +61,7 @@ private: void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); void writeRowsBeforeLimitAtLeast(); + void writeRowsBeforeAggregationAtLeast(); void writeStatistics(); void writeException(); diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 63423628e57..dc099765870 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -45,8 +45,8 @@ void LazyOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) info.setRowsBeforeLimit(rows_before_limit); } -void LazyOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +void LazyOutputFormat::setRowsBeforeAggregation(size_t rows_before_aggregation) { - info.setRowsBeforeGroupBy(rows_before_group_by); + info.setRowsBeforeAggregation(rows_before_aggregation); } } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 83abb2ff1a1..b0936e3d258 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -28,7 +28,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; - void setRowsBeforeGroupBy(size_t rows_before_group_by) override; + void setRowsBeforeAggregation(size_t rows_before_aggregation) override; void onCancel() override { diff --git a/src/Processors/Formats/PullingOutputFormat.cpp b/src/Processors/Formats/PullingOutputFormat.cpp index 646755deb6b..37050fb9675 100644 --- a/src/Processors/Formats/PullingOutputFormat.cpp +++ b/src/Processors/Formats/PullingOutputFormat.cpp @@ -42,8 +42,8 @@ void PullingOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) { info.setRowsBeforeLimit(rows_before_limit); } -void PullingOutputFormat::setRowsBeforeGroupBy(size_t rows_before_group_by) +void PullingOutputFormat::setRowsBeforeAggregation(size_t rows_before_aggregation) { - info.setRowsBeforeGroupBy(rows_before_group_by); + info.setRowsBeforeAggregation(rows_before_aggregation); } } diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index c4d8cf4aab2..f2546cca180 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -22,7 +22,7 @@ public: ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; - void setRowsBeforeGroupBy(size_t rows_before_group_by) override; + void setRowsBeforeAggregation(size_t rows_before_aggregation) override; bool expectMaterializedColumns() const override { return false; } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 0df4b3168e3..a06958fed73 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -24,7 +24,7 @@ using StorageLimitsList = std::list; class RowsBeforeLimitCounter; using RowsBeforeLimitCounterPtr = std::shared_ptr; -using RowsBeforeGroupByCounterPtr = std::shared_ptr; +using RowsBeforeAggregationCounterPtr = std::shared_ptr; class IProcessor; using ProcessorPtr = std::shared_ptr; @@ -368,9 +368,9 @@ public: /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} - /// Set rows_before_group_by counter for current processor. + /// Set rows_before_aggregation counter for current processor. /// This counter is used to calculate the number of rows right before AggregatingTransform. - virtual void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr /* counter */) { } + virtual void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr /* counter */) { } protected: virtual void onCancel() {} diff --git a/src/Processors/RowsBeforeLimitCounter.h b/src/Processors/RowsBeforeLimitCounter.h index f5eb40ff84a..5aa867ffc43 100644 --- a/src/Processors/RowsBeforeLimitCounter.h +++ b/src/Processors/RowsBeforeLimitCounter.h @@ -5,7 +5,7 @@ namespace DB { -/// This class helps to calculate rows_before_limit_at_least. +/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. class RowsBeforeLimitCounter { public: diff --git a/src/Processors/Sources/DelayedSource.cpp b/src/Processors/Sources/DelayedSource.cpp index f7928f89015..788017e3df0 100644 --- a/src/Processors/Sources/DelayedSource.cpp +++ b/src/Processors/Sources/DelayedSource.cpp @@ -139,6 +139,12 @@ void DelayedSource::work() processor->setRowsBeforeLimitCounter(rows_before_limit); } + if (rows_before_aggregation) + { + for (auto & processor : processors) + processor->setRowsBeforeAggregationCounter(rows_before_aggregation); + } + synchronizePorts(totals_output, totals, header, processors); synchronizePorts(extremes_output, extremes, header, processors); } diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index bd100f29a47..3138a1ab42a 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -31,14 +31,14 @@ public: OutputPort * getExtremesPort() { return extremes; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: QueryPlanResourceHolder resources; Creator creator; Processors processors; RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_group_by; + RowsBeforeLimitCounterPtr rows_before_aggregation; /// Outputs for DelayedSource. OutputPort * main = nullptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index a78db630786..9a6fe239ee6 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -46,12 +46,12 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit } - if (rows_before_group_by) + if (rows_before_aggregation) { - if (info.hasAppliedGroupBy()) - rows_before_group_by->add(info.getRowsBeforeGroupBy()); + if (info.hasAppliedAggregation()) + rows_before_aggregation->add(info.getRowsBeforeAggregation()); else - manually_add_rows_before_group_by_counter = true; /// Remote subquery doesn't contain a group by + manually_add_rows_before_aggregation_counter = true; /// Remote subquery doesn't contain a group by } }); } @@ -171,8 +171,8 @@ std::optional RemoteSource::tryGenerate() { if (manually_add_rows_before_limit_counter) rows_before_limit->add(rows); - if (manually_add_rows_before_group_by_counter) - rows_before_group_by->add(rows); + if (manually_add_rows_before_aggregation_counter) + rows_before_aggregation->add(rows); query_executor->finish(); return {}; } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index b2ea6d50e01..c2f4d6842bb 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -3,8 +3,8 @@ #include #include #include -#include +#include namespace DB { @@ -26,7 +26,7 @@ public: String getName() const override { return "Remote"; } void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeGroupByCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_group_by.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -46,7 +46,7 @@ private: bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_group_by; + RowsBeforeLimitCounterPtr rows_before_aggregation; const bool async_read; const bool async_query_sending; @@ -54,7 +54,7 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; - bool manually_add_rows_before_group_by_counter = false; + bool manually_add_rows_before_aggregation_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 7c0e222f89b..684de0a3e8c 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -683,8 +683,8 @@ void AggregatingTransform::consume(Chunk chunk) LOG_TRACE(log, "Aggregating"); is_consume_started = true; } - if (rows_before_group_by_at_least) - rows_before_group_by_at_least->add(num_rows); + if (rows_before_aggregation_at_least) + rows_before_aggregation_at_least->add(num_rows); src_rows += num_rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 6e7b04f9191..53939ea6a99 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -169,7 +169,10 @@ public: Status prepare() override; void work() override; Processors expandPipeline() override; - void setRowsBeforeGroupByCounter(RowsBeforeGroupByCounterPtr counter) override { rows_before_group_by_at_least.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr counter) override + { + rows_before_aggregation_at_least.swap(counter); + } protected: void consume(Chunk chunk); @@ -213,7 +216,7 @@ private: bool is_consume_started = false; - RowsBeforeGroupByCounterPtr rows_before_group_by_at_least; + RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least; void initGenerate(); }; diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index cec179ecfad..87729b7c90e 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -16,8 +16,8 @@ void ProfileInfo::read(ReadBuffer & in) readBinary(applied_limit, in); readVarUInt(rows_before_limit, in); readBinary(calculated_rows_before_limit, in); - readBinary(applied_group_by, in); - readVarUInt(rows_before_group_by, in); + readBinary(applied_aggregation, in); + readVarUInt(rows_before_aggregation, in); } @@ -29,8 +29,8 @@ void ProfileInfo::write(WriteBuffer & out) const writeBinary(hasAppliedLimit(), out); writeVarUInt(getRowsBeforeLimit(), out); writeBinary(calculated_rows_before_limit, out); - writeBinary(hasAppliedGroupBy(), out); - writeVarUInt(getRowsBeforeGroupBy(), out); + writeBinary(hasAppliedAggregation(), out); + writeVarUInt(getRowsBeforeAggregation(), out); } @@ -45,8 +45,8 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) applied_limit = rhs.applied_limit; rows_before_limit = rhs.rows_before_limit; calculated_rows_before_limit = rhs.calculated_rows_before_limit; - applied_group_by = rhs.applied_group_by; - rows_before_group_by = rhs.rows_before_group_by; + applied_aggregation = rhs.applied_aggregation; + rows_before_aggregation = rhs.rows_before_aggregation; } @@ -63,15 +63,15 @@ bool ProfileInfo::hasAppliedLimit() const return applied_limit; } -size_t ProfileInfo::getRowsBeforeGroupBy() const +size_t ProfileInfo::getRowsBeforeAggregation() const { - return rows_before_group_by; + return rows_before_aggregation; } -bool ProfileInfo::hasAppliedGroupBy() const +bool ProfileInfo::hasAppliedAggregation() const { - return applied_group_by; + return applied_aggregation; } diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index 141adc7430d..e2467afd6f4 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -32,8 +32,8 @@ struct ProfileInfo size_t getRowsBeforeLimit() const; bool hasAppliedLimit() const; - size_t getRowsBeforeGroupBy() const; - bool hasAppliedGroupBy() const; + size_t getRowsBeforeAggregation() const; + bool hasAppliedAggregation() const; void update(Block & block); void update(size_t num_rows, size_t num_bytes); @@ -55,20 +55,20 @@ struct ProfileInfo } /// Only for Processors. - void setRowsBeforeGroupBy(size_t rows_before_group_by_) + void setRowsBeforeAggregation(size_t rows_before_aggregation_) { - applied_group_by = true; - rows_before_group_by = rows_before_group_by_; + applied_aggregation = true; + rows_before_aggregation = rows_before_aggregation_; } private: /// For these fields we make accessors, because they must be calculated beforehand. mutable bool applied_limit = false; /// Whether LIMIT was applied mutable size_t rows_before_limit = 0; - mutable bool calculated_rows_before_limit = false; /// Whether the field rows_before_limit was calculated + mutable bool calculated_rows_before_limit = false; /// Whether the field rows was calculated - mutable bool applied_group_by = false; /// Whether GROUP BY was applied - mutable size_t rows_before_group_by = 0; + mutable bool applied_aggregation = false; /// Whether GROUP BY was applied + mutable size_t rows_before_aggregation = 0; }; } diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 5e0885ed4e8..0e3eec21d50 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -274,18 +274,20 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } } -static void initRowsBeforeGroupBy(std::shared_ptr processors, IOutputFormat * output_format) +static void initRowsBeforeAggregation(std::shared_ptr processors, IOutputFormat * output_format) { if (!processors->empty()) { - RowsBeforeGroupByCounterPtr rows_before_group_by_at_least = std::make_shared(); + RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); for (auto & processor : *processors) { if (auto transform = std::dynamic_pointer_cast(processor)) - transform->setRowsBeforeGroupByCounter(rows_before_group_by_at_least); + transform->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + if (auto remote = std::dynamic_pointer_cast(processor)) + remote->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } - rows_before_group_by_at_least->add(0); - output_format->setRowsBeforeLimitCounter(rows_before_group_by_at_least); + rows_before_aggregation_at_least->add(0); + output_format->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } } @@ -535,7 +537,14 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - initRowsBeforeGroupBy(processors, format.get()); + for (const auto context : resources.interpreter_context) + { + if (context->getSettingsRef().rows_before_aggregation) + { + initRowsBeforeAggregation(processors, format.get()); + break; + } + } output_format = format.get(); processors->emplace_back(std::move(format)); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 37e4342f3b0..9651ce8f660 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1577,7 +1577,8 @@ namespace stats.set_allocated_bytes(info.bytes); stats.set_applied_limit(info.hasAppliedLimit()); stats.set_rows_before_limit(info.getRowsBeforeLimit()); - stats.set_rows_before_group_by(info.getRowsBeforeGroupBy()); + stats.set_applied_aggregation(info.hasAppliedAggregation()); + stats.set_rows_before_aggregation(info.getRowsBeforeAggregation()); } void Call::addLogsToResult() diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index 02b6988b8c0..2ada4e8a641 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -179,8 +179,8 @@ message Stats { uint64 allocated_bytes = 3; bool applied_limit = 4; uint64 rows_before_limit = 5; - bool applied_group_by = 6; - uint64 rows_before_group_by = 7; + bool applied_aggregation = 6; + uint64 rows_before_aggregation = 7; } message Exception { diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference new file mode 100644 index 00000000000..8ad00273bbf --- /dev/null +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -0,0 +1,355 @@ +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9] + ], + + "rows": 10, + + "rows_before_aggregation_at_least": 10 +} + + + + + + i + Int32 + + + + + + 0 + + + 1 + + + 2 + + + 3 + + + 4 + + + 5 + + + 6 + + + 7 + + + 8 + + + 9 + + + 10 + 10 + +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [10], + [11], + [12] + ], + + "rows": 3, + + "rows_before_aggregation_at_least": 3 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0] + ], + + "rows": 1, + + "rows_before_limit_at_least": 20, + + "rows_before_aggregation_at_least": 20 +} +{ + "meta": + [ + { + "name": "max(i)", + "type": "Int32" + } + ], + + "data": + [ + [19] + ], + + "rows": 1, + + "rows_before_aggregation_at_least": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 60, + + "rows_before_aggregation_at_least": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19] + ], + + "rows": 20, + + "rows_before_limit_at_least": 40, + + "rows_before_aggregation_at_least": 40 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 30, + + "rows_before_aggregation_at_least": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19] + ], + + "rows": 20, + + "rows_before_limit_at_least": 20, + + "rows_before_aggregation_at_least": 40 +} +{ + "meta": + [ + { + "name": "max(i)", + "type": "Int32" + } + ], + + "data": + [ + [19] + ], + + "rows": 1, + + "rows_before_limit_at_least": 1, + + "rows_before_aggregation_at_least": 40 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [4], + [3], + [2], + [5], + [1], + [6], + [7], + [9], + [8] + ], + + "rows": 10, + + "rows_before_limit_at_least": 10, + + "rows_before_aggregation_at_least": 20 +} diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql new file mode 100644 index 00000000000..6a3759c11e8 --- /dev/null +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -0,0 +1,31 @@ +-- Tags: no-parallel, no-random-merge-tree-settings + +drop table if exists test; + +create table test (i int) engine MergeTree order by tuple(); + +insert into test select arrayJoin(range(10000)); + +set rows_before_aggregation = 1, output_format_write_statistics = 0, max_block_size = 100; + +select * from test where i < 10 group by i order by i FORMAT JSONCompact; +select * from test where i < 10 group by i order by i FORMAT XML; + +select * from test group by i having i in (10, 11, 12) order by i FORMAT JSONCompact; + +select * from test where i < 20 group by i order by i limit 1 FORMAT JSONCompact; + +select max(i) from test where i < 20 FORMAT JSONCompact; + +set prefer_localhost_replica = 0; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; + +set prefer_localhost_replica = 1; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; +select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; + +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i FORMAT JSONCompact; + +drop table if exists test; \ No newline at end of file From 92ce1368b3aa2056e05a89ad013d7fa818f9888d Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 18:03:40 +0800 Subject: [PATCH 0212/1722] Rename RowsBeforeLimitCounter to RowsBeforeStepCounter --- src/Processors/Formats/IOutputFormat.cpp | 4 +-- src/Processors/Formats/IOutputFormat.h | 4 +-- src/Processors/IProcessor.h | 6 ++-- src/Processors/LimitTransform.h | 4 +-- src/Processors/OffsetTransform.h | 4 +-- src/Processors/RowsBeforeLimitCounter.h | 36 ------------------- src/Processors/RowsBeforeStepCounter.h | 36 +++++++++++++++++++ src/Processors/Sources/RemoteSource.h | 2 +- .../Transforms/AggregatingTransform.h | 2 +- .../Transforms/PartialSortingTransform.h | 4 +-- src/QueryPipeline/QueryPipeline.cpp | 4 +-- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...74_exact_rows_before_aggregation.reference | 27 +++++++++++--- .../03174_exact_rows_before_aggregation.sql | 9 +++-- 14 files changed, 81 insertions(+), 63 deletions(-) delete mode 100644 src/Processors/RowsBeforeLimitCounter.h create mode 100644 src/Processors/RowsBeforeStepCounter.h diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 7eaecfab3dc..97628778adb 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -69,9 +69,9 @@ void IOutputFormat::work() if (finished && !finalized) { - if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) + if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedStep()) setRowsBeforeLimit(rows_before_limit_counter->get()); - if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedLimit()) + if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedStep()) setRowsBeforeAggregation(rows_before_aggregation_counter->get()); finalize(); if (auto_flush) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 1119797e7ff..721e02f1c1d 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -1,9 +1,9 @@ #pragma once #include -#include -#include #include +#include +#include #include namespace DB diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index a06958fed73..ccdd6308de5 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -21,10 +21,10 @@ class IQueryPlanStep; struct StorageLimits; using StorageLimitsList = std::list; -class RowsBeforeLimitCounter; -using RowsBeforeLimitCounterPtr = std::shared_ptr; +class RowsBeforeStepCounter; +using RowsBeforeLimitCounterPtr = std::shared_ptr; -using RowsBeforeAggregationCounterPtr = std::shared_ptr; +using RowsBeforeAggregationCounterPtr = std::shared_ptr; class IProcessor; using ProcessorPtr = std::shared_ptr; diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 33ff968985f..515203f6829 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -1,8 +1,8 @@ #pragma once -#include -#include #include +#include +#include namespace DB { diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h index 79a7d15fe0b..7ef16518540 100644 --- a/src/Processors/OffsetTransform.h +++ b/src/Processors/OffsetTransform.h @@ -1,8 +1,8 @@ #pragma once -#include -#include #include +#include +#include namespace DB { diff --git a/src/Processors/RowsBeforeLimitCounter.h b/src/Processors/RowsBeforeLimitCounter.h deleted file mode 100644 index 5aa867ffc43..00000000000 --- a/src/Processors/RowsBeforeLimitCounter.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. -class RowsBeforeLimitCounter -{ -public: - void add(uint64_t rows) - { - setAppliedLimit(); - rows_before_limit.fetch_add(rows, std::memory_order_release); - } - - void set(uint64_t rows) - { - setAppliedLimit(); - rows_before_limit.store(rows, std::memory_order_release); - } - - uint64_t get() const { return rows_before_limit.load(std::memory_order_acquire); } - - void setAppliedLimit() { has_applied_limit.store(true, std::memory_order_release); } - bool hasAppliedLimit() const { return has_applied_limit.load(std::memory_order_acquire); } - -private: - std::atomic rows_before_limit = 0; - std::atomic_bool has_applied_limit = false; -}; - -using RowsBeforeLimitCounterPtr = std::shared_ptr; - -} diff --git a/src/Processors/RowsBeforeStepCounter.h b/src/Processors/RowsBeforeStepCounter.h new file mode 100644 index 00000000000..d9912bfa076 --- /dev/null +++ b/src/Processors/RowsBeforeStepCounter.h @@ -0,0 +1,36 @@ +#pragma once +#include +#include + +namespace DB +{ + +/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. +class RowsBeforeStepCounter +{ +public: + void add(uint64_t rows) + { + setAppliedStep(); + rows_before_step.fetch_add(rows, std::memory_order_release); + } + + void set(uint64_t rows) + { + setAppliedStep(); + rows_before_step.store(rows, std::memory_order_release); + } + + uint64_t get() const { return rows_before_step.load(std::memory_order_acquire); } + + void setAppliedStep() { has_applied_step.store(true, std::memory_order_release); } + bool hasAppliedStep() const { return has_applied_step.load(std::memory_order_acquire); } + +private: + std::atomic rows_before_step = 0; + std::atomic_bool has_applied_step = false; +}; + +using RowsBeforeLimitCounterPtr = std::shared_ptr; + +} diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index c2f4d6842bb..bbc563ec5fe 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 53939ea6a99..9f9638175f0 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index 8f25c93037f..abb4b290322 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -1,7 +1,7 @@ #pragma once -#include -#include #include +#include +#include #include namespace DB diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 0e3eec21d50..acccbed2585 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -262,7 +262,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) if (!processors.empty()) { - rows_before_limit_at_least = std::make_shared(); + rows_before_limit_at_least = std::make_shared(); for (auto & processor : processors) processor->setRowsBeforeLimitCounter(rows_before_limit_at_least); @@ -278,7 +278,7 @@ static void initRowsBeforeAggregation(std::shared_ptr processors, IO { if (!processors->empty()) { - RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); + RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); for (auto & processor : *processors) { if (auto transform = std::dynamic_pointer_cast(processor)) diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 803d1686ad7..d276fed60a2 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference index 8ad00273bbf..36db9721599 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -101,12 +101,29 @@ "data": [ - [0] + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19] ], - "rows": 1, - - "rows_before_limit_at_least": 20, + "rows": 20, "rows_before_aggregation_at_least": 20 } @@ -126,6 +143,8 @@ "rows": 1, + "rows_before_limit_at_least": 1, + "rows_before_aggregation_at_least": 20 } { diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 6a3759c11e8..8ccbce42706 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -6,16 +6,15 @@ create table test (i int) engine MergeTree order by tuple(); insert into test select arrayJoin(range(10000)); -set rows_before_aggregation = 1, output_format_write_statistics = 0, max_block_size = 100; +set rows_before_aggregation = 1, exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; select * from test where i < 10 group by i order by i FORMAT JSONCompact; select * from test where i < 10 group by i order by i FORMAT XML; select * from test group by i having i in (10, 11, 12) order by i FORMAT JSONCompact; -select * from test where i < 20 group by i order by i limit 1 FORMAT JSONCompact; - -select max(i) from test where i < 20 FORMAT JSONCompact; +select * from test where i < 20 group by i order by i FORMAT JSONCompact; +select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; set prefer_localhost_replica = 0; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; @@ -26,6 +25,6 @@ select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; -select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; drop table if exists test; \ No newline at end of file From 355f144cda1838acce8f89f54e3ae84300263ea3 Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 18:11:26 +0800 Subject: [PATCH 0213/1722] Fix rename bug --- src/Processors/Formats/IOutputFormat.h | 4 ++-- src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 721e02f1c1d..40ac1317618 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -36,13 +36,13 @@ public: void setAutoFlush() { auto_flush = true; } /// Value for rows_before_limit_at_least field. - virtual void setRowsBeforeLimit(size_t /*rows*/) { } + virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) { } /// Counter to calculate rows_before_limit_at_least in processors pipeline. void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } /// Value for rows_before_aggregation_at_least field. - virtual void setRowsBeforeAggregation(size_t /*rows*/) { } + virtual void setRowsBeforeAggregation(size_t /*rows_before_limit*/) { } /// Counter to calculate rows_before_aggregation_at_least in processors pipeline. void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index 4bd6684dff4..5d6db17aaa2 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -82,7 +82,7 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu return ResultsetPart::ExtremesMax; else if (part == "rows") return ResultsetPart::Rows; - else if (part == "rows") + else if (part == "rows_before_limit") return ResultsetPart::RowsBeforeLimit; else if (part == "time") return ResultsetPart::TimeElapsed; @@ -165,7 +165,7 @@ void TemplateBlockOutputFormat::finalizeImpl() break; case ResultsetPart::RowsBeforeLimit: if (!statistics.applied_limit) - format.throwInvalidFormat("Cannot print rows for this request", i); + format.throwInvalidFormat("Cannot print rows_before_limit for this request", i); writeValue(statistics.rows_before_limit, format.escaping_rules[i]); break; case ResultsetPart::TimeElapsed: From 5f42e1518277e54c34fce274b5cf46cd5069cb2a Mon Sep 17 00:00:00 2001 From: morning-color Date: Thu, 4 Jul 2024 19:58:22 +0800 Subject: [PATCH 0214/1722] Fix Bug --- src/QueryPipeline/QueryPipeline.cpp | 14 ++++++++++---- .../03174_exact_rows_before_aggregation.sql | 2 +- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index acccbed2585..311c8a60531 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -276,17 +276,23 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) } static void initRowsBeforeAggregation(std::shared_ptr processors, IOutputFormat * output_format) { + bool has_aggregation = false; + if (!processors->empty()) { RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); - for (auto & processor : *processors) + for (auto processor : *processors) { if (auto transform = std::dynamic_pointer_cast(processor)) + { transform->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); - if (auto remote = std::dynamic_pointer_cast(processor)) - remote->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + has_aggregation = true; + } + if (typeid_cast(processor.get()) || typeid_cast(processor.get())) + processor->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } - rows_before_aggregation_at_least->add(0); + if (has_aggregation) + rows_before_aggregation_at_least->add(0); output_format->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); } } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 8ccbce42706..31b817e8a65 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -27,4 +27,4 @@ select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) whe select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; -drop table if exists test; \ No newline at end of file +drop table if exists test; From 4e350ff44a4dbdecb1c044fb10df131f6c31ae3d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Jul 2024 18:48:41 +0200 Subject: [PATCH 0215/1722] don't allow explicit uuids and rmt args in replicated db --- src/Core/Settings.h | 3 +- src/Databases/DatabaseReplicated.cpp | 12 ----- src/Interpreters/InterpreterCreateQuery.cpp | 20 ++++++++ ...tractZooKeeperPathFromReplicatedTableDef.h | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 46 ++++++++++++++----- tests/config/users.d/database_replicated.xml | 1 + .../configs/settings.xml | 1 + .../configs/settings2.xml | 1 + .../02858_explicit_uuid_and_zk_path.reference | 10 ++++ .../02858_explicit_uuid_and_zk_path.sh | 41 +++++++++++++++++ 10 files changed, 112 insertions(+), 25 deletions(-) create mode 100644 tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference create mode 100755 tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d84e5b149f6..c162c75b6a0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -731,7 +731,8 @@ class IColumn; M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \ - M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ + M(UInt64, database_replicated_allow_replicated_engine_arguments, 0, "0 - Don't allow to explicitly specify ZooKeeper path and replica name for *MergeTree tables in Replicated databases. 1 - Allow. 2 - Allow, but ignore the specified path and use default one instead.", 0) \ + M(UInt64, database_replicated_allow_explicit_uuid, 0, "0 - Don't allow to explicitly specify UUIDs for tables in Replicated databases. 1 - Allow. 2 - Allow, but ignore the specified UUID and generate a random one instead.", 0) \ M(Bool, cloud_mode, false, "Only available in ClickHouse Cloud", 0) \ M(UInt64, cloud_mode_engine, 1, "Only available in ClickHouse Cloud", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result, one of: 'none', 'throw', 'null_status_on_timeout', 'never_throw', 'none_only_active', 'throw_only_active', 'null_status_on_timeout_only_active'", 0) \ diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 4ca9afc49eb..507d1a9affc 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -774,18 +774,6 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ bool maybe_replica_macros = info.expanded_other; bool enable_functional_tests_helper = getContext()->getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); - if (!enable_functional_tests_helper) - { - if (query_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments) - LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); - else - throw Exception(ErrorCodes::INCORRECT_QUERY, - "It's not allowed to specify explicit zookeeper_path and replica_name " - "for ReplicatedMergeTree arguments in Replicated database. If you really want to " - "specify them explicitly, enable setting " - "database_replicated_allow_replicated_engine_arguments."); - } - if (maybe_shard_macros && maybe_replica_macros) return; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a990eb651ce..28827a1fe37 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1055,6 +1055,26 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data bool from_path = create.attach_from_path.has_value(); bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + if (database->getEngineName() == "Replicated" && create.uuid != UUIDHelpers::Nil && !is_replicated_database_internal) + { + if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 0) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "It's not allowed to explicitly specify UUIDs for tables in Replicated databases, " + "see database_replicated_allow_explicit_uuid"); + } + else if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 0) + { + LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "It's not recommended to explicitly specify UUIDs for tables in Replicated databases"); + } + else // if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 2) + { + UUID old_uuid = create.uuid; + create.generateRandomUUID(/*always_generate_new_uuid*/ true); + LOG_INFO(&Poco::Logger::get("InterpreterCreateQuery"), "Replaced a user-provided UUID ({}) with a random one ({}) " + "to make sure it's unique", old_uuid, create.uuid); + } + } + if (is_replicated_database_internal && !internal) { if (create.uuid == UUIDHelpers::Nil) diff --git a/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h b/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h index 5ef5e1db62e..212dc048868 100644 --- a/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h +++ b/src/Storages/MergeTree/extractZooKeeperPathFromReplicatedTableDef.h @@ -14,6 +14,6 @@ using ContextPtr = std::shared_ptr; /// Extracts a zookeeper path from a specified CREATE TABLE query. /// The function checks the table engine and if it is Replicated*MergeTree then it takes the first argument and expands macros in it. /// Returns std::nullopt if the specified CREATE query doesn't describe a Replicated table or its arguments can't be evaluated. -std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & create_query, const ContextPtr & context); +std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & create_query, const ContextPtr & local_context); } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index d234103e52b..66f7b2114bf 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -185,7 +186,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( const String & engine_name, ASTs & engine_args, LoadingStrictnessLevel mode, - const ContextPtr & context, + const ContextPtr & local_context, String & zookeeper_path, String & replica_name, RenamingRestrictions & renaming_restrictions) @@ -202,11 +203,11 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( { /// Allow expressions in engine arguments. /// In new syntax argument can be literal or identifier or array/tuple of identifiers. - evaluateEngineArgs(engine_args, context); + evaluateEngineArgs(engine_args, local_context); } - bool is_on_cluster = context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; - bool is_replicated_database = context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && + bool is_on_cluster = local_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + bool is_replicated_database = local_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && DatabaseCatalog::instance().getDatabase(table_id.database_name)->getEngineName() == "Replicated"; /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries @@ -226,10 +227,10 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( /// We did unfold it in previous versions to make moving table from Atomic to Ordinary database work correctly, /// but now it's not allowed (and it was the only reason to unfold {uuid} macro). info.table_id.uuid = UUIDHelpers::Nil; - zookeeper_path = context->getMacros()->expand(zookeeper_path, info); + zookeeper_path = local_context->getMacros()->expand(zookeeper_path, info); info.level = 0; - replica_name = context->getMacros()->expand(replica_name, info); + replica_name = local_context->getMacros()->expand(replica_name, info); } ast_zk_path->value = zookeeper_path; @@ -247,11 +248,11 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( } if (!allow_uuid_macro) info.table_id.uuid = UUIDHelpers::Nil; - zookeeper_path = context->getMacros()->expand(zookeeper_path, info); + zookeeper_path = local_context->getMacros()->expand(zookeeper_path, info); info.level = 0; info.table_id.uuid = UUIDHelpers::Nil; - replica_name = context->getMacros()->expand(replica_name, info); + replica_name = local_context->getMacros()->expand(replica_name, info); /// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE. /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation @@ -268,9 +269,24 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( bool has_arguments = (arg_num + 2 <= arg_cnt); bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + const auto & server_settings = local_context->getServerSettings(); if (has_valid_arguments) { + if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 0) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "It's not allowed to specify explicit zookeeper_path and replica_name " + "for ReplicatedMergeTree arguments in Replicated database. If you really want to " + "specify them explicitly, enable setting " + "database_replicated_allow_replicated_engine_arguments."); + } + else if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 1) + { + LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "It's not recommended to explicitly specify " + "zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + } + /// Get path and name from engine arguments auto * ast_zk_path = engine_args[arg_num]->as(); if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) @@ -284,6 +300,15 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must be a string literal{}", verbose_help_message); + + if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 2) + { + LOG_INFO(&Poco::Logger::get("registerStorageMergeTree"), "Replacing user-provided ZooKeeper path and replica name ({}, {}) " + "with default arguments", zookeeper_path, replica_name); + engine_args[arg_num]->as()->value = zookeeper_path = server_settings.default_replica_path; + engine_args[arg_num + 1]->as()->value = replica_name = server_settings.default_replica_name; + } + expand_macro(ast_zk_path, ast_replica_name); } else if (is_extended_storage_def @@ -293,7 +318,6 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( { /// Try use default values if arguments are not specified. /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. - const auto & server_settings = context->getServerSettings(); zookeeper_path = server_settings.default_replica_path; /// TODO maybe use hostname if {replica} is not defined? replica_name = server_settings.default_replica_name; @@ -318,7 +342,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( } /// Extracts a zookeeper path from a specified CREATE TABLE query. -std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & query, const ContextPtr & context) +std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreateQuery & query, const ContextPtr & local_context) { if (!query.storage || !query.storage->engine) return {}; @@ -342,7 +366,7 @@ std::optional extractZooKeeperPathFromReplicatedTableDef(const ASTCreate try { - extractZooKeeperPathAndReplicaNameFromEngineArgs(query, table_id, engine_name, engine_args, mode, context, + extractZooKeeperPathAndReplicaNameFromEngineArgs(query, table_id, engine_name, engine_args, mode, local_context, zookeeper_path, replica_name, renaming_restrictions); } catch (Exception & e) diff --git a/tests/config/users.d/database_replicated.xml b/tests/config/users.d/database_replicated.xml index c049c3559fc..1c2cf2ac22b 100644 --- a/tests/config/users.d/database_replicated.xml +++ b/tests/config/users.d/database_replicated.xml @@ -6,6 +6,7 @@ 120 1 1 + 3 diff --git a/tests/integration/test_replicated_database/configs/settings.xml b/tests/integration/test_replicated_database/configs/settings.xml index c637fe8eead..4b1122b1b5f 100644 --- a/tests/integration/test_replicated_database/configs/settings.xml +++ b/tests/integration/test_replicated_database/configs/settings.xml @@ -5,6 +5,7 @@ 1 0 0 + 3 diff --git a/tests/integration/test_replicated_database/configs/settings2.xml b/tests/integration/test_replicated_database/configs/settings2.xml index dad5740a8ae..9a3038871e1 100644 --- a/tests/integration/test_replicated_database/configs/settings2.xml +++ b/tests/integration/test_replicated_database/configs/settings2.xml @@ -5,6 +5,7 @@ 1 0 0 + 3 0 diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference new file mode 100644 index 00000000000..1d3a90daccd --- /dev/null +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference @@ -0,0 +1,10 @@ +2 +3 +m1 +m2 +rmt1 +rmt2 +02858000-1000-4000-8000-0000000000 +0 +CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/default/auto_{shard}\', \'1auto_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh new file mode 100755 index 00000000000..f2c2f47cd0d --- /dev/null +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +db=$CLICKHOUSE_DATABASE +if [[ $($CLICKHOUSE_CLIENT -q "SELECT engine = 'Replicated' FROM system.databases WHERE name='$CLICKHOUSE_DATABASE'") != 1 ]]; then + $CLICKHOUSE_CLIENT -q "CREATE DATABASE rdb_$CLICKHOUSE_DATABASE ENGINE=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', '1', '1')" + db="rdb_$CLICKHOUSE_DATABASE" +fi + +$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=0 -q "CREATE TABLE $db.m0 +UUID '02858000-1000-4000-8000-000000000000' (n int) ENGINE=Memory" 2>&1| grep -Fac "database_replicated_allow_explicit_uuid" + +$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1 +UUID '02858000-1000-4000-8000-0000000000$(($RANDOM % 10))$(($RANDOM % 10))' (n int) ENGINE=Memory" + +$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=2 -q "CREATE TABLE $db.m2 +UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory" + + +$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int) +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" + +$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int) +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" + +$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int) +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" + + +$CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name" + +$CLICKHOUSE_CLIENT -q "SELECT substring(toString(uuid) as s, 1, length(s) - 2) FROM system.tables WHERE database='$db' and name='m1'" +$CLICKHOUSE_CLIENT -q "SELECT toString(uuid) LIKE '02858000%' FROM system.tables WHERE database='$db' and name='m2'" + +$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1" +$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2" + +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS rdb_$CLICKHOUSE_DATABASE" From aa9591419d3f0cb1575ddd2cf7b82f2cfc397e2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Jul 2024 19:15:06 +0200 Subject: [PATCH 0216/1722] fix --- .../0_stateless/02858_explicit_uuid_and_zk_path.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh index f2c2f47cd0d..1b5d1a06a9b 100755 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) db=$CLICKHOUSE_DATABASE if [[ $($CLICKHOUSE_CLIENT -q "SELECT engine = 'Replicated' FROM system.databases WHERE name='$CLICKHOUSE_DATABASE'") != 1 ]]; then - $CLICKHOUSE_CLIENT -q "CREATE DATABASE rdb_$CLICKHOUSE_DATABASE ENGINE=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', '1', '1')" + $CLICKHOUSE_CLIENT -q "CREATE DATABASE rdb_$CLICKHOUSE_DATABASE ENGINE=Replicated('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rdb', '1', '1')" db="rdb_$CLICKHOUSE_DATABASE" fi @@ -21,13 +21,13 @@ UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory" $CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" $CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n" $CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_DATABASE', '1') ORDER BY n" +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n" $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name" From 826f6c1ce59893e0e1787a207a182c2eb012c7ae Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 17:56:08 +0000 Subject: [PATCH 0217/1722] fix initialization --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/StorageMergeTree.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 954e2fd951e..223ca989feb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,7 +154,7 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions; + Int64 num_alter_conversions = 0; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d05c6739738..40e5a8e5ea4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,7 +150,7 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions; + Int64 num_alter_conversions = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; From 77408c9fed1e67e6abd1e2ff1c2054a59438b729 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jul 2024 18:22:58 +0000 Subject: [PATCH 0218/1722] Conflict --- .../02933_replicated_database_forbid_create_as_select.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh index 15f169d880f..34df0544e13 100755 --- a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh +++ b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh @@ -12,14 +12,14 @@ ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = # Non-replicated engines are allowed ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test (id UInt64) ENGINE = MergeTree() ORDER BY id AS SELECT 1" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv (id UInt64) ENGINE = MergeTree() ORDER BY id POPULATE AS SELECT 1" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv (id UInt64) ENGINE = MergeTree() ORDER BY id POPULATE AS SELECT 1 AS id" # Replicated storafes are forbidden ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" |& grep -cm1 "SUPPORT_IS_DISABLED" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" |& grep -cm1 "SUPPORT_IS_DISABLED" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1 AS id" |& grep -cm1 "SUPPORT_IS_DISABLED" # But it is allowed with the special setting ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" --database_replicated_allow_heavy_create=1 -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" --database_replicated_allow_heavy_create=1 +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1 AS id" --database_replicated_allow_heavy_create=1 ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" From 54fda05094d1d3d788e0e65591c0d15b0e5838b0 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 10:25:04 +0000 Subject: [PATCH 0219/1722] reset session send/recv hooks at connection destruction --- src/Common/HTTPConnectionPool.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index bac12fd438d..3798b7624ea 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -385,13 +385,8 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); - else - Session::setReceiveDataHooks(); - if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); - else - Session::setSendDataHooks(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); @@ -449,9 +444,8 @@ private: } } response_stream = nullptr; - // FIXME: We are not sure that response stream is fully read at this moment, so hooks could possible be called after this point, right? - // Session::setSendDataHooks(); - // Session::setReceiveDataHooks(); + Session::setSendDataHooks(); + Session::setReceiveDataHooks(); group->atConnectionDestroy(); From 3fa5ad92b4612b59ca9c35aab5853dcb3318a112 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 10:33:22 +0000 Subject: [PATCH 0220/1722] rename `IHTTPSessionDataHooks` methods --- base/poco/Net/include/Poco/Net/HTTPSession.h | 6 +++--- base/poco/Net/src/HTTPSession.cpp | 12 ++++++------ src/Common/HTTPConnectionPool.cpp | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index b25ad68cc67..2038fd2aff5 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -42,13 +42,13 @@ namespace Net public: virtual ~IHTTPSessionDataHooks() = default; - virtual void start(int bytes) = 0; + virtual void atStart(int bytes) = 0; /// Called before sending/receiving data `bytes` to/from socket. - virtual void finish(int bytes) = 0; + virtual void atFinish(int bytes) = 0; /// Called when sending/receiving of data `bytes` is successfully finished. - virtual void fail() = 0; + virtual void atFail() = 0; /// If an error occurred during send/receive `fail()` is called instead of `finish()`. }; diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 596185703fa..f30ccb21129 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -167,16 +167,16 @@ int HTTPSession::write(const char* buffer, std::streamsize length) try { if (_sendDataHooks) - _sendDataHooks->start((int) length); + _sendDataHooks->atStart((int) length); int result = _socket.sendBytes(buffer, (int) length); if (_sendDataHooks) - _sendDataHooks->finish(result); + _sendDataHooks->atFinish(result); return result; } catch (Poco::Exception& exc) { if (_sendDataHooks) - _sendDataHooks->fail(); + _sendDataHooks->atFail(); setException(exc); throw; } @@ -188,16 +188,16 @@ int HTTPSession::receive(char* buffer, int length) try { if (_receiveDataHooks) - _receiveDataHooks->start(length); + _receiveDataHooks->atStart(length); int result = _socket.receiveBytes(buffer, length); if (_receiveDataHooks) - _receiveDataHooks->finish(result); + _receiveDataHooks->atFinish(result); return result; } catch (Poco::Exception& exc) { if (_receiveDataHooks) - _receiveDataHooks->fail(); + _receiveDataHooks->atFail(); setException(exc); throw; } diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 3798b7624ea..62362c61605 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -239,13 +239,13 @@ public: // Session data hooks implementation for integration with resource scheduler. // Hooks are created per every request-response pair and are registered/unregistered in HTTP session. -// * `start()` send resource request to the scheduler every time HTTP session is going to send or receive +// * `atStart()` send resource request to the scheduler every time HTTP session is going to send or receive // data to/from socket. `start()` waits for the scheduler confirmation. This way scheduler might // throttle and/or schedule socket data streams. -// * `finish()` hook is called on successful socket read/write operation. +// * `atFinish()` hook is called on successful socket read/write operation. // It informs the scheduler that operation is complete, which allows the scheduler to control the total // amount of in-flight bytes and/or operations. -// * `fail()` hook is called on failure of socket operation. The purpose is to correct the amount of bytes +// * `atFail()` hook is called on failure of socket operation. The purpose is to correct the amount of bytes // passed through the scheduler queue to ensure fair bandwidth allocation even in presence of errors. struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks { @@ -261,18 +261,18 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks request.assertFinished(); // Never destruct with an active request } - void start(int bytes) override + void atStart(int bytes) override { request.enqueue(bytes, link); request.wait(); } - void finish(int bytes) override + void atFinish(int bytes) override { request.finish(bytes, link); } - void fail() override + void atFail() override { request.finish(0, link); } From fefcc52c24b4560cc2434decdbd0981edfcbc5ca Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 11:09:10 +0000 Subject: [PATCH 0221/1722] add logging of too long resource requests for http sessions --- src/Common/HTTPConnectionPool.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 62362c61605..acddcc8530d 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -249,8 +250,10 @@ public: // passed through the scheduler queue to ensure fair bandwidth allocation even in presence of errors. struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks { - explicit ResourceGuardSessionDataHooks(const ResourceGuard::Metrics * metrics, ResourceLink link_) + ResourceGuardSessionDataHooks(ResourceLink link_, const ResourceGuard::Metrics * metrics, LoggerPtr log_, const String & method, const String & uri) : link(link_) + , log(log_) + , http_request(method + " " + uri) { request.metrics = metrics; chassert(link); @@ -263,8 +266,12 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks void atStart(int bytes) override { + Stopwatch timer; request.enqueue(bytes, link); request.wait(); + timer.stop(); + if (timer.elapsedMilliseconds() >= 5000) + LOG_INFO(log, "Resource request took too long to finish: {} ms for {}", timer.elapsedMilliseconds(), http_request); } void atFinish(int bytes) override @@ -279,6 +286,8 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks ResourceLink link; ResourceGuard::Request request; + LoggerPtr log; + String http_request; }; @@ -384,9 +393,9 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) - Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); + Session::setReceiveDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIORead(), log, request.getMethod(), request.getURI())); if (ResourceLink link = CurrentThread::getWriteResourceLink()) - Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + Session::setSendDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIOWrite(), log, request.getMethod(), request.getURI())); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); From 942f7d7532059cf931242ce5c94a39ea0344b50b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 5 Jul 2024 12:44:31 +0000 Subject: [PATCH 0222/1722] fixes after review --- programs/obfuscator/Obfuscator.cpp | 4 +- src/Core/Settings.h | 6 +- src/Core/SettingsChangesHistory.cpp | 6 +- src/Formats/ReadSchemaUtils.cpp | 23 +- src/Formats/ReadSchemaUtils.h | 6 +- .../DataLakes/IStorageDataLake.h | 3 +- .../ObjectStorage/ReadBufferIterator.cpp | 2 +- .../ObjectStorage/ReadBufferIterator.h | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 60 ++--- .../ObjectStorage/StorageObjectStorage.h | 6 + .../StorageObjectStorageCluster.cpp | 3 +- .../StorageObjectStorageSource.cpp | 13 +- src/Storages/ObjectStorage/Utils.cpp | 7 +- src/Storages/ObjectStorage/Utils.h | 1 + .../StorageObjectStorageQueue.cpp | 3 +- src/Storages/StorageFile.cpp | 53 ++--- src/Storages/StorageURL.cpp | 24 +- src/Storages/VirtualColumnUtils.cpp | 63 +++-- src/Storages/VirtualColumnUtils.h | 9 +- src/TableFunctions/TableFunctionFormat.cpp | 10 +- .../TableFunctionObjectStorage.cpp | 3 +- .../__init__.py | 0 .../configs/cluster_azure.xml | 39 ---- .../configs/cluster_hdfs.xml | 33 --- .../configs/disable_profilers_azure.xml | 9 - .../configs/macro_hdfs.xml | 5 - .../configs/named_collections_azure.xml | 14 -- .../configs/schema_cache_azure.xml | 3 - .../configs/schema_cache_hdfs.xml | 3 - .../configs/users_azure.xml | 9 - .../test_azure.py | 219 ------------------ .../test_hdfs.py | 87 ------- .../test_storage_azure_blob_storage/test.py | 110 +++++++++ tests/integration/test_storage_hdfs/test.py | 49 ++++ 34 files changed, 304 insertions(+), 583 deletions(-) delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py delete mode 100644 tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 688ae1a1143..11e85bc1302 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1307,7 +1307,9 @@ try throw ErrnoException(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Input must be seekable file (it will be read twice)"); SingleReadBufferIterator read_buffer_iterator(std::move(file)); - schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, context_const); + + std::string sample_string; + schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, sample_string, context_const); } else { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8399d3925db..65b93b893b6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -935,6 +935,7 @@ class IColumn; M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ M(Bool, allow_deprecated_snowflake_conversion_functions, false, "Enables deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ + M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. @@ -1106,11 +1107,6 @@ class IColumn; M(Bool, input_format_tsv_skip_trailing_empty_lines, false, "Skip trailing empty lines in TSV format", 0) \ M(Bool, input_format_custom_skip_trailing_empty_lines, false, "Skip trailing empty lines in CustomSeparated format", 0) \ M(Bool, input_format_tsv_crlf_end_of_line, false, "If it is set true, file function will read TSV format with \\r\\n instead of \\n.", 0) \ - M(Bool, file_hive_partitioning, false, "Allows to use hive partitioning for file format", 0)\ - M(Bool, url_hive_partitioning, false, "Allows to use hive partitioning for url format", 0)\ - M(Bool, s3_hive_partitioning, false, "Allows to use hive partitioning for s3 format", 0)\ - M(Bool, azure_blob_storage_hive_partitioning, false, "Allows to use hive partitioning for AzureBlobStorage format", 0)\ - M(Bool, hdfs_hive_partitioning, false, "Allows to use hive partitioning for hdfs format", 0)\ \ M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 607f9b6d858..b676cd85ce6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,11 +59,7 @@ static std::initializer_list readSchemaFromFormatImpl( std::optional format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context) try { @@ -143,6 +144,10 @@ try { iterator_data = read_buffer_iterator.next(); + /// Extracting the File path for hive-style partitioning + if (sample_path.empty()) + sample_path = read_buffer_iterator.getLastFilePath(); + /// Read buffer iterator can determine the data format if it's unknown. /// For example by scanning schema cache or by finding new file with format extension. if (!format_name && iterator_data.format_name) @@ -163,7 +168,7 @@ try return {*iterator_data.cached_columns, *format_name}; } - schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFileName()); + schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFilePath()); continue; } @@ -249,7 +254,7 @@ try if (!names_and_types.empty()) read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); - schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName()); + schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath()); } catch (...) { @@ -410,7 +415,7 @@ try throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files. You can specify the format manually"); read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); - schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName()); + schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath()); } if (format_name && mode == SchemaInferenceMode::DEFAULT) @@ -526,9 +531,9 @@ try } catch (Exception & e) { - auto file_name = read_buffer_iterator.getLastFileName(); - if (!file_name.empty()) - e.addMessage(fmt::format("(in file/uri {})", file_name)); + auto file_path = read_buffer_iterator.getLastFilePath(); + if (!file_path.empty()) + e.addMessage(fmt::format("(in file/uri {})", file_path)); throw; } @@ -536,17 +541,19 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, context).first; + return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, sample_path, context).first; } std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, context); + return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, sample_path, context); } SchemaCache::Key getKeyForSchemaCache( diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index bb5e068f696..6c562a06bf0 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -56,8 +56,8 @@ struct IReadBufferIterator /// Set auto detected format name. virtual void setFormatName(const String & /*format_name*/) {} - /// Get last processed file name for better exception messages. - virtual String getLastFileName() const { return ""; } + /// Get last processed file path for better exception messages. + virtual String getLastFilePath() const { return ""; } /// Return true if method recreateLastReadBuffer is implemented. virtual bool supportsLastReadBufferRecreation() const { return false; } @@ -122,6 +122,7 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context); /// Try to detect the format of the data and it's schema. @@ -131,6 +132,7 @@ ColumnsDescription readSchemaFromFormat( std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, + std::string & sample_path, const ContextPtr & context); SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context); diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 83865c47eb8..5c40cda442b 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -89,8 +89,9 @@ public: { ConfigurationPtr configuration = base_configuration->clone(); configuration->setPaths(metadata->getDataFiles()); + std::string sample_string; return Storage::resolveSchemaFromData( - object_storage_, configuration, format_settings_, local_context); + object_storage_, configuration, format_settings_, sample_string, local_context); } } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 78cdc442f64..a47049791ae 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -131,7 +131,7 @@ void ReadBufferIterator::setFormatName(const String & format_name) format = format_name; } -String ReadBufferIterator::getLastFileName() const +String ReadBufferIterator::getLastFilePath() const { if (current_object_info) return current_object_info->getPath(); diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 6eeb52ec2ed..b81aebb7b07 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -33,7 +33,7 @@ public: void setResultingSchema(const ColumnsDescription & columns) override; - String getLastFileName() const override; + String getLastFilePath() const override; void setFormatName(const String & format_name) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ae7c211330c..717f48983f3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -33,17 +33,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -bool checkIfHiveSettingEnabled(const ContextPtr & context, const std::string & storage_type_name) +std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { - if (storage_type_name == "s3") - return context->getSettings().s3_hive_partitioning; - else if (storage_type_name == "hdfs") - return context->getSettings().hdfs_hive_partitioning; - else if (storage_type_name == "azure") - return context->getSettings().azure_blob_storage_hive_partitioning; - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + object_storage, + distributed_processing, + context, + {}, // predicate + metadata.getColumns().getAll(), // virtual_columns + nullptr, // read_keys + {} // file_progress_callback + ); + + if (auto file = file_iterator->next(0)) + return file->getPath(); + return ""; } StorageObjectStorage::StorageObjectStorage( @@ -66,7 +71,9 @@ StorageObjectStorage::StorageObjectStorage( , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { ColumnsDescription columns{columns_}; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context); + + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context); configuration->check(context); StorageInMemoryMetadata metadata; @@ -74,23 +81,13 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - auto file_iterator = StorageObjectStorageSource::createFileIterator( - configuration, - object_storage, - distributed_processing_, - context, - {}, // predicate - metadata.getColumns().getAll(), // virtual_columns - nullptr, // read_keys - {} // file_progress_callback - ); + + if (sample_path.empty() && context->getSettings().use_hive_partitioning) + sample_path = getPathSample(metadata, context); + else if (!context->getSettings().use_hive_partitioning) + sample_path = ""; - Strings paths; - - if (checkIfHiveSettingEnabled(context, configuration->getTypeName())) - if (auto file = file_iterator->next(0)) - paths = {file->getPath()}; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), paths)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), sample_path)); setInMemoryMetadata(metadata); } @@ -386,33 +383,36 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context) { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); + return readSchemaFromFormat(configuration->format, format_settings, *iterator, sample_path, context); } std::string StorageObjectStorage::resolveFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context) { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return detectFormatAndReadSchema(format_settings, *iterator, context).second; + return detectFormatAndReadSchema(format_settings, *iterator, sample_path, context).second; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context) { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); + auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, sample_path, context); configuration->format = format; return std::pair(columns, format); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index cf8ec113653..dd7ec31c970 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -42,6 +42,7 @@ public: size_t list_object_keys_size; bool throw_on_zero_files_match; bool ignore_non_existent_file; + bool use_hive_partitioning; }; StorageObjectStorage( @@ -100,23 +101,28 @@ public: const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context); static std::string resolveFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context); static std::pair resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, + std::string & sample_path, const ContextPtr & context); protected: virtual void updateConfiguration(ContextPtr local_context); + std::string getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); + static std::unique_ptr createReadBufferIterator( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 78f568d8ae2..0dc4b845a47 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -33,7 +33,8 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( , object_storage(object_storage_) { ColumnsDescription columns{columns_}; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, context_); + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, sample_path, context_); configuration->check(context_); StorageInMemoryMetadata metadata; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index afb23961312..ecb3ff9d856 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -197,16 +197,6 @@ Chunk StorageObjectStorageSource::generate() const auto & filename = object_info->getFileName(); chassert(object_info->metadata); - auto hive_map = VirtualColumnUtils::parsePartitionMapFromPath(object_info->getPath()); - bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), - [&](const auto& pair) - { - return read_from_format_info.requested_virtual_columns.contains(pair.first); - }); - - if (!contains_virtual_column) - hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk - VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, { @@ -214,8 +204,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - .hive_partitioning_map = hive_map - }); + }, object_info->getPath()); return chunk; } diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index e49e14d2a0c..73410d959e0 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -49,19 +49,20 @@ void resolveSchemaAndFormat( ObjectStoragePtr object_storage, const StorageObjectStorage::ConfigurationPtr & configuration, std::optional format_settings, + std::string & sample_path, const ContextPtr & context) { if (columns.empty()) { if (format == "auto") std::tie(columns, format) = - StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context); + StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, sample_path, context); else - columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, context); + columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, sample_path, context); } else if (format == "auto") { - format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, context); + format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, sample_path, context); } if (!columns.hasOnlyOrdinary()) diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 2077999df41..7ee14f50979 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -19,6 +19,7 @@ void resolveSchemaAndFormat( ObjectStoragePtr object_storage, const StorageObjectStorage::ConfigurationPtr & configuration, std::optional format_settings, + std::string & sample_path, const ContextPtr & context); } diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 95265cde9ea..c12cdddeec7 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -160,7 +160,8 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( configuration->check(context_); ColumnsDescription columns{columns_}; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context_); + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context_); configuration->check(context_); StorageInMemoryMetadata storage_metadata; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 0c32f29cb34..9751d596fff 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -502,7 +502,7 @@ namespace StorageFile::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); } - String getLastFileName() const override + String getLastFilePath() const override { if (current_index != 0) return paths[current_index - 1]; @@ -777,7 +777,7 @@ namespace format = format_name; } - String getLastFileName() const override + String getLastFilePath() const override { return last_read_file_path; } @@ -880,10 +880,11 @@ std::pair StorageFile::getTableStructureAndFormatFro auto read_buffer_iterator = SingleReadBufferIterator(std::move(read_buf)); ColumnsDescription columns; + std::string sample_path; if (format) - columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context); + columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context); else - std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); peekable_read_buffer_from_fd = read_buffer_iterator.releaseBuffer(); if (peekable_read_buffer_from_fd) @@ -928,20 +929,21 @@ std::pair StorageFile::getTableStructureAndFormatFro } + std::string sample_path; if (archive_info) { ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); } ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); } ColumnsDescription StorageFile::getTableStructureFromFile( @@ -1097,10 +1099,10 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - Strings paths_for_virtuals; - if (args.getContext()->getSettingsRef().file_hive_partitioning) - paths_for_virtuals = paths; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), paths_for_virtuals)); + std::string path_for_virtuals; + if (args.getContext()->getSettingsRef().use_hive_partitioning && !paths.empty()) + path_for_virtuals = paths[0]; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); } @@ -1442,14 +1444,9 @@ Chunk StorageFileSource::generate() chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - std::map hive_map; - if (getContext()->getSettingsRef().file_hive_partitioning) - { - hive_map = VirtualColumnUtils::parsePartitionMapFromPath(current_path); - - for (const auto& item : hive_map) - requested_virtual_columns.push_back(NameAndTypePair(item.first, std::make_shared())); - } + std::string hive_partitioning_path; + if (getContext()->getSettingsRef().use_hive_partitioning) + hive_partitioning_path = current_path; /// Enrich with virtual columns. VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( @@ -1459,8 +1456,7 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified, - .hive_partitioning_map = hive_map - }); + }, hive_partitioning_path); return chunk; } @@ -1636,16 +1632,6 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate) storage->distributed_processing); } -void addPartitionColumnsToInfoHeader(Strings paths, ReadFromFormatInfo & info) -{ - for (const auto& path : paths) - { - auto map = VirtualColumnUtils::parsePartitionMapFromPath(path); - for (const auto& item : map) - info.source_header.insertUnique(ColumnWithTypeAndName(std::make_shared(), item.first)); - } -} - void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); @@ -1665,9 +1651,6 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui paths = storage->paths; } - if (getContext()->getSettingsRef().file_hive_partitioning) - addPartitionColumnsToInfoHeader(paths, info); - if (max_num_streams > files_to_read) num_streams = files_to_read; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f6374701fc2..59c5465a381 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -153,10 +153,10 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - Strings uri_for_partitioning; - if (context_->getSettingsRef().url_hive_partitioning) - uri_for_partitioning = {uri}; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning)); + std::string uri_for_partitioning; + if (context_->getSettingsRef().use_hive_partitioning) + uri_for_partitioning = uri; + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning, format_settings.value_or(FormatSettings{}))); } @@ -415,9 +415,9 @@ Chunk StorageURLSource::generate() size_t chunk_size = 0; if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); - std::map hive_map; - if (getContext()->getSettingsRef().url_hive_partitioning) - hive_map = VirtualColumnUtils::parsePartitionMapFromPath(curr_uri.getPath()); + std::string hive_partitioning_path; + if (getContext()->getSettingsRef().use_hive_partitioning) + hive_partitioning_path = curr_uri.getPath(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( @@ -425,8 +425,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - .hive_partitioning_map = hive_map - }); + }, hive_partitioning_path); return chunk; } @@ -859,7 +858,7 @@ namespace format = format_name; } - String getLastFileName() const override { return current_url_option; } + String getLastFilePath() const override { return current_url_option; } bool supportsLastReadBufferRecreation() const override { return true; } @@ -960,9 +959,10 @@ std::pair IStorageURLBase::getTableStructureAndForma urls_to_check = {uri}; ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context); + std::string sample_path; if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); } ColumnsDescription IStorageURLBase::getTableStructureFromData( diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 0b79e3b7a16..379b14d8e51 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" @@ -115,22 +116,19 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -Strings parseVirtualColumnNameFromPath(const std::string & path) +std::map parseFromPath(const std::string& path) { std::string pattern = "/([^/]+)=([^/]+)"; - // Map to store the key-value pairs - std::map key_values; - re2::StringPiece input_piece(path); - std::string key; - Strings result; - while (RE2::FindAndConsume(&input_piece, pattern, &key)) - result.push_back(key); - return result; + std::map key_values; + std::string key, value; + while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) + key_values["_" + key] = value; + return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths) +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path, FormatSettings settings) { VirtualColumnsDescription desc; @@ -147,11 +145,13 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - for (const auto& path : paths) + auto map = parseFromPath(path); + for (const auto& item : map) { - auto names = parseVirtualColumnNameFromPath(path); - for (const auto& name : names) - add_virtual("_" + name, std::make_shared(std::make_shared())); + auto type = tryInferDataTypeForSingleField(item.second, settings); + if (type == nullptr) + type = std::make_shared(); + add_virtual(item.first, std::make_shared(type)); } return desc; @@ -213,25 +213,11 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -std::map parsePartitionMapFromPath(const std::string & path) -{ - std::string pattern = "/([^/]+)=([^/]+)"; // Regex to capture key=value pairs - // Map to store the key-value pairs - std::map key_values; - - re2::StringPiece input_piece(path); - std::string key; - std::string value; - while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) - key_values["_" + key] = value; - - return key_values; -} - void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values) + VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path) { + auto hive_map = parseFromPath(hive_partitioning_path); for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -265,13 +251,22 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } - else + else if (!hive_map.empty()) { - auto it = virtual_values.hive_partitioning_map.find(virtual_column.getNameInStorage()); - if (it != virtual_values.hive_partitioning_map.end()) + bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), + [&](const auto& pair) + { + return requested_virtual_columns.contains(pair.first); + }); + + if (!contains_virtual_column) + hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk + + auto it = hive_map.find(virtual_column.getNameInStorage()); + if (it != hive_map.end()) { chunk.addColumn(virtual_column.getTypeInStorage()->createColumnConst(chunk.getNumRows(), it->second)->convertToFullColumnIfConst()); - virtual_values.hive_partitioning_map.erase(it); + hive_map.erase(it); } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index f9b49cc48ed..72922be60bd 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -49,7 +50,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) } NameSet getVirtualNamesForFileLikeStorage(); -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, Strings paths = {}); +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path = "", FormatSettings settings = FormatSettings()); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); @@ -76,15 +77,13 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - std::map hive_partitioning_map {}; - }; -std::map parsePartitionMapFromPath(const std::string & path); +std::map parseFromPath(const std::string& path); void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values); + VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path = ""); } } diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index ad2a142a140..66152cb0c91 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -85,9 +85,10 @@ ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr conte if (structure == "auto") { SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); + std::string sample_path; if (format == "auto") - return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context).first; - return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); + return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context).first; + return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); } return parseColumnsListFromString(structure, context); } @@ -131,11 +132,12 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con String format_name = format; if (structure == "auto") { + std::string sample_path; SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); if (format_name == "auto") - std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context); + std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context); else - columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); + columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); } else { diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..39392a4c44c 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -84,7 +84,8 @@ ColumnsDescription TableFunctionObjectStorage< context->checkAccess(getSourceAccessType()); ColumnsDescription columns; auto storage = getObjectStorage(context, !is_insert_query); - resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, context); + std::string sample_path; + resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, sample_path, context); return columns; } else diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml deleted file mode 100644 index ffa4673c9ee..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_azure.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - - node_0 - 9000 - - - node_1 - 9000 - - - node_2 - 9000 - - - - - - - - node_0 - 9000 - - - - - node_1 - 19000 - - - - - - - simple_cluster - - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml deleted file mode 100644 index b99b21ea40b..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/cluster_hdfs.xml +++ /dev/null @@ -1,33 +0,0 @@ - - - - - - node1 - 9000 - - - - - node1 - 19000 - - - - - - - - 127.0.0.1 - 9000 - - - - - 127.0.0.2 - 9000 - - - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml deleted file mode 100644 index a39badbf8ec..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/disable_profilers_azure.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - 0 - 0 - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml deleted file mode 100644 index c2e11b47a5e..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/macro_hdfs.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - test_cluster_two_shards - - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml deleted file mode 100644 index bd7f9ff97f1..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/named_collections_azure.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - cont - test_simple_write_named.csv - key UInt64, data String - CSV - - - devstoreaccount1 - Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml deleted file mode 100644 index e2168ecd06d..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_azure.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 2 - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml deleted file mode 100644 index 37639649b5f..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/schema_cache_hdfs.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 2 - \ No newline at end of file diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml b/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/configs/users_azure.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py deleted file mode 100644 index 0be697821f0..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_azure.py +++ /dev/null @@ -1,219 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import time - -from helpers.cluster import ClickHouseCluster, is_arm -import re - -from azure.storage.blob import BlobServiceClient -from helpers.cluster import ClickHouseCluster, ClickHouseInstance - -if is_arm(): - pytestmark = pytest.mark.skip - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node", - main_configs=[ - "configs/named_collections_azure.xml", - "configs/schema_cache_azure.xml", - ], - user_configs=[ - "configs/disable_profilers_azure.xml", - "configs/users_azure.xml", - ], - with_azurite=True, - ) - cluster.start() - container_client = cluster.blob_service_client.get_container_client("cont") - container_client.create_container() - yield cluster - finally: - cluster.shutdown() - - -def azure_query( - node, query, expect_error=False, try_num=10, settings={}, query_on_retry=None -): - for i in range(try_num): - try: - if expect_error: - return node.query_and_get_error(query, settings=settings) - else: - return node.query(query, settings=settings) - except Exception as ex: - retriable_errors = [ - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", - "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", - "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", - "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", - "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", - "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", - "Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read", - ] - retry = False - for error in retriable_errors: - if error in str(ex): - retry = True - print(f"Try num: {i}. Having retriable error: {ex}") - time.sleep(i) - break - if not retry or i == try_num - 1: - raise Exception(ex) - if query_on_retry is not None: - node.query(query_on_retry) - continue - - -def get_azure_file_content(filename, port): - container_name = "cont" - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" - ) - blob_service_client = BlobServiceClient.from_connection_string( - str(connection_string) - ) - container_client = blob_service_client.get_container_client(container_name) - blob_client = container_client.get_blob_client(filename) - download_stream = blob_client.download_blob() - return download_stream.readall().decode("utf-8") - - -@pytest.fixture(autouse=True, scope="function") -def delete_all_files(cluster): - port = cluster.env_variables["AZURITE_PORT"] - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" - ) - blob_service_client = BlobServiceClient.from_connection_string(connection_string) - containers = blob_service_client.list_containers() - for container in containers: - container_client = blob_service_client.get_container_client(container) - blob_list = container_client.list_blobs() - for blob in blob_list: - print(blob) - blob_client = container_client.get_blob_client(blob) - blob_client.delete_blob() - - assert len(list(container_client.list_blobs())) == 0 - - yield - - -def test_azure_partitioning_with_one_parameter(cluster): - # type: (ClickHouseCluster) -> None - node = cluster.instances["node"] # type: ClickHouseInstance - table_format = "column1 String, column2 String" - values = f"('Elizabeth', 'Gordon')" - path = "a/column1=Elizabeth/sample.csv" - - azure_query( - node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", - ) - - query = ( - f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}')" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( - bucket="cont", max_path=path - ) - ] - - query = ( - f"SELECT column2 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == ["Gordon"] - - -def test_azure_partitioning_with_two_parameters(cluster): - # type: (ClickHouseCluster) -> None - node = cluster.instances["node"] # type: ClickHouseInstance - table_format = "column1 String, column2 String" - values_1 = f"('Elizabeth', 'Gordon')" - values_2 = f"('Emilia', 'Gregor')" - path = "a/column1=Elizabeth/column2=Gordon/sample.csv" - - azure_query( - node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", - ) - - query = ( - f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( - bucket="cont", max_path=path - ) - ] - - query = ( - f"SELECT column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == ["Elizabeth"] - - query = ( - f"SELECT column1 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" - ) - assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} - ).splitlines() == ["Elizabeth"] - - -def test_azure_partitioning_without_setting(cluster): - # type: (ClickHouseCluster) -> None - node = cluster.instances["node"] # type: ClickHouseInstance - table_format = "column1 String, column2 String" - values_1 = f"('Elizabeth', 'Gordon')" - values_2 = f"('Emilia', 'Gregor')" - path = "a/column1=Elizabeth/column2=Gordon/sample.csv" - - azure_query( - node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", - ) - - query = ( - f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " - f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" - ) - pattern = re.compile( - r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL - ) - - with pytest.raises(Exception, match=pattern): - azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py b/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py deleted file mode 100644 index 4667d18688a..00000000000 --- a/tests/integration/test_hive_style_partitioning_hdfs_azure/test_hdfs.py +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env python3 - -import pytest - -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, is_arm -import re - -from helpers.cluster import ClickHouseCluster - -if is_arm(): - pytestmark = pytest.mark.skip - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=[ - "configs/macro_hdfs.xml", - "configs/schema_cache_hdfs.xml", - "configs/cluster_hdfs.xml", - ], - with_hdfs=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_hdfs_partitioning_with_one_parameter(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") - assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" - - r = node1.query( - "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", - settings={"hdfs_hive_partitioning": 1}, - ) - assert r == f"Elizabeth\n" - - -def test_hdfs_partitioning_with_two_parameters(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") - == f"Elizabeth\tGordon\n" - ) - - r = node1.query( - "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 1}, - ) - assert r == f"Gordon\n" - - -def test_hdfs_partitioning_without_setting(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") - == f"Elizabeth\tGordon\n" - ) - pattern = re.compile( - r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL - ) - - with pytest.raises(QueryRuntimeException, match=pattern): - node1.query( - f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 0}, - ) - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 20b004a7605..893df6d23aa 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -5,6 +5,7 @@ import json import logging import os import io +import re import random import threading import time @@ -1462,3 +1463,112 @@ def test_insert_create_new_file(cluster): assert TSV(res) == TSV( "test_create_new_file.csv\t1\ntest_create_new_file.1.csv\t2\n" ) + + +def test_hive_partitioning_with_one_parameter(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values = f"('Elizabeth', 'Gordon')" + path = "a/column1=Elizabeth/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}')" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Gordon"] + + +def test_hive_partitioning_with_two_parameters(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == [ + "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( + bucket="cont", max_path=path + ) + ] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] + + query = ( + f"SELECT column1 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" + ) + assert azure_query( + node, query, settings={"azure_blob_storage_hive_partitioning": 1} + ).splitlines() == ["Elizabeth"] + + +def test_hive_partitioning_without_setting(cluster): + # type: (ClickHouseCluster) -> None + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 String, column2 String" + values_1 = f"('Elizabeth', 'Gordon')" + values_2 = f"('Emilia', 'Gregor')" + path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + ) + + query = ( + f"SELECT column1, column2, _file, _path, _column1, _column2 FROM azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" + ) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) + + with pytest.raises(Exception, match=pattern): + azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 47d8f44c0b7..8071b520a4f 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -3,6 +3,7 @@ import os import pytest import time from helpers.cluster import ClickHouseCluster, is_arm +from helpers.client import QueryRuntimeException from helpers.test_tools import TSV from pyhdfs import HdfsClient @@ -1180,6 +1181,54 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): assert int(result) == 44 +def test_hive_partitioning_with_one_parameter(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") + assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" + + r = node1.query( + "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + settings={"hdfs_hive_partitioning": 1}, + ) + assert r == f"Elizabeth\n" + + +def test_hive_partitioning_with_two_parameters(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + + r = node1.query( + "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 1}, + ) + assert r == f"Gordon\n" + + +def test_hive_partitioning_without_setting(started_cluster): + hdfs_api = started_cluster.hdfs_api + hdfs_api.write_data( + f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") + == f"Elizabeth\tGordon\n" + ) + pattern = re.compile( + r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + ) + + with pytest.raises(QueryRuntimeException, match=pattern): + node1.query( + f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + settings={"hdfs_hive_partitioning": 0}, + ) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 3fb50aa5d8560f3cd5f4fc9b35bfa47a60d2ca80 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 5 Jul 2024 13:28:47 +0000 Subject: [PATCH 0223/1722] style fix --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 - tests/integration/test_storage_hdfs/test.py | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 717f48983f3..4b5b514e67d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -81,7 +81,6 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - if (sample_path.empty() && context->getSettings().use_hive_partitioning) sample_path = getPathSample(metadata, context); else if (!context->getSettings().use_hive_partitioning) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 8071b520a4f..da46756841d 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1229,6 +1229,7 @@ def test_hive_partitioning_without_setting(started_cluster): settings={"hdfs_hive_partitioning": 0}, ) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 613636bb1b324acf7500c8a760abf5aea764418b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Jul 2024 20:49:10 +0200 Subject: [PATCH 0224/1722] fix --- src/Core/SettingsChangesHistory.cpp | 2 ++ src/Databases/DatabaseReplicated.cpp | 3 ++- src/Interpreters/InterpreterCreateQuery.cpp | 6 +++--- .../MergeTree/registerStorageMergeTree.cpp | 2 +- .../01148_zookeeper_path_macros_unfolding.sql | 2 ++ ...atabase_replicated_no_arguments_for_rmt.sh | 2 +- .../02858_explicit_uuid_and_zk_path.reference | 2 +- .../02858_explicit_uuid_and_zk_path.sh | 21 +++++++++++-------- 8 files changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6af6b4b15aa..6c730940c87 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -63,6 +63,8 @@ static std::initializer_listgetSettingsRef().database_replicated_allow_replicated_engine_arguments == 0) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. " "If you really want to specify it explicitly, then you should use some macros " "to distinguish different shards and replicas"); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c32a32be5cc..2862a907bec 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1062,15 +1062,15 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data throw Exception(ErrorCodes::BAD_ARGUMENTS, "It's not allowed to explicitly specify UUIDs for tables in Replicated databases, " "see database_replicated_allow_explicit_uuid"); } - else if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 0) + else if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 1) { LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "It's not recommended to explicitly specify UUIDs for tables in Replicated databases"); } - else // if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 2) + else if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 2) { UUID old_uuid = create.uuid; create.generateRandomUUID(/*always_generate_new_uuid*/ true); - LOG_INFO(&Poco::Logger::get("InterpreterCreateQuery"), "Replaced a user-provided UUID ({}) with a random one ({}) " + LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "Replaced a user-provided UUID ({}) with a random one ({}) " "to make sure it's unique", old_uuid, create.uuid); } } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 66f7b2114bf..d27c786d422 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -303,7 +303,7 @@ static void extractZooKeeperPathAndReplicaNameFromEngineArgs( if (is_replicated_database && local_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments == 2) { - LOG_INFO(&Poco::Logger::get("registerStorageMergeTree"), "Replacing user-provided ZooKeeper path and replica name ({}, {}) " + LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "Replacing user-provided ZooKeeper path and replica name ({}, {}) " "with default arguments", zookeeper_path, replica_name); engine_args[arg_num]->as()->value = zookeeper_path = server_settings.default_replica_path; engine_args[arg_num + 1]->as()->value = replica_name = server_settings.default_replica_name; diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql index a585ef1c324..c689542e4c3 100644 --- a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql @@ -7,6 +7,8 @@ DROP TABLE IF EXISTS rmt1; DROP TABLE IF EXISTS rmt2; DROP TABLE IF EXISTS rmt3; +SET database_replicated_allow_replicated_engine_arguments=1; + CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n; SHOW CREATE TABLE rmt; RENAME TABLE rmt TO rmt1; diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index a0f228e6af4..7b600ade9a0 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO us ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 36 }" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference index 1d3a90daccd..1241d9714f9 100644 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference @@ -6,5 +6,5 @@ rmt1 rmt2 02858000-1000-4000-8000-0000000000 0 -CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/default/auto_{shard}\', \'1auto_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/auto_{shard}\', \'1auto_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh index 1b5d1a06a9b..b968c02a07a 100755 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh @@ -1,33 +1,36 @@ #!/usr/bin/env bash +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh + db=$CLICKHOUSE_DATABASE if [[ $($CLICKHOUSE_CLIENT -q "SELECT engine = 'Replicated' FROM system.databases WHERE name='$CLICKHOUSE_DATABASE'") != 1 ]]; then $CLICKHOUSE_CLIENT -q "CREATE DATABASE rdb_$CLICKHOUSE_DATABASE ENGINE=Replicated('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rdb', '1', '1')" db="rdb_$CLICKHOUSE_DATABASE" fi -$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=0 -q "CREATE TABLE $db.m0 +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=0 -q "CREATE TABLE $db.m0 UUID '02858000-1000-4000-8000-000000000000' (n int) ENGINE=Memory" 2>&1| grep -Fac "database_replicated_allow_explicit_uuid" -$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1 +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1 UUID '02858000-1000-4000-8000-0000000000$(($RANDOM % 10))$(($RANDOM % 10))' (n int) ENGINE=Memory" -$CLICKHOUSE_CLIENT --database_replicated_allow_explicit_uuid=2 -q "CREATE TABLE $db.m2 +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=2 -q "CREATE TABLE $db.m2 UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory" -$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int) +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" -$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n" +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int) +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY n" -$CLICKHOUSE_CLIENT --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY n" +$CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int) +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY n" $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name" From 6c4c17f119fb95d66f894a92ce8c91fa6664ff5b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jul 2024 16:44:26 +0200 Subject: [PATCH 0225/1722] remove use_hive_partitioning from query settings --- src/Storages/ObjectStorage/StorageObjectStorage.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 3fbfc3aacd7..f97d2620fe5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -43,7 +43,6 @@ public: size_t list_object_keys_size; bool throw_on_zero_files_match; bool ignore_non_existent_file; - bool use_hive_partitioning; }; StorageObjectStorage( From 0e6fd2251e9048bbdb8535c3af0971fb0c670350 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 5 Jul 2024 17:15:04 +0200 Subject: [PATCH 0226/1722] fix --- .../0_stateless/02858_explicit_uuid_and_zk_path.reference | 2 +- .../queries/0_stateless/02858_explicit_uuid_and_zk_path.sh | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference index 1241d9714f9..cb6d0135adf 100644 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference @@ -6,5 +6,5 @@ rmt1 rmt2 02858000-1000-4000-8000-0000000000 0 -CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/auto_{shard}\', \'1auto_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh index b968c02a07a..a959446c822 100755 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh @@ -24,13 +24,13 @@ UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=0 -q "CREATE TABLE $db.rmt0 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}', '_{replica}') ORDER BY n" 2>&1| grep -Fac "database_replicated_allow_replicated_engine_arguments" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=1 -q "CREATE TABLE $db.rmt1 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY n" +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}', '_{replica}') ORDER BY n" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_replicated_engine_arguments=2 -q "CREATE TABLE $db.rmt2 (n int) -ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY n" +ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}', '_{replica}') ORDER BY n" $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name" From 9064bb1b8389e301f45bc78a5365665292f51c6e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jul 2024 19:36:59 +0200 Subject: [PATCH 0227/1722] fix settings in tests --- .../test_storage_azure_blob_storage/test.py | 12 ++++++------ tests/integration/test_storage_hdfs/test.py | 6 +++--- .../0_stateless/03203_hive_style_partitioning.sh | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f2a1f9e35a9..6966abfee4f 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1484,7 +1484,7 @@ def test_hive_partitioning_with_one_parameter(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}')" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( bucket="cont", max_path=path @@ -1497,7 +1497,7 @@ def test_hive_partitioning_with_one_parameter(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Gordon"] @@ -1521,7 +1521,7 @@ def test_hive_partitioning_with_two_parameters(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}\tElizabeth\tGordon".format( bucket="cont", max_path=path @@ -1534,7 +1534,7 @@ def test_hive_partitioning_with_two_parameters(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Elizabeth"] query = ( @@ -1543,7 +1543,7 @@ def test_hive_partitioning_with_two_parameters(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column2=_column2 AND column1=_column1;" ) assert azure_query( - node, query, settings={"azure_blob_storage_hive_partitioning": 1} + node, query, settings={"use_hive_partitioning": 1} ).splitlines() == ["Elizabeth"] @@ -1571,4 +1571,4 @@ def test_hive_partitioning_without_setting(cluster): ) with pytest.raises(Exception, match=pattern): - azure_query(node, query, settings={"azure_blob_storage_hive_partitioning": 0}) + azure_query(node, query, settings={"use_hive_partitioning": 0}) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index da46756841d..aa3efb8ba4a 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1188,7 +1188,7 @@ def test_hive_partitioning_with_one_parameter(started_cluster): r = node1.query( "SELECT _column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", - settings={"hdfs_hive_partitioning": 1}, + settings={"use_hive_partitioning": 1}, ) assert r == f"Elizabeth\n" @@ -1205,7 +1205,7 @@ def test_hive_partitioning_with_two_parameters(started_cluster): r = node1.query( "SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 1}, + settings={"use_hive_partitioning": 1}, ) assert r == f"Gordon\n" @@ -1226,7 +1226,7 @@ def test_hive_partitioning_without_setting(started_cluster): with pytest.raises(QueryRuntimeException, match=pattern): node1.query( f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"hdfs_hive_partitioning": 0}, + settings={"use_hive_partitioning": 0}, ) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 98c039f3454..544fd17ffff 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ -set file_hive_partitioning = 1; +set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -31,7 +31,7 @@ SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_exi SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" $CLICKHOUSE_LOCAL -n -q """ -set file_hive_partitioning = 0; +set use_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -41,7 +41,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ -set url_hive_partitioning = 1; +set use_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -63,7 +63,7 @@ SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitionin SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" $CLICKHOUSE_LOCAL -n -q """ -set url_hive_partitioning = 0; +set use_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -73,7 +73,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ -set s3_hive_partitioning = 1; +set use_hive_partitioning = 1; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -96,7 +96,7 @@ SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/s """ $CLICKHOUSE_LOCAL -n -q """ -set s3_hive_partitioning = 0; +set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" From e7105a3faa50ba012e6cff1e9e26e62fce6d75ba Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 5 Jul 2024 20:51:30 +0200 Subject: [PATCH 0228/1722] fix --- src/Backups/RestorerFromBackup.cpp | 6 +++++- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../0_stateless/02858_explicit_uuid_and_zk_path.reference | 2 +- .../queries/0_stateless/02858_explicit_uuid_and_zk_path.sh | 4 ++-- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 454a0468e9f..63ca272db0b 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -903,11 +903,15 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name) table_info.database = DatabaseCatalog::instance().getDatabase(table_name.database); DatabasePtr database = table_info.database; + auto query_context = Context::createCopy(context); + query_context->setSetting("database_replicated_allow_explicit_uuid", 3); + query_context->setSetting("database_replicated_allow_replicated_engine_arguments", 3); + /// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some /// database-specific things). database->createTableRestoredFromBackup( create_table_query, - context, + query_context, restore_coordination, std::chrono::duration_cast(create_table_timeout).count()); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 2862a907bec..89f503f5fb1 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1055,7 +1055,7 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data bool from_path = create.attach_from_path.has_value(); bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; - if (database->getEngineName() == "Replicated" && create.uuid != UUIDHelpers::Nil && !is_replicated_database_internal) + if (database->getEngineName() == "Replicated" && create.uuid != UUIDHelpers::Nil && !is_replicated_database_internal && !create.attach) { if (getContext()->getSettingsRef().database_replicated_allow_explicit_uuid == 0) { diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference index cb6d0135adf..6444e10bb48 100644 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference @@ -4,7 +4,7 @@ m1 m2 rmt1 rmt2 -02858000-1000-4000-8000-0000000000 +02858000-1000-4000-8000-000000000 0 CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh index a959446c822..b011aed613a 100755 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allo UUID '02858000-1000-4000-8000-000000000000' (n int) ENGINE=Memory" 2>&1| grep -Fac "database_replicated_allow_explicit_uuid" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=1 -q "CREATE TABLE $db.m1 -UUID '02858000-1000-4000-8000-0000000000$(($RANDOM % 10))$(($RANDOM % 10))' (n int) ENGINE=Memory" +UUID '02858000-1000-4000-8000-000000000$(($RANDOM % 10))$(($RANDOM % 10))$(($RANDOM % 10))' (n int) ENGINE=Memory" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none --database_replicated_allow_explicit_uuid=2 -q "CREATE TABLE $db.m2 UUID '02858000-1000-4000-8000-000000000002' (n int) ENGINE=Memory" @@ -35,7 +35,7 @@ ENGINE=ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt/{shard}' $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER BY name" -$CLICKHOUSE_CLIENT -q "SELECT substring(toString(uuid) as s, 1, length(s) - 2) FROM system.tables WHERE database='$db' and name='m1'" +$CLICKHOUSE_CLIENT -q "SELECT substring(toString(uuid) as s, 1, length(s) - 3) FROM system.tables WHERE database='$db' and name='m1'" $CLICKHOUSE_CLIENT -q "SELECT toString(uuid) LIKE '02858000%' FROM system.tables WHERE database='$db' and name='m2'" $CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1" From ebb10d7f8fe16e533593178a1778632c00a3c1b7 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 6 Jul 2024 02:12:01 +0000 Subject: [PATCH 0229/1722] add rebuild option in projection and LWD --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/InterpreterDeleteQuery.cpp | 56 +++++++++++++++---- ...61_lightweight_delete_projection.reference | 3 + .../03161_lightweight_delete_projection.sql | 27 +++++++++ 7 files changed, 80 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a343d864db..bd691fe0dee 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -612,7 +612,7 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection of this table then do lightweight delete, or do lightweight delete then rebuild projections.", 0) \ M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6af6b4b15aa..951dd4d74f3 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -61,7 +61,7 @@ static std::initializer_listgetSettingsRef().lightweight_mutation_projection_mode; - if (mode == LightweightMutationProjectionMode::THROW) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DELETE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", - table->getStorageID().getFullTableName()); - } - else if (mode == LightweightMutationProjectionMode::DROP) + + auto dropOrClearProjections = [&](bool isDrop) { std::vector all_projections = metadata_snapshot->projections.getAllRegisteredNames(); - context->setSetting("mutations_sync", Field(context->getSettingsRef().lightweight_deletes_sync)); - /// Drop projections first so that lightweight delete can be performed. for (const auto & projection : all_projections) { String alter_query = "ALTER TABLE " + table->getStorageID().getFullTableName() + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " DROP PROJECTION IF EXISTS " + projection; + + (isDrop ? " DROP" : " CLEAR") +" PROJECTION " + projection; ParserAlterQuery parser; ASTPtr alter_ast = parseQuery( @@ -151,6 +143,48 @@ BlockIO InterpreterDeleteQuery::execute() InterpreterAlterQuery alter_interpreter(alter_ast, context); alter_interpreter.execute(); } + + return all_projections; + }; + + if (mode == LightweightMutationProjectionMode::THROW) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DELETE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + table->getStorageID().getFullTableName()); + } + else if (mode == LightweightMutationProjectionMode::DROP) + { + dropOrClearProjections(true); + } + else if (mode == LightweightMutationProjectionMode::REBUILD) + { + std::vector all_projections{dropOrClearProjections(false)}; + BlockIO res = lightweightDelete(); + + for (const auto & projection : all_projections) + { + String alter_query = + "ALTER TABLE " + table->getStorageID().getFullTableName() + + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) + + " MATERIALIZE PROJECTION " + projection; + + ParserAlterQuery parser; + ASTPtr alter_ast = parseQuery( + parser, + alter_query.data(), + alter_query.data() + alter_query.size(), + "ALTER query", + 0, + DBMS_DEFAULT_MAX_PARSER_DEPTH, + DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + InterpreterAlterQuery alter_interpreter(alter_ast, context); + alter_interpreter.execute(); + } + + return res; } else { diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index c5a6cbab0bc..307d3cb53fc 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,2 +1,5 @@ 1231 John 33 8888 Alice 50 +6666 Ksenia 48 +8888 Alice 50 +p users 3 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index b189388e356..fb32646b46a 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -19,6 +19,8 @@ DELETE FROM users WHERE uid = 8888 SETTINGS lightweight_mutation_projection_mode DELETE FROM users WHERE uid = 6666 SETTINGS lightweight_mutation_projection_mode = 'drop'; +SYSTEM FLUSH LOGS; + -- expecting no projection SELECT name, @@ -29,3 +31,28 @@ WHERE (database = currentDatabase()) AND (`table` = 'users'); SELECT * FROM users ORDER BY uid; DROP TABLE users; + +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p (select * order by age) +) ENGINE = MergeTree order by uid; + +INSERT INTO users VALUES (1231, 'John', 33), (6666, 'Ksenia', 48), (8888, 'Alice', 50); + +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'rebuild'; + +SELECT * FROM users ORDER BY uid; + +SYSTEM FLUSH LOGS; + +-- expecting projection p with 3 rows is active +SELECT + name, + `table`, + rows, +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND active = 1; + +DROP TABLE users; \ No newline at end of file From 6e5e680797f5b2147455826e4e223c27be5039a6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 6 Jul 2024 05:42:21 +0000 Subject: [PATCH 0230/1722] bump libprotobuf-mutator, fix build --- contrib/libprotobuf-mutator | 2 +- src/AggregateFunctions/fuzzers/CMakeLists.txt | 2 +- ..._function_state_deserialization_fuzzer.cpp | 24 +------------------ src/Core/fuzzers/CMakeLists.txt | 2 +- src/Core/fuzzers/names_and_types_fuzzer.cpp | 22 ----------------- src/DataTypes/fuzzers/CMakeLists.txt | 2 +- .../data_type_deserialization_fuzzer.cpp | 22 ----------------- src/Formats/fuzzers/CMakeLists.txt | 2 +- src/Formats/fuzzers/format_fuzzer.cpp | 21 ---------------- src/Storages/fuzzers/CMakeLists.txt | 2 +- .../fuzzers/columns_description_fuzzer.cpp | 21 ---------------- 11 files changed, 7 insertions(+), 115 deletions(-) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index a304ec48dcf..b922c8ab900 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf +Subproject commit b922c8ab9004ef9944982e4f165e2747b13223fa diff --git a/src/AggregateFunctions/fuzzers/CMakeLists.txt b/src/AggregateFunctions/fuzzers/CMakeLists.txt index 907a275b4b3..1ce0c52feb7 100644 --- a/src/AggregateFunctions/fuzzers/CMakeLists.txt +++ b/src/AggregateFunctions/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_functions clickhouse_aggregate_functions) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index a956d9906bc..31fc93e4288 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -12,33 +12,11 @@ #include -#include - +#include #include #include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for aggregate_function_state_deserialization_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Core/fuzzers/CMakeLists.txt b/src/Core/fuzzers/CMakeLists.txt index 51db6fa0b53..61d6b9629eb 100644 --- a/src/Core/fuzzers/CMakeLists.txt +++ b/src/Core/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) -target_link_libraries (names_and_types_fuzzer PRIVATE dbms) +target_link_libraries (names_and_types_fuzzer PRIVATE clickhouse_functions) diff --git a/src/Core/fuzzers/names_and_types_fuzzer.cpp b/src/Core/fuzzers/names_and_types_fuzzer.cpp index 74debedf2a3..6fdd8703014 100644 --- a/src/Core/fuzzers/names_and_types_fuzzer.cpp +++ b/src/Core/fuzzers/names_and_types_fuzzer.cpp @@ -1,29 +1,7 @@ -#include -#include #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for names_and_types_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 939bf5f5e3f..e54ef0a860c 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(data_type_deserialization_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_functions clickhouse_aggregate_functions) diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 7d9a0513d18..0ae325871fb 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -8,33 +8,11 @@ #include #include -#include - #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for data_type_deserialization_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Formats/fuzzers/CMakeLists.txt b/src/Formats/fuzzers/CMakeLists.txt index 38009aeec1d..b8a7e78b6e2 100644 --- a/src/Formats/fuzzers/CMakeLists.txt +++ b/src/Formats/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS}) -target_link_libraries(format_fuzzer PRIVATE dbms clickhouse_aggregate_functions) +target_link_libraries(format_fuzzer PRIVATE clickhouse_functions clickhouse_aggregate_functions) diff --git a/src/Formats/fuzzers/format_fuzzer.cpp b/src/Formats/fuzzers/format_fuzzer.cpp index 2c1ec65e54d..27f7d7b292f 100644 --- a/src/Formats/fuzzers/format_fuzzer.cpp +++ b/src/Formats/fuzzers/format_fuzzer.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include @@ -21,26 +20,6 @@ #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for format_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index 719b9b77cd9..7bee2da2e26 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -4,4 +4,4 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.c target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms) clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) -target_link_libraries (columns_description_fuzzer PRIVATE dbms) +target_link_libraries (columns_description_fuzzer PRIVATE clickhouse_functions) diff --git a/src/Storages/fuzzers/columns_description_fuzzer.cpp b/src/Storages/fuzzers/columns_description_fuzzer.cpp index ac285ea50f7..e10e0cc52f5 100644 --- a/src/Storages/fuzzers/columns_description_fuzzer.cpp +++ b/src/Storages/fuzzers/columns_description_fuzzer.cpp @@ -1,28 +1,7 @@ -#include #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class IFunctionBase; -using FunctionBasePtr = std::shared_ptr; - -FunctionBasePtr createFunctionBaseCast( - ContextPtr, const char *, const ColumnsWithTypeAndName &, const DataTypePtr &, std::optional, CastType) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversions are not implemented for columns_description_fuzzer"); -} - -} - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try From 88736a0d74b068f271ee21c69760128f82adbb91 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 7 Jul 2024 21:12:26 +0200 Subject: [PATCH 0231/1722] Update JSONCompactWithProgressRowOutputFormat to print JSON on each row --- src/Formats/JSONUtils.cpp | 64 +++++++++++++++++++ src/Formats/JSONUtils.h | 10 +++ ...JSONCompactWithProgressRowOutputFormat.cpp | 39 ++++++++++- .../JSONCompactWithProgressRowOutputFormat.h | 4 ++ 4 files changed, 115 insertions(+), 2 deletions(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index f0985f4a6b7..16083441f3a 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -483,6 +483,33 @@ namespace JSONUtils writeArrayEnd(out, 1); } + + void writeCompactMetadata(const Names & names, const DataTypes & types, const FormatSettings & settings, WriteBuffer & out) + { + writeCompactArrayStart(out, 0, "meta"); + + for (size_t i = 0; i < names.size(); ++i) + { + writeCompactObjectStart(out); + writeTitle("name", out, 0, ""); + + /// The field names are pre-escaped to be put into JSON string literal. + writeChar('"', out); + writeString(names[i], out); + writeChar('"', out); + + writeFieldCompactDelimiter(out); + writeTitle("type", out, 0, ""); + writeJSONString(types[i]->getName(), out, settings); + writeCompactObjectEnd(out); + + if (i + 1 < names.size()) + writeFieldCompactDelimiter(out); + } + + writeCompactArrayEnd(out); + } + void writeAdditionalInfo( size_t rows, size_t rows_before_limit, @@ -523,6 +550,43 @@ namespace JSONUtils } } + void writeCompactAdditionalInfo( + size_t rows, + size_t rows_before_limit, + bool applied_limit, + const Stopwatch & watch, + const Progress & progress, + bool write_statistics, + WriteBuffer & out) + { + writeCompactObjectStart(out, 0, "statistics"); + writeTitle("rows", out, 0, ""); + writeIntText(rows, out); + writeFieldCompactDelimiter(out); + + if (applied_limit) + { + writeTitle("rows_before_limit_at_least", out, 0, ""); + writeIntText(rows_before_limit, out); + writeFieldCompactDelimiter(out); + } + + if (write_statistics) + { + writeTitle("elapsed", out, 0, ""); + writeText(watch.elapsedSeconds(), out); + writeFieldCompactDelimiter(out); + + writeTitle("rows_read", out, 0, ""); + writeText(progress.read_rows.load(), out); + writeFieldCompactDelimiter(out); + + writeTitle("bytes_read", out, 0, ""); + writeText(progress.read_bytes.load(), out); + } + writeCompactObjectEnd(out); + } + void writeException(const String & exception_message, WriteBuffer & out, const FormatSettings & settings, size_t indent) { writeTitle("exception", out, indent, " "); diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index 7ee111c1285..718b9b2f610 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -99,6 +99,7 @@ namespace JSONUtils WriteBuffer & out); void writeMetadata(const Names & names, const DataTypes & types, const FormatSettings & settings, WriteBuffer & out); + void writeCompactMetadata(const Names & names, const DataTypes & types, const FormatSettings & settings, WriteBuffer & out); void writeAdditionalInfo( size_t rows, @@ -109,6 +110,15 @@ namespace JSONUtils bool write_statistics, WriteBuffer & out); + void writeCompactAdditionalInfo( + size_t rows, + size_t rows_before_limit, + bool applied_limit, + const Stopwatch & watch, + const Progress & progress, + bool write_statistics, + WriteBuffer & out); + void writeException(const String & exception_message, WriteBuffer & out, const FormatSettings & settings, size_t indent = 0); void skipColon(ReadBuffer & in); diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 39532fb76fb..7a19b56d0ca 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -17,6 +17,14 @@ JSONCompactWithProgressRowOutputFormat::JSONCompactWithProgressRowOutputFormat( { } +void JSONCompactWithProgressRowOutputFormat::writePrefix() +{ + JSONUtils::writeCompactObjectStart(*ostr); + JSONUtils::writeCompactMetadata(names, types, settings, *ostr); + JSONUtils::writeCompactObjectEnd(*ostr); + writeCString("}\n", *ostr); +} + void JSONCompactWithProgressRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) { JSONUtils::writeFieldFromColumn(column, serialization, row_num, yield_strings, settings, *ostr); @@ -32,16 +40,22 @@ void JSONCompactWithProgressRowOutputFormat::writeRowStartDelimiter() { if (has_progress) writeProgress(); - JSONUtils::writeCompactArrayStart(*ostr, 2); + writeCString("{\"data\":", *ostr); + JSONUtils::writeCompactArrayStart(*ostr); } void JSONCompactWithProgressRowOutputFormat::writeRowEndDelimiter() { JSONUtils::writeCompactArrayEnd(*ostr); + writeCString("}\n", *ostr); field_number = 0; ++row_count; } +void JSONCompactWithProgressRowOutputFormat::writeRowBetweenDelimiter() +{ +} + void JSONCompactWithProgressRowOutputFormat::writeBeforeTotals() { JSONUtils::writeFieldDelimiter(*ostr, 2); @@ -91,7 +105,6 @@ void JSONCompactWithProgressRowOutputFormat::writeSuffix() { if (has_progress) writeProgress(); - JSONRowOutputFormat::writeSuffix(); } void JSONCompactWithProgressRowOutputFormat::writeProgress() @@ -103,6 +116,28 @@ void JSONCompactWithProgressRowOutputFormat::writeProgress() has_progress = false; } +void JSONCompactWithProgressRowOutputFormat::finalizeImpl() +{ + JSONUtils::writeCompactAdditionalInfo( + row_count, + statistics.rows_before_limit, + statistics.applied_limit, + statistics.watch, + statistics.progress, + settings.write_statistics && exception_message.empty(), + *ostr); + + exception_message = "Test exception message."; + if (!exception_message.empty()) + { + writeCString("\n", *ostr); + JSONUtils::writeCompactObjectStart(*ostr); + JSONUtils::writeException(exception_message, *ostr, settings, 0); + JSONUtils::writeCompactObjectEnd(*ostr); + } + ostr->next(); +} + void registerOutputFormatJSONCompactWithProgress(FormatFactory & factory) { factory.registerOutputFormat("JSONCompactWithProgress", []( diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h index 4bc10d41f19..0c7e6b295e8 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h @@ -32,6 +32,7 @@ private: void writeFieldDelimiter() override; void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; + void writeRowBetweenDelimiter() override; bool supportTotals() const override { return true; } bool supportExtremes() const override { return true; } void writeBeforeTotals() override; @@ -40,7 +41,10 @@ private: void writeTotals(const Columns & columns, size_t row_num) override; void writeProgress(); + void writePrefix() override; void writeSuffix() override; + void finalizeImpl() override; + Progress progress; std::vector progress_lines; From 2ae9490a681d3d522dd30e96b9f36b2d638ffd10 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 7 Jul 2024 21:22:02 +0200 Subject: [PATCH 0232/1722] Fix progress rows and bytes read --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 5 ++--- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.h | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 7a19b56d0ca..d72a6556c2b 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -81,11 +81,11 @@ void JSONCompactWithProgressRowOutputFormat::writeExtremesElement(const char * t void JSONCompactWithProgressRowOutputFormat::onProgress(const Progress & value) { - progress.incrementPiecewiseAtomically(value); + statistics.progress.incrementPiecewiseAtomically(value); String progress_line; WriteBufferFromString buf(progress_line); writeCString("{\"progress\":", buf); - progress.writeJSON(buf); + statistics.progress.writeJSON(buf); writeCString("}\n", buf); buf.finalize(); std::lock_guard lock(progress_lines_mutex); @@ -127,7 +127,6 @@ void JSONCompactWithProgressRowOutputFormat::finalizeImpl() settings.write_statistics && exception_message.empty(), *ostr); - exception_message = "Test exception message."; if (!exception_message.empty()) { writeCString("\n", *ostr); diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h index 0c7e6b295e8..669e342c583 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h @@ -46,7 +46,6 @@ private: void finalizeImpl() override; - Progress progress; std::vector progress_lines; std::mutex progress_lines_mutex; /// To not lock mutex and check progress_lines every row, From 2469647609c53ac968d7fba60867c78ecebe522c Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 7 Jul 2024 21:30:36 +0200 Subject: [PATCH 0233/1722] Fix statistics section --- src/Formats/JSONUtils.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 16083441f3a..0918658cf6f 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -559,6 +559,7 @@ namespace JSONUtils bool write_statistics, WriteBuffer & out) { + writeCompactObjectStart(out); writeCompactObjectStart(out, 0, "statistics"); writeTitle("rows", out, 0, ""); writeIntText(rows, out); @@ -585,6 +586,7 @@ namespace JSONUtils writeText(progress.read_bytes.load(), out); } writeCompactObjectEnd(out); + writeCompactObjectEnd(out); } void writeException(const String & exception_message, WriteBuffer & out, const FormatSettings & settings, size_t indent) From c13371166c9a9545155c9274d36c3b91a1ade5cb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Sun, 7 Jul 2024 22:25:15 +0000 Subject: [PATCH 0234/1722] add import re to tests --- tests/integration/test_storage_hdfs/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index da46756841d..9a166cba2ab 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -2,6 +2,7 @@ import os import pytest import time +import re from helpers.cluster import ClickHouseCluster, is_arm from helpers.client import QueryRuntimeException from helpers.test_tools import TSV From 00e47d64b0c12023d926b383c6dea327e9806fc7 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Mon, 8 Jul 2024 09:02:07 +0200 Subject: [PATCH 0235/1722] Add JSONCompactWithProgressStrings format --- .../Impl/JSONCompactWithProgressRowOutputFormat.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index d72a6556c2b..91608cd5050 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -146,6 +146,14 @@ void registerOutputFormatJSONCompactWithProgress(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings, false); }); + + factory.registerOutputFormat("JSONCompactWithProgressStrings", []( + WriteBuffer & buf, + const Block & sample, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, format_settings, true); + }); } } From 58d34e306cec5b41ac372934e43639c2df24e54e Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Mon, 8 Jul 2024 09:08:03 +0200 Subject: [PATCH 0236/1722] Add documentation on JSONCompactWithProgress --- docs/en/interfaces/formats.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ffdd7e2ca25..6161cddefdd 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -38,6 +38,7 @@ The supported formats are: | [JSONCompact](#jsoncompact) | ✔ | ✔ | | [JSONCompactStrings](#jsoncompactstrings) | ✗ | ✔ | | [JSONCompactColumns](#jsoncompactcolumns) | ✔ | ✔ | +| [JSONCompactWithProgress](#jsoncompactwithprogress) | ✗ | ✔ | | [JSONEachRow](#jsoneachrow) | ✔ | ✔ | | [PrettyJSONEachRow](#prettyjsoneachrow) | ✗ | ✔ | | [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ | @@ -925,6 +926,23 @@ Example: Columns that are not present in the block will be filled with default values (you can use [input_format_defaults_for_omitted_fields](/docs/en/operations/settings/settings-formats.md/#input_format_defaults_for_omitted_fields) setting here) +## JSONCompactWithProgress (#jsoncompactwithprogress) + +In this format, ClickHouse outputs each row as a separated, newline-delimited JSON Object. + +Each row is either a metadata object, data object, progress information or statistics object. + +Example: + +```json +{"meta": [{"name":"id", "type":"UInt32"}, {"name":"name", "type":"String"}]}} +{"progress":{"read_rows":"8","read_bytes":"168","written_rows":"0","written_bytes":"0","total_rows_to_read":"2","result_rows":"0","result_bytes":"0","elapsed_ns":"0"}} +{"data":["1", "John Doe"]} +{"data":["2", "Joe Doe"]} +{"statistics": {"rows":2, "rows_before_limit_at_least":8, "elapsed":0.001995, "rows_read":8, "bytes_read":168}} +``` + + ## JSONEachRow {#jsoneachrow} In this format, ClickHouse outputs each row as a separated, newline-delimited JSON Object. From 78cb02503f05d1c1c84dbb17fd51700d74f697df Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Mon, 8 Jul 2024 09:22:07 +0200 Subject: [PATCH 0237/1722] Remove extra space --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 91608cd5050..6be2d092305 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -111,7 +111,7 @@ void JSONCompactWithProgressRowOutputFormat::writeProgress() { std::lock_guard lock(progress_lines_mutex); for (const auto & progress_line : progress_lines) - writeString(progress_line, *ostr); + writeString(progress_line, *ostr); progress_lines.clear(); has_progress = false; } From 7fbcaeaae2bddc07898fd4e2eb7aa9331e2d0df5 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Mon, 8 Jul 2024 09:29:14 +0200 Subject: [PATCH 0238/1722] Remove wrong comment --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h index 669e342c583..dc7a70229e7 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h @@ -11,8 +11,6 @@ namespace DB struct FormatSettings; -/** The stream for outputting data in the JSONCompact- formats. - */ class JSONCompactWithProgressRowOutputFormat final : public JSONRowOutputFormat { public: From a3360d0e63c08132ecd73423d2dd9fc576b54589 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Mon, 8 Jul 2024 13:53:24 +0200 Subject: [PATCH 0239/1722] Fix code style --- ...JSONCompactWithProgressRowOutputFormat.cpp | 29 +++++++------------ .../JSONCompactWithProgressRowOutputFormat.h | 6 +--- 2 files changed, 11 insertions(+), 24 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 6be2d092305..9603ce0265d 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -1,6 +1,6 @@ -#include #include #include +#include #include @@ -9,10 +9,7 @@ namespace DB { JSONCompactWithProgressRowOutputFormat::JSONCompactWithProgressRowOutputFormat( - WriteBuffer & out_, - const Block & header, - const FormatSettings & settings_, - bool yield_strings_) + WriteBuffer & out_, const Block & header, const FormatSettings & settings_, bool yield_strings_) : JSONRowOutputFormat(out_, header, settings_, yield_strings_) { } @@ -139,21 +136,15 @@ void JSONCompactWithProgressRowOutputFormat::finalizeImpl() void registerOutputFormatJSONCompactWithProgress(FormatFactory & factory) { - factory.registerOutputFormat("JSONCompactWithProgress", []( - WriteBuffer & buf, - const Block & sample, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, format_settings, false); - }); + factory.registerOutputFormat( + "JSONCompactWithProgress", + [](WriteBuffer & buf, const Block & sample, const FormatSettings & format_settings) + { return std::make_shared(buf, sample, format_settings, false); }); - factory.registerOutputFormat("JSONCompactWithProgressStrings", []( - WriteBuffer & buf, - const Block & sample, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, format_settings, true); - }); + factory.registerOutputFormat( + "JSONCompactWithProgressStrings", + [](WriteBuffer & buf, const Block & sample, const FormatSettings & format_settings) + { return std::make_shared(buf, sample, format_settings, true); }); } } diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h index dc7a70229e7..1c21914d8cb 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.h @@ -14,11 +14,7 @@ struct FormatSettings; class JSONCompactWithProgressRowOutputFormat final : public JSONRowOutputFormat { public: - JSONCompactWithProgressRowOutputFormat( - WriteBuffer & out_, - const Block & header, - const FormatSettings & settings_, - bool yield_strings_); + JSONCompactWithProgressRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_, bool yield_strings_); String getName() const override { return "JSONCompactWithProgressRowOutputFormat"; } From 7e790d80845fcb3ae955a773c22ab7a96a595d4b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 8 Jul 2024 13:25:51 +0000 Subject: [PATCH 0240/1722] fix rename --- src/Storages/MergeTree/MergeTreeData.cpp | 50 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 16 ++++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 +++--- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 +- src/Storages/StorageMergeTree.cpp | 16 +++--- src/Storages/StorageMergeTree.h | 3 +- 6 files changed, 69 insertions(+), 35 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6083a441a35..d9529b6870c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8668,33 +8668,59 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -static void updateAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, Int64 increment) +static void updateMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + Int64 increment) { - if (num_alter_conversions < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data alter conversions counter is negative ({})", num_alter_conversions); + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + + bool has_data_mutation = false; + bool has_metadata_mutation = false; for (const auto & command : commands) { - if (AlterConversions::isSupportedDataMutation(command.type) || AlterConversions::isSupportedMetadataMutation(command.type)) + if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - num_alter_conversions += increment; + data_mutations_to_apply += increment; + has_data_mutation = true; - if (num_alter_conversions < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_alter_conversions); + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + } - return; + if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) + { + metadata_mutations_to_apply += increment; + has_metadata_mutation = true; + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); } } } -void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) { - updateAlterConversionsCounter(num_alter_conversions, commands, 1); + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); } -void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) { - updateAlterConversionsCounter(num_alter_conversions, commands, -1); + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b0773cb4a20..75a7553d157 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -453,9 +453,6 @@ public: Int64 metadata_version = -1; Int64 min_part_metadata_version = -1; bool need_data_mutations = false; - - bool needAnyMutations() const { return need_data_mutations || needMetadataMutations(); } - bool needMetadataMutations() const { return min_part_metadata_version < metadata_version; } }; Params params; @@ -1776,7 +1773,16 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated -void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); -void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); + +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 807fbeebfc4..0175e427079 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -951,7 +951,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementAlterConversionsCounter(num_alter_conversions, commands, state_lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, state_lock); } else it = mutations_by_znode.erase(it); @@ -1001,7 +1001,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; - incrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) @@ -1076,7 +1076,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// decrementAlterConversionsCounter() will be called in updateMutations() + /// decrementMutationsCounters() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1913,7 +1913,7 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo MutationCommands result; bool seen_all_data_mutations = !params.need_data_mutations; - bool seen_all_metadata_mutations = !params.needMetadataMutations(); + bool seen_all_metadata_mutations = part_metadata_version >= params.metadata_version; if (seen_all_data_mutations && seen_all_metadata_mutations) return {}; @@ -1968,8 +1968,8 @@ MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapsh std::lock_guard lock(state_mutex); - bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; - bool need_metatadata_mutations = res->params.needMetadataMutations(); + bool need_data_mutations = params.need_data_mutations && data_mutations_to_apply > 0; + bool need_metatadata_mutations = params.min_part_metadata_version < params.metadata_version; if (!need_data_mutations && !need_metatadata_mutations) return res; @@ -2113,7 +2113,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementAlterConversionsCounter(num_alter_conversions, mutation.entry->commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2170,7 +2170,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 223ca989feb..a46fdaf3ac4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,7 +154,8 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions = 0; + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ef2a7bda118..5c15818632c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -502,7 +502,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -538,7 +538,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementAlterConversionsCounter(num_alter_conversions, *entry.commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); } } else @@ -819,7 +819,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); } to_kill.emplace(std::move(it->second)); @@ -904,7 +904,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementAlterConversionsCounter(num_alter_conversions, *entry_it->second.commands, lock); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2449,10 +2449,10 @@ MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const std::lock_guard lock(currently_processing_in_background_mutex); - bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; - bool need_metatadata_mutations = res->params.needMetadataMutations(); + bool need_data_mutations = res->params.need_data_mutations && data_mutations_to_apply > 0; + bool need_metadata_mutations = metadata_mutations_to_apply > 0; - if (!need_data_mutations && !need_metatadata_mutations) + if (!need_data_mutations && !need_metadata_mutations) return res; for (const auto & [version, entry] : current_mutations_by_version) @@ -2462,7 +2462,7 @@ MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) return true; - if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + if (need_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) return true; return false; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 40e5a8e5ea4..d2ade17e309 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,7 +150,8 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions = 0; + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; From a3c4cbfce257f171e66e04598ac9eae548c3836f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 8 Jul 2024 13:57:54 +0000 Subject: [PATCH 0241/1722] clang-tidy, fix cp with minio --- docker/test/stateless/setup_minio.sh | 2 +- .../03203_hive_style_partitioning.sh | 52 +++++++++---------- utils/keeper-bench/Runner.cpp | 3 +- 3 files changed, 29 insertions(+), 28 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 2b9433edd20..aacb9d88a45 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -101,7 +101,7 @@ upload_data() { # shellcheck disable=SC2045 for file in $(ls "${data_path}"); do echo "${file}"; - ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}"; + ./mc cp --recursive "${data_path}"/"${file}" clickminio/test/"${file}"; done } diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 544fd17ffff..334bfef4f02 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -43,29 +43,29 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM url('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, _column0 FROM url('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -75,28 +75,28 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; """ $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, _column0 FROM s3('http://localhost:11111/test/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 587e015b340..f8a0e37d1a9 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -544,7 +544,8 @@ struct ZooKeeperRequestFromLogReader file_read_buf = DB::wrapReadBufferWithCompressionMethod(std::move(file_read_buf), compression_method); DB::SingleReadBufferIterator read_buffer_iterator(std::move(file_read_buf)); - auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + std::string sample_path; + auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); DB::ColumnsWithTypeAndName columns; columns.reserve(columns_description.size()); From 3cc6a133c64861f4493849905950abb5cc1fbaac Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 9 Jul 2024 12:38:16 +0200 Subject: [PATCH 0242/1722] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index aacb9d88a45..0e344cbb9c4 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -101,7 +101,7 @@ upload_data() { # shellcheck disable=SC2045 for file in $(ls "${data_path}"); do echo "${file}"; - ./mc cp --recursive "${data_path}"/"${file}" clickminio/test/"${file}"; + ./mc cp --recursive "${data_path}"/ clickminio/test/; done } @@ -148,4 +148,4 @@ main() { setup_aws_credentials } -main "$@" \ No newline at end of file +main "$@" From 362bf4befcd55de5f49e2665c7c7f9483a700dc8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 9 Jul 2024 17:30:08 +0200 Subject: [PATCH 0243/1722] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 0e344cbb9c4..8bd75f16321 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -101,7 +101,7 @@ upload_data() { # shellcheck disable=SC2045 for file in $(ls "${data_path}"); do echo "${file}"; - ./mc cp --recursive "${data_path}"/ clickminio/test/; + ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}"; done } From 1761102b3a071143b40039ee1e83666ebffa88fb Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 9 Jul 2024 17:31:41 +0200 Subject: [PATCH 0244/1722] fix path --- .../column1=Gordon/sample.parquet | Bin .../column1=Schmidt/sample.parquet | Bin .../sample.parquet | Bin 3 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/data_minio/hive_partitioning/{coumn0=Elizabeth => column0=Elizabeth}/column1=Gordon/sample.parquet (100%) rename tests/queries/0_stateless/data_minio/hive_partitioning/{coumn0=Elizabeth => column0=Elizabeth}/column1=Schmidt/sample.parquet (100%) rename tests/queries/0_stateless/data_minio/hive_partitioning/{coumn0=Elizabeth => column0=Elizabeth}/sample.parquet (100%) diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Gordon/sample.parquet rename to tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/column1=Schmidt/sample.parquet rename to tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet diff --git a/tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_minio/hive_partitioning/coumn0=Elizabeth/sample.parquet rename to tests/queries/0_stateless/data_minio/hive_partitioning/column0=Elizabeth/sample.parquet From 6cb361413eff6f870eb2bd43876d82d1c2c5e882 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 9 Jul 2024 16:45:15 +0200 Subject: [PATCH 0245/1722] fix --- src/Databases/DatabaseReplicated.cpp | 3 +++ tests/integration/test_disk_over_web_server/test.py | 3 ++- .../00510_materizlized_view_and_deduplication_zookeeper.sql | 2 ++ tests/queries/0_stateless/00609_mv_index_in_in.sql | 1 + tests/queries/0_stateless/00738_lock_for_inner_table.sh | 2 ++ tests/queries/0_stateless/01153_attach_mv_uuid.sql | 1 + .../0_stateless/02858_explicit_uuid_and_zk_path.reference | 4 ++-- tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh | 4 ++-- .../0_stateless/02888_replicated_merge_tree_creation.sh | 1 + 9 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index bc9b2537d1e..25d4a3eb704 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1008,6 +1008,9 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("allow_create_index_without_type", 1); query_context->setSetting("allow_experimental_s3queue", 1); + query_context->setSetting("database_replicated_allow_explicit_uuid", 3); + query_context->setSetting("database_replicated_allow_replicated_engine_arguments", 3); + auto txn = std::make_shared(current_zookeeper, zookeeper_path, false, ""); query_context->initZooKeeperMetadataTransaction(txn); return query_context; diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 891ee8f00f5..ec0bef23731 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -311,7 +311,8 @@ def test_replicated_database(cluster): SETTINGS storage_policy = 'web'; """.format( uuids[0] - ) + ), + settings={"database_replicated_allow_explicit_uuid": 3}, ) node2 = cluster.instances["node2"] diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index d3c4da86b41..5d2f8192d04 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -6,6 +6,8 @@ DROP TABLE IF EXISTS without_deduplication; DROP TABLE IF EXISTS with_deduplication_mv; DROP TABLE IF EXISTS without_deduplication_mv; +SET database_replicated_allow_explicit_uuid=3; +SET database_replicated_allow_replicated_engine_arguments=3; CREATE TABLE with_deduplication(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x; CREATE TABLE without_deduplication(x UInt32) diff --git a/tests/queries/0_stateless/00609_mv_index_in_in.sql b/tests/queries/0_stateless/00609_mv_index_in_in.sql index bd9f35350c1..89fc85651ad 100644 --- a/tests/queries/0_stateless/00609_mv_index_in_in.sql +++ b/tests/queries/0_stateless/00609_mv_index_in_in.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS test_mv_00609; create table test_00609 (a Int8) engine=Memory; insert into test_00609 values (1); +set database_replicated_allow_explicit_uuid=3; set allow_deprecated_syntax_for_merge_tree=1; create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609; diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index b62a639d8f4..9bc84dd1063 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -7,6 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --database_replicated_allow_explicit_uuid 3" + # there are some issues with Atomic database, let's generate it uniq # otherwise flaky check will not pass. uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())") diff --git a/tests/queries/0_stateless/01153_attach_mv_uuid.sql b/tests/queries/0_stateless/01153_attach_mv_uuid.sql index 00cce8a1de4..6b167253d8f 100644 --- a/tests/queries/0_stateless/01153_attach_mv_uuid.sql +++ b/tests/queries/0_stateless/01153_attach_mv_uuid.sql @@ -14,6 +14,7 @@ INSERT INTO src VALUES (3), (4); SELECT * FROM mv ORDER BY n; DROP TABLE mv SYNC; +SET database_replicated_allow_explicit_uuid=3; SET show_table_uuid_in_table_create_query_if_not_nil=1; CREATE TABLE ".inner_id.e15f3ab5-6cae-4df3-b879-f40deafd82c2" (n Int32, n2 Int64) ENGINE = MergeTree PARTITION BY n % 10 ORDER BY n; ATTACH MATERIALIZED VIEW mv UUID 'e15f3ab5-6cae-4df3-b879-f40deafd82c2' (n Int32, n2 Int64) ENGINE = MergeTree PARTITION BY n % 10 ORDER BY n AS SELECT n, n * n AS n2 FROM src; diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference index 6444e10bb48..874494fb061 100644 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.reference @@ -6,5 +6,5 @@ rmt1 rmt2 02858000-1000-4000-8000-000000000 0 -CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 -CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh index b011aed613a..81a9cef02ff 100755 --- a/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh +++ b/tests/queries/0_stateless/02858_explicit_uuid_and_zk_path.sh @@ -38,7 +38,7 @@ $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER $CLICKHOUSE_CLIENT -q "SELECT substring(toString(uuid) as s, 1, length(s) - 3) FROM system.tables WHERE database='$db' and name='m1'" $CLICKHOUSE_CLIENT -q "SELECT toString(uuid) LIKE '02858000%' FROM system.tables WHERE database='$db' and name='m2'" -$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1" -$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2" +$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1" | sed "s/$db/default/g" +$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2" | sed "s/$db/default/g" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS rdb_$CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh index b9603e75d2e..466f0d01a7f 100755 --- a/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh +++ b/tests/queries/0_stateless/02888_replicated_merge_tree_creation.sh @@ -5,6 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --database_replicated_allow_explicit_uuid 3 --database_replicated_allow_replicated_engine_arguments 3" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC" From 9db80a6e2d14c6341c7afc66aeaf6998c98f9f8a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 9 Jul 2024 17:47:05 +0000 Subject: [PATCH 0246/1722] more testing with chunked --- programs/benchmark/Benchmark.cpp | 4 ++-- src/Client/ConnectionParameters.cpp | 4 ++-- src/Client/ConnectionParameters.h | 4 ++-- src/Dictionaries/ClickHouseDictionarySource.cpp | 8 ++++---- src/Interpreters/Cluster.cpp | 4 ++-- src/Interpreters/Cluster.h | 4 ++-- 6 files changed, 14 insertions(+), 14 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 251761e0bad..0a7faf5ec01 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -666,8 +666,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); - String proto_send_chunked {"notchunked_optional"}; - String proto_recv_chunked {"notchunked_optional"}; + String proto_send_chunked {"chunked"}; + String proto_recv_chunked {"chunked"}; if (options.count("proto_caps")) { diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 4bca65083c4..50af589dba3 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -107,8 +107,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } } - proto_send_chunked = config.getString("proto_caps.send", "notchunked_optional"); - proto_recv_chunked = config.getString("proto_caps.recv", "notchunked_optional"); + proto_send_chunked = config.getString("proto_caps.send", "chunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); quota_key = config.getString("quota_key", ""); diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index 71057a2b543..ef4df17143e 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,8 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; - std::string proto_send_chunked = "notchunked_optional"; - std::string proto_recv_chunked = "notchunked_optional"; + std::string proto_send_chunked = "chunked"; + std::string proto_recv_chunked = "chunked"; std::string quota_key; SSHKey ssh_private_key; std::string jwt; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 3b096da92c6..14c6aac24f6 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -236,8 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), - .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked_optional"), - .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked_optional"), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "chunked"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "chunked"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -262,8 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), - .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked_optional"), - .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked_optional"), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "chunked"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "chunked"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 1d7ccd484d0..9b227fcc1fc 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,8 +113,8 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; - proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked_optional"); - proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked_optional"); + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "chunked"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "chunked"); const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index f3146ac0134..009ef15df6c 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,8 +114,8 @@ public: UInt16 port{0}; String user; String password; - String proto_send_chunked = "notchunked_optional"; - String proto_recv_chunked = "notchunked_optional"; + String proto_send_chunked = "chunked"; + String proto_recv_chunked = "chunked"; String quota_key; /// For inter-server authorization From d7f08ffdb74b4fce89eff3133e36a5f50fc4ef0b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Jul 2024 21:01:37 +0200 Subject: [PATCH 0247/1722] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 8bd75f16321..49837fdb1ac 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -99,10 +99,7 @@ upload_data() { # iterating over globs will cause redundant file variable to be # a path to a file, not a filename # shellcheck disable=SC2045 - for file in $(ls "${data_path}"); do - echo "${file}"; - ./mc cp "${data_path}"/"${file}" clickminio/test/"${file}"; - done + ./mc cp --recursive "${data_path}"/ clickminio/test/ } setup_aws_credentials() { From 2794b7bf84faf91cfb92d4a8fb76bb3a8183de44 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 10 Jul 2024 00:20:11 +0000 Subject: [PATCH 0248/1722] defaults to notchunked, add docs to server's and client's configs --- programs/benchmark/Benchmark.cpp | 4 ++-- programs/client/clickhouse-client.xml | 15 +++++++++++++++ programs/server/config.xml | 15 +++++++++++++++ src/Client/ConnectionParameters.cpp | 4 ++-- src/Client/ConnectionParameters.h | 4 ++-- src/Dictionaries/ClickHouseDictionarySource.cpp | 8 ++++---- src/Interpreters/Cluster.cpp | 4 ++-- src/Interpreters/Cluster.h | 4 ++-- src/Server/TCPHandler.cpp | 8 ++++---- 9 files changed, 48 insertions(+), 18 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 0a7faf5ec01..36f774a3c12 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -666,8 +666,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) Strings hosts = options.count("host") ? options["host"].as() : Strings({"localhost"}); - String proto_send_chunked {"chunked"}; - String proto_recv_chunked {"chunked"}; + String proto_send_chunked {"notchunked"}; + String proto_recv_chunked {"notchunked"}; if (options.count("proto_caps")) { diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index d0deb818c1e..376e64906e2 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -37,6 +37,21 @@ {display_name} \e[1;31m:)\e[0m + + + 9000 + + + diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 50af589dba3..4d0a9ffa08c 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -107,8 +107,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } } - proto_send_chunked = config.getString("proto_caps.send", "chunked"); - proto_recv_chunked = config.getString("proto_caps.recv", "chunked"); + proto_send_chunked = config.getString("proto_caps.send", "notchunked"); + proto_recv_chunked = config.getString("proto_caps.recv", "notchunked"); quota_key = config.getString("quota_key", ""); diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index ef4df17143e..382bfe34a3d 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -20,8 +20,8 @@ struct ConnectionParameters std::string default_database; std::string user; std::string password; - std::string proto_send_chunked = "chunked"; - std::string proto_recv_chunked = "chunked"; + std::string proto_send_chunked = "notchunked"; + std::string proto_recv_chunked = "notchunked"; std::string quota_key; SSHKey ssh_private_key; std::string jwt; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 14c6aac24f6..b36d53a6159 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -236,8 +236,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = named_collection->getAnyOrDefault({"user", "username"}, "default"), .password = named_collection->getOrDefault("password", ""), - .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "chunked"), - .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "chunked"), + .proto_send_chunked = named_collection->getOrDefault("proto_send_chunked", "notchunked"), + .proto_recv_chunked = named_collection->getOrDefault("proto_recv_chunked", "notchunked"), .quota_key = named_collection->getOrDefault("quota_key", ""), .db = named_collection->getAnyOrDefault({"db", "database"}, default_database), .table = named_collection->getOrDefault("table", ""), @@ -262,8 +262,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .host = host, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), - .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "chunked"), - .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "chunked"), + .proto_send_chunked = config.getString(settings_config_prefix + ".proto_caps.send", "notchunked"), + .proto_recv_chunked = config.getString(settings_config_prefix + ".proto_caps.recv", "notchunked"), .quota_key = config.getString(settings_config_prefix + ".quota_key", ""), .db = config.getString(settings_config_prefix + ".db", default_database), .table = config.getString(settings_config_prefix + ".table", ""), diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 9b227fcc1fc..dd9e35834eb 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -113,8 +113,8 @@ Cluster::Address::Address( secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = Priority{config.getInt(config_prefix + ".priority", 1)}; - proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "chunked"); - proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "chunked"); + proto_send_chunked = config.getString(config_prefix + ".proto_caps.send", "notchunked"); + proto_recv_chunked = config.getString(config_prefix + ".proto_caps.recv", "notchunked"); const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 009ef15df6c..c69d77668ab 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -114,8 +114,8 @@ public: UInt16 port{0}; String user; String password; - String proto_send_chunked = "chunked"; - String proto_recv_chunked = "chunked"; + String proto_send_chunked = "notchunked"; + String proto_recv_chunked = "notchunked"; String quota_key; /// For inter-server authorization diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 40fd3848455..9c5e5e9c572 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -310,8 +310,8 @@ void TCPHandler::runImpl() return chunked_srv; }; - bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "chunked_optional"), proto_recv_chunked_cl, "send"); - bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "chunked_optional"), proto_send_chunked_cl, "recv"); + bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked"), proto_recv_chunked_cl, "send"); + bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked"), proto_send_chunked_cl, "recv"); if (out_chunked) out->enableChunked(); @@ -1660,8 +1660,8 @@ void TCPHandler::sendHello() writeVarUInt(VERSION_PATCH, *out); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS) { - writeStringBinary(server.config().getString("proto_caps.send", "chunked"), *out); - writeStringBinary(server.config().getString("proto_caps.recv", "chunked"), *out); + writeStringBinary(server.config().getString("proto_caps.send", "notchunked"), *out); + writeStringBinary(server.config().getString("proto_caps.recv", "notchunked"), *out); } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES) { From 7b58722c07e8feb6acca5f0762411a55b8c58915 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 Jul 2024 13:15:44 +0200 Subject: [PATCH 0249/1722] Update setup_minio.sh --- docker/test/stateless/setup_minio.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 49837fdb1ac..02e3d117de2 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -99,7 +99,9 @@ upload_data() { # iterating over globs will cause redundant file variable to be # a path to a file, not a filename # shellcheck disable=SC2045 - ./mc cp --recursive "${data_path}"/ clickminio/test/ + if [ -d "${data_path}" ]; then + ./mc cp --recursive "${data_path}"/ clickminio/test/ + fi } setup_aws_credentials() { From a751719a33e2691426bdb057eaf509a74e84753d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 10 Jul 2024 17:44:06 +0000 Subject: [PATCH 0250/1722] fixes due to review --- docs/en/operations/settings/settings.md | 6 +++++ docs/en/sql-reference/table-functions/file.md | 17 ++++++++++++ docs/en/sql-reference/table-functions/hdfs.md | 17 ++++++++++++ docs/en/sql-reference/table-functions/s3.md | 17 ++++++++++++ programs/obfuscator/Obfuscator.cpp | 3 +-- src/Formats/ReadSchemaUtils.cpp | 11 ++------ src/Formats/ReadSchemaUtils.h | 2 -- src/Storages/Hive/StorageHive.cpp | 2 +- .../DataLakes/IStorageDataLake.h | 4 +-- .../ObjectStorage/StorageObjectStorage.cpp | 14 ++++++---- .../StorageObjectStorageCluster.cpp | 2 +- .../StorageObjectStorageSource.cpp | 11 +++++--- .../StorageObjectStorageSource.h | 3 ++- .../StorageObjectStorageQueue.cpp | 2 +- src/Storages/StorageFile.cpp | 20 +++++++------- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageURL.cpp | 23 +++++++++++----- src/Storages/StorageURLCluster.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 25 +++++++++++------- src/Storages/VirtualColumnUtils.h | 9 +++++-- src/TableFunctions/TableFunctionFormat.cpp | 10 +++---- .../03203_hive_style_partitioning.reference | 5 +++- .../03203_hive_style_partitioning.sh | 11 +++++--- .../array=[1,2,3]/float=42.42/sample.parquet | Bin 0 -> 1308 bytes .../number=42/date=2020-01-01/sample.parquet | Bin 0 -> 1308 bytes 25 files changed, 152 insertions(+), 66 deletions(-) create mode 100644 tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1d74a63b972..e100e0f27f7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5591,3 +5591,9 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## use_hive_partitioning + +Allows the usage of Hive-style partitioning in queries. When enabled, ClickHouse interprets and maintains table partitions in a way that is consistent with the Hive partitioning scheme, which is commonly used in Hadoop ecosystems. + +Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 3a3162dad9a..88af3663552 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -198,6 +198,23 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-patitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_patitioning = 1; +SELECT _specified_column from file('/specified_column=specified_data/file.txt'); +``` + +``` reference +specified_data +``` + ## Settings {#settings} - [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-empty_if-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 28cba5ccc6a..beb1ad12532 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -99,6 +99,23 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-patitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_patitioning = 1; +SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); +``` + +``` reference +specified_data +``` + ## Storage Settings {#storage-settings} - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 35e5d86034c..45c4caa1a13 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -274,6 +274,23 @@ FROM s3( - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. In case of archive shows uncompressed file size of the file inside the archive. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-patitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_patitioning = 1; +SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); +``` + +``` reference +specified_data +``` + ## Storage Settings {#storage-settings} - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 11e85bc1302..4c3981c1d01 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1308,8 +1308,7 @@ try SingleReadBufferIterator read_buffer_iterator(std::move(file)); - std::string sample_string; - schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, sample_string, context_const); + schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, context_const); } else { diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 1e70840f91f..1920459c378 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -94,7 +94,6 @@ std::pair readSchemaFromFormatImpl( std::optional format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context) try { @@ -144,10 +143,6 @@ try { iterator_data = read_buffer_iterator.next(); - /// Extracting the File path for hive-style partitioning - if (sample_path.empty()) - sample_path = read_buffer_iterator.getLastFilePath(); - /// Read buffer iterator can determine the data format if it's unknown. /// For example by scanning schema cache or by finding new file with format extension. if (!format_name && iterator_data.format_name) @@ -541,19 +536,17 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, sample_path, context).first; + return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, context).first; } std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context) { - return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, sample_path, context); + return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, context); } SchemaCache::Key getKeyForSchemaCache( diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index 6c562a06bf0..7168e7f0817 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -122,7 +122,6 @@ ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context); /// Try to detect the format of the data and it's schema. @@ -132,7 +131,6 @@ ColumnsDescription readSchemaFromFormat( std::pair detectFormatAndReadSchema( const std::optional & format_settings, IReadBufferIterator & read_buffer_iterator, - std::string & sample_path, const ContextPtr & context); SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 28d8128e052..255dadea387 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -445,7 +445,7 @@ StorageHive::StorageHive( storage_metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, storage_metadata.columns, getContext()); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), getContext())); } void StorageHive::lazyInitialize() diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index a651be6017f..ec8e740b1c9 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -89,9 +89,9 @@ public: { ConfigurationPtr configuration = base_configuration->clone(); configuration->setPaths(metadata->getDataFiles()); - std::string sample_string; + std::string sample_path; return Storage::resolveSchemaFromData( - object_storage_, configuration, format_settings_, sample_string, local_context); + object_storage_, configuration, format_settings_, sample_path, local_context); } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8ab8b6b6881..48e9118e321 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -43,7 +43,8 @@ std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata {}, // predicate metadata.getColumns().getAll(), // virtual_columns nullptr, // read_keys - {} // file_progress_callback + {}, // file_progress_callback + true // override_settings_for_hive_partitioning ); if (auto file = file_iterator->next(0)) @@ -86,7 +87,7 @@ StorageObjectStorage::StorageObjectStorage( else if (!context->getSettings().use_hive_partitioning) sample_path = ""; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), sample_path)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path)); setInMemoryMetadata(metadata); } @@ -396,7 +397,8 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return readSchemaFromFormat(configuration->format, format_settings, *iterator, sample_path, context); + sample_path = iterator->getLastFilePath(); + return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); } std::string StorageObjectStorage::resolveFormatFromData( @@ -408,7 +410,8 @@ std::string StorageObjectStorage::resolveFormatFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - return detectFormatAndReadSchema(format_settings, *iterator, sample_path, context).second; + sample_path = iterator->getLastFilePath(); + return detectFormatAndReadSchema(format_settings, *iterator, context).second; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( @@ -420,7 +423,8 @@ std::pair StorageObjectStorage::resolveSchemaAn { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, sample_path, context); + sample_path = iterator->getLastFilePath(); + auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); configuration->format = format; return std::pair(columns, format); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 0dc4b845a47..92327b4cde0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -41,7 +41,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setColumns(columns); metadata.setConstraints(constraints_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 88ae0a2319c..e5c9318de5d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -105,7 +105,8 @@ std::shared_ptr StorageObjectStorageSourc const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback) + std::function file_progress_callback, + bool override_settings_for_hive_partitioning) { if (distributed_processing) return std::make_shared( @@ -122,11 +123,14 @@ std::shared_ptr StorageObjectStorageSourc std::unique_ptr iterator; if (configuration->isPathWithGlobs()) { + bool throw_on_zero_files_match = settings.throw_on_zero_files_match; + if (override_settings_for_hive_partitioning) + throw_on_zero_files_match = false; /// Iterate through disclosed globs and make a source for each file iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, - settings.throw_on_zero_files_match, file_progress_callback); + throw_on_zero_files_match, file_progress_callback); } else { @@ -204,7 +208,8 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - }, object_info->getPath()); + .hive_partitioning_path = object_info->getPath(), + }); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 271b38fa75c..a99bb068372 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -58,7 +58,8 @@ public: const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback = {}); + std::function file_progress_callback = {}, + bool override_settings_for_hive_partitioning = false); static std::string getUniqueStoragePathIdentifier( const Configuration & configuration, diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 98f8cdc7e7a..a9239e3ad06 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -168,7 +168,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_)); setInMemoryMetadata(storage_metadata); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9751d596fff..e6b9137444e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ #include #include #include +#include "Formats/FormatSettings.h" #include #include @@ -880,11 +881,10 @@ std::pair StorageFile::getTableStructureAndFormatFro auto read_buffer_iterator = SingleReadBufferIterator(std::move(read_buf)); ColumnsDescription columns; - std::string sample_path; if (format) - columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context); + columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context); else - std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); peekable_read_buffer_from_fd = read_buffer_iterator.releaseBuffer(); if (peekable_read_buffer_from_fd) @@ -929,21 +929,20 @@ std::pair StorageFile::getTableStructureAndFormatFro } - std::string sample_path; if (archive_info) { ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); } ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); } ColumnsDescription StorageFile::getTableStructureFromFile( @@ -1102,7 +1101,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) std::string path_for_virtuals; if (args.getContext()->getSettingsRef().use_hive_partitioning && !paths.empty()) path_for_virtuals = paths[0]; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); } @@ -1456,7 +1455,8 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified, - }, hive_partitioning_path); + .hive_partitioning_path = hive_partitioning_path, + }); return chunk; } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index d43e242f70c..f7684182e79 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -61,7 +61,7 @@ StorageFileCluster::StorageFileCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); } void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 59c5465a381..5da42638b87 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -99,6 +99,17 @@ static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout); } +String getSampleURI(String uri, ContextPtr context) +{ + if (urlWithGlobs(uri)) + { + auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + if (!uris.empty()) + return uris[0]; + } + return uri; +} + IStorageURLBase::IStorageURLBase( const String & uri_, const ContextPtr & context_, @@ -155,8 +166,8 @@ IStorageURLBase::IStorageURLBase( std::string uri_for_partitioning; if (context_->getSettingsRef().use_hive_partitioning) - uri_for_partitioning = uri; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), uri_for_partitioning, format_settings.value_or(FormatSettings{}))); + uri_for_partitioning = getSampleURI(uri, context_); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, uri_for_partitioning, format_settings.value_or(FormatSettings{}))); } @@ -425,7 +436,8 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - }, hive_partitioning_path); + .hive_partitioning_path = hive_partitioning_path, + }); return chunk; } @@ -959,10 +971,9 @@ std::pair IStorageURLBase::getTableStructureAndForma urls_to_check = {uri}; ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context); - std::string sample_path; if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, sample_path, context), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format}; + return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); } ColumnsDescription IStorageURLBase::getTableStructureFromData( diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 2e7c63d0097..592bd71f546 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -75,7 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 379b14d8e51..b7669c65992 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -39,10 +39,13 @@ #include #include #include +#include +#include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" #include "Functions/indexHint.h" +#include #include #include #include @@ -116,7 +119,7 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -std::map parseFromPath(const std::string& path) +std::map parseHivePartitioningKeysAndValues(const std::string& path) { std::string pattern = "/([^/]+)=([^/]+)"; re2::StringPiece input_piece(path); @@ -128,7 +131,7 @@ std::map parseFromPath(const std::string& path) return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path, FormatSettings settings) +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, std::string path, std::optional format_settings_) { VirtualColumnsDescription desc; @@ -145,13 +148,17 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - auto map = parseFromPath(path); - for (const auto& item : map) + auto map = parseHivePartitioningKeysAndValues(path); + for (auto& item : map) { - auto type = tryInferDataTypeForSingleField(item.second, settings); + auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); + auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw); if (type == nullptr) type = std::make_shared(); - add_virtual(item.first, std::make_shared(type)); + if (type->canBeInsideLowCardinality()) + add_virtual(item.first, std::make_shared(type)); + else + add_virtual(item.first, type); } return desc; @@ -215,9 +222,9 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path) + VirtualsForFileLikeStorage virtual_values) { - auto hive_map = parseFromPath(hive_partitioning_path); + auto hive_map = parseHivePartitioningKeysAndValues(virtual_values.hive_partitioning_path); for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -265,7 +272,7 @@ void addRequestedFileLikeStorageVirtualsToChunk( auto it = hive_map.find(virtual_column.getNameInStorage()); if (it != hive_map.end()) { - chunk.addColumn(virtual_column.getTypeInStorage()->createColumnConst(chunk.getNumRows(), it->second)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst()); hive_map.erase(it); } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 72922be60bd..594253a32c1 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -50,7 +50,11 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) } NameSet getVirtualNamesForFileLikeStorage(); -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, std::string path = "", FormatSettings settings = FormatSettings()); +VirtualColumnsDescription getVirtualsForFileLikeStorage( + const ColumnsDescription & storage_columns, + const ContextPtr & context, + std::string sample_path = "", + std::optional format_settings_ = std::nullopt); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); @@ -77,13 +81,14 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; + const String & hive_partitioning_path = ""; }; std::map parseFromPath(const std::string& path); void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, const std::string & hive_partitioning_path = ""); + VirtualsForFileLikeStorage virtual_values); } } diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index 66152cb0c91..ad2a142a140 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -85,10 +85,9 @@ ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr conte if (structure == "auto") { SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); - std::string sample_path; if (format == "auto") - return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context).first; - return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); + return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context).first; + return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); } return parseColumnsListFromString(structure, context); } @@ -132,12 +131,11 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con String format_name = format; if (structure == "auto") { - std::string sample_path; SingleReadBufferIterator read_buffer_iterator(std::make_unique(data)); if (format_name == "auto") - std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, sample_path, context); + std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context); else - columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, sample_path, context); + columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context); } else { diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 6ef1fcdf652..e0f46caf1c8 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -60,7 +60,10 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth +42 2020-01-01 +[1,2,3] 42.42 +Array(Int64) LowCardinality(Float64) +101 1 TESTING THE S3 PARTITIONING first last Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 334bfef4f02..9d805b39b8a 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -28,7 +28,13 @@ SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=E SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; + +SELECT _number, _date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; +SELECT _array, _float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; +SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; +SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; +""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; @@ -59,8 +65,7 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0;""" +SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; diff --git a/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 From 1f33eb32b0c80b9dde27a8d7aa9ad26c271aceae Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 03:02:15 +0000 Subject: [PATCH 0251/1722] try to drop projection correctly --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 3 +- src/Interpreters/InterpreterDeleteQuery.cpp | 61 +------------------ src/Interpreters/MutationsInterpreter.cpp | 6 +- src/Storages/MergeTree/MutateTask.cpp | 7 ++- ...61_lightweight_delete_projection.reference | 5 -- .../03161_lightweight_delete_projection.sql | 33 +--------- 9 files changed, 18 insertions(+), 104 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c884f8f80c4..f7b44ea775c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -588,7 +588,7 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection of this table then do lightweight delete, or do lightweight delete then rebuild projections.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5174cf82c2e..194292a467e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -70,7 +70,7 @@ static std::initializer_listgetSettingsRef().lightweight_mutation_projection_mode; - auto dropOrClearProjections = [&](bool isDrop) - { - std::vector all_projections = metadata_snapshot->projections.getAllRegisteredNames(); - - /// Drop projections first so that lightweight delete can be performed. - for (const auto & projection : all_projections) - { - String alter_query = - "ALTER TABLE " + table->getStorageID().getFullTableName() - + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + (isDrop ? " DROP" : " CLEAR") +" PROJECTION " + projection; - - ParserAlterQuery parser; - ASTPtr alter_ast = parseQuery( - parser, - alter_query.data(), - alter_query.data() + alter_query.size(), - "ALTER query", - 0, - DBMS_DEFAULT_MAX_PARSER_DEPTH, - DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); - - InterpreterAlterQuery alter_interpreter(alter_ast, context); - alter_interpreter.execute(); - } - - return all_projections; - }; - if (mode == LightweightMutationProjectionMode::THROW) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, @@ -156,43 +127,13 @@ BlockIO InterpreterDeleteQuery::execute() } else if (mode == LightweightMutationProjectionMode::DROP) { - dropOrClearProjections(true); - } - else if (mode == LightweightMutationProjectionMode::REBUILD) - { - std::vector all_projections{dropOrClearProjections(false)}; - BlockIO res = lightweightDelete(); - - for (const auto & projection : all_projections) - { - String alter_query = - "ALTER TABLE " + table->getStorageID().getFullTableName() - + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " MATERIALIZE PROJECTION " + projection; - - ParserAlterQuery parser; - ASTPtr alter_ast = parseQuery( - parser, - alter_query.data(), - alter_query.data() + alter_query.size(), - "ALTER query", - 0, - DBMS_DEFAULT_MAX_PARSER_DEPTH, - DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); - - InterpreterAlterQuery alter_interpreter(alter_ast, context); - alter_interpreter.execute(); - } - - return res; + return lightweightDelete(); } else { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unrecognized lightweight_mutation_projection_mode, only throw and drop are allowed."); } - - return lightweightDelete(); } throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6d3a4f30b34..ace285bcfc9 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -487,7 +487,11 @@ static void validateUpdateColumns( if (column_name == RowExistsColumn::name) { if (!source.supportsLightweightDelete()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + { + // if (!source.getStorage()->isMergeTree() + // || context->getSettingsRef().lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) + // throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + } } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a552ee89aee..8ca987eb1f8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1042,6 +1042,8 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; + + bool lightweight_mutation_mode; }; using MutationContextPtr = std::shared_ptr; @@ -1571,7 +1573,7 @@ private: } else { - if (ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!ctx->lightweight_mutation_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } @@ -2255,7 +2257,8 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->updated_header.has(RowExistsColumn::name)) + ctx->lightweight_mutation_mode = ctx->updated_header.has(RowExistsColumn::name); + if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_mutation_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, /// Reset existing_rows_count of new data part to 0 and it will be updated while writing _row_exists column diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index 307d3cb53fc..e69de29bb2d 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,5 +0,0 @@ -1231 John 33 -8888 Alice 50 -6666 Ksenia 48 -8888 Alice 50 -p users 3 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index fb32646b46a..4e674fa0cfd 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -10,14 +10,12 @@ CREATE TABLE users ( ) ENGINE = MergeTree order by uid; INSERT INTO users VALUES (1231, 'John', 33); -INSERT INTO users VALUES (6666, 'Ksenia', 48); -INSERT INTO users VALUES (8888, 'Alice', 50); DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 8888 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 6666 SETTINGS lightweight_mutation_projection_mode = 'drop'; +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; SYSTEM FLUSH LOGS; @@ -26,33 +24,8 @@ SELECT name, `table` FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users'); +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); SELECT * FROM users ORDER BY uid; -DROP TABLE users; - -CREATE TABLE users ( - uid Int16, - name String, - age Int16, - projection p (select * order by age) -) ENGINE = MergeTree order by uid; - -INSERT INTO users VALUES (1231, 'John', 33), (6666, 'Ksenia', 48), (8888, 'Alice', 50); - -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'rebuild'; - -SELECT * FROM users ORDER BY uid; - -SYSTEM FLUSH LOGS; - --- expecting projection p with 3 rows is active -SELECT - name, - `table`, - rows, -FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND active = 1; - DROP TABLE users; \ No newline at end of file From fa65e374dcb66c2e927f52ea521e9a8586feef65 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 11:57:33 +0200 Subject: [PATCH 0252/1722] fix docs --- docs/en/sql-reference/table-functions/file.md | 4 ++-- docs/en/sql-reference/table-functions/hdfs.md | 4 ++-- docs/en/sql-reference/table-functions/s3.md | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index d21f523ab8e..838a7ab61de 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -206,7 +206,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. -## Hive-style partitioning {#hive-style-patitioning} +## Hive-style partitioning {#hive-style-partitioning} When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. @@ -215,7 +215,7 @@ When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtu Use virtual column, created with Hive-style partitioning ``` sql -SET use_hive_patitioning = 1; +SET use_hive_partitioning = 1; SELECT _specified_column from file('/specified_column=specified_data/file.txt'); ``` diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index beb1ad12532..fc84c431066 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -99,7 +99,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. -## Hive-style partitioning {#hive-style-patitioning} +## Hive-style partitioning {#hive-style-partitioning} When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. @@ -108,7 +108,7 @@ When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtu Use virtual column, created with Hive-style partitioning ``` sql -SET use_hive_patitioning = 1; +SET use_hive_partitioning = 1; SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); ``` diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 45c4caa1a13..15074a77475 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -274,7 +274,7 @@ FROM s3( - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. In case of archive shows uncompressed file size of the file inside the archive. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. -## Hive-style partitioning {#hive-style-patitioning} +## Hive-style partitioning {#hive-style-partitioning} When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. @@ -283,7 +283,7 @@ When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtu Use virtual column, created with Hive-style partitioning ``` sql -SET use_hive_patitioning = 1; +SET use_hive_partitioning = 1; SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); ``` From eb085ea585d10f077d1ce66ee3f663ca016d24e8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 13:06:29 +0000 Subject: [PATCH 0253/1722] fix --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index ace285bcfc9..c2341463041 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -488,7 +488,7 @@ static void validateUpdateColumns( { if (!source.supportsLightweightDelete()) { - // if (!source.getStorage()->isMergeTree() + // if (!source.getStorage()->isMergeTree() // || context->getSettingsRef().lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) // throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); } From df104abcc60366df3946a23c52c2e67a92dcb545 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 17:20:19 +0200 Subject: [PATCH 0254/1722] try to fix tests --- .../03203_hive_style_partitioning.reference | 8 ++++---- .../03203_hive_style_partitioning.sh | 18 +++++++++--------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index e0f46caf1c8..0e6b6052946 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -29,6 +29,10 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth +42 2020-01-01 +[1,2,3] 42.42 +Array(Int64) LowCardinality(Float64) +101 1 TESTING THE URL PARTITIONING first last Elizabeth @@ -60,10 +64,6 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -42 2020-01-01 -[1,2,3] 42.42 -Array(Int64) LowCardinality(Float64) -101 1 TESTING THE S3 PARTITIONING first last Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 9d805b39b8a..e74f24bfd80 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" +$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -36,17 +36,17 @@ SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/parti SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; """ -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" +$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE URL PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -67,17 +67,17 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" +$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE S3 PARTITIONING'" -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -100,7 +100,7 @@ SELECT *, _non_existing_column FROM s3('http://localhost:11111/test/hive_partiti SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; """ -$CLICKHOUSE_LOCAL -n -q """ +$CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; From 9257c4aac299836dc3b1e215c8fd8ba9b190d3b4 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 15:31:51 +0000 Subject: [PATCH 0255/1722] change support lightweight delete condition --- src/Interpreters/InterpreterDeleteQuery.cpp | 5 +++-- src/Interpreters/MutationsInterpreter.cpp | 6 +----- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +--- 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 0f081c522dd..a7d0264f0b0 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -60,6 +60,7 @@ BlockIO InterpreterDeleteQuery::execute() auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); + bool hasProjection = table->hasProjection(); auto lightweightDelete = [&]() { @@ -107,13 +108,13 @@ BlockIO InterpreterDeleteQuery::execute() table->mutate(mutation_commands, getContext()); return {}; } - else if (table->supportsLightweightDelete()) + else if (!hasProjection && table->supportsLightweightDelete()) { return lightweightDelete(); } else { - if (table->hasProjection()) + if (hasProjection) { auto context = Context::createCopy(getContext()); auto mode = context->getSettingsRef().lightweight_mutation_projection_mode; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index c2341463041..6d3a4f30b34 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -487,11 +487,7 @@ static void validateUpdateColumns( if (column_name == RowExistsColumn::name) { if (!source.supportsLightweightDelete()) - { - // if (!source.getStorage()->isMergeTree() - // || context->getSettingsRef().lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) - // throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); - } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c2e0e778220..0ef8bcfc681 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1641,11 +1641,9 @@ void IMergeTreeDataPart::loadColumns(bool require) } -/// Project part / part with project parts / compact part doesn't support LWD. bool IMergeTreeDataPart::supportLightweightDeleteMutate() const { - return (part_type == MergeTreeDataPartType::Wide || part_type == MergeTreeDataPartType::Compact) && - parent_part == nullptr && projection_parts.empty(); + return (part_type == MergeTreeDataPartType::Wide || part_type == MergeTreeDataPartType::Compact); } bool IMergeTreeDataPart::hasLightweightDelete() const From 4f11dbc7f372d46769da4ab3af6db83b7967faa0 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 18:25:33 +0000 Subject: [PATCH 0256/1722] fix with wide part --- src/Storages/MergeTree/MutateTask.cpp | 11 +++--- .../03161_lightweight_delete_projection.sql | 36 ++++++++++++++++++- 2 files changed, 41 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8ca987eb1f8..57784067720 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1043,7 +1043,7 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; - bool lightweight_mutation_mode; + bool lightweight_delete_mode; }; using MutationContextPtr = std::shared_ptr; @@ -1573,7 +1573,7 @@ private: } else { - if (!ctx->lightweight_mutation_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!ctx->lightweight_delete_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } @@ -1843,7 +1843,8 @@ private: hardlinked_files.insert(it->name()); } } - else if (!endsWith(it->name(), ".tmp_proj")) // ignore projection tmp merge dir + /// Ignore projection tmp merge dir, and under lightweight delete mode ignore projection files. + else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) { // it's a projection part directory ctx->new_data_part->getDataPartStorage().createProjection(destination); @@ -2257,8 +2258,8 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - ctx->lightweight_mutation_mode = ctx->updated_header.has(RowExistsColumn::name); - if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_mutation_mode) + ctx->lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_delete_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, /// Reset existing_rows_count of new data part to 0 and it will be updated while writing _row_exists column diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 4e674fa0cfd..bfeb0127fa4 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,13 +1,47 @@ DROP TABLE IF EXISTS users; +-- compact part CREATE TABLE users ( uid Int16, name String, age Int16, projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) -) ENGINE = MergeTree order by uid; +) ENGINE = MergeTree order by uid +SETTINGS min_bytes_for_wide_part = 10485760; + +INSERT INTO users VALUES (1231, 'John', 33); + +DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } + +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } + +DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; + +SYSTEM FLUSH LOGS; + +-- expecting no projection +SELECT + name, + `table` +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +SELECT * FROM users ORDER BY uid; + +DROP TABLE users; + + +-- wide part +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p1 (select count(), age group by age), + projection p2 (select age, name group by age, name) +) ENGINE = MergeTree order by uid +SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); From df9211c345e8bcfc53ed392a351e6320991240d1 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 11 Jul 2024 18:32:38 +0000 Subject: [PATCH 0257/1722] fix --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 57784067720..2adcb49d6a3 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1844,7 +1844,7 @@ private: } } /// Ignore projection tmp merge dir, and under lightweight delete mode ignore projection files. - else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) + else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) { // it's a projection part directory ctx->new_data_part->getDataPartStorage().createProjection(destination); From 10dd4a9fe66914899ec5e5c89e5b9cc24096f64c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 21:16:47 +0200 Subject: [PATCH 0258/1722] debugging tests --- .../0_stateless/03203_hive_style_partitioning.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index e74f24bfd80..14b4a116596 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -36,17 +36,17 @@ SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/parti SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; """ -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE URL PARTITIONING'" +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; @@ -67,14 +67,14 @@ SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/colum SELECT *, _non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" -$CLICKHOUSE_CLIENT -n -q """ +$CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" -$CLICKHOUSE_CLIENT -q "SELECT 'TESTING THE S3 PARTITIONING'" +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ From 119777cd7346a32f45588d069bf6a89efe091867 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 12 Jul 2024 12:03:25 +0200 Subject: [PATCH 0259/1722] update reference --- .../0_stateless/03203_hive_style_partitioning.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 0e6b6052946..d187f4cdd2c 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -29,9 +29,9 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -42 2020-01-01 -[1,2,3] 42.42 -Array(Int64) LowCardinality(Float64) +42 2020-01-01 +[1,2,3] 42.42 +Array(Int64) LowCardinality(Float64) 101 1 TESTING THE URL PARTITIONING From 0988e1deadf34736f126e9eb3a2162d3abbe1314 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 12 Jul 2024 14:59:43 +0200 Subject: [PATCH 0260/1722] update tests --- .../queries/0_stateless/03203_hive_style_partitioning.reference | 2 +- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index d187f4cdd2c..be43048dd01 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -96,4 +96,4 @@ Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth -1 +OK diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 14b4a116596..58a74a3ca8f 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -104,4 +104,4 @@ $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -""" 2>&1 | grep -c "UNKNOWN_IDENTIFIER" +""" 2>&1 | grep -F -q "UNKNOWN_IDENTIFIER" && echo "OK" || echo "FAIL"; From 9c6a49b6d474836ee894ddaaa02ebb982370d25c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 12 Jul 2024 14:30:46 +0000 Subject: [PATCH 0261/1722] fix WriteBufferFromPocoSocketChunked::finalizeImpl() --- src/IO/WriteBufferFromPocoSocketChunked.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocketChunked.cpp b/src/IO/WriteBufferFromPocoSocketChunked.cpp index 98c5126c24b..9da46ee2d10 100644 --- a/src/IO/WriteBufferFromPocoSocketChunked.cpp +++ b/src/IO/WriteBufferFromPocoSocketChunked.cpp @@ -202,7 +202,7 @@ void WriteBufferFromPocoSocketChunked::nextImpl() void WriteBufferFromPocoSocketChunked::finalizeImpl() { - if (offset() == sizeof(*chunk_size_ptr)) + if (chunked && offset() == sizeof(*chunk_size_ptr)) pos -= sizeof(*chunk_size_ptr); WriteBufferFromPocoSocket::finalizeImpl(); } From dfc3db23b1d263ed0fcfce36e2df0b925592d7c3 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 14:38:14 +0000 Subject: [PATCH 0262/1722] Implement new JSON data type --- .../Passes/FunctionToSubcolumnsPass.cpp | 21 +- src/Analyzer/Resolve/IdentifierResolver.cpp | 44 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 12 +- src/Columns/ColumnDynamic.h | 3 +- src/Columns/ColumnObject.cpp | 2008 +++++++++-------- src/Columns/ColumnObject.h | 385 ++-- src/Columns/ColumnObjectDeprecated.cpp | 1096 +++++++++ src/Columns/ColumnObjectDeprecated.h | 265 +++ src/Columns/IColumn.cpp | 6 +- src/Columns/tests/gtest_column_object.cpp | 351 +++ src/Core/Settings.h | 5 +- src/Core/SettingsChangesHistory.cpp | 5 +- src/Core/TypeId.h | 1 + src/DataTypes/DataTypeDynamic.cpp | 51 +- src/DataTypes/DataTypeDynamic.h | 5 +- src/DataTypes/DataTypeFactory.cpp | 3 +- src/DataTypes/DataTypeFactory.h | 3 +- src/DataTypes/DataTypeObject.cpp | 481 +++- src/DataTypes/DataTypeObject.h | 62 +- src/DataTypes/DataTypeObjectDeprecated.cpp | 85 + src/DataTypes/DataTypeObjectDeprecated.h | 48 + src/DataTypes/DataTypeTuple.cpp | 15 +- src/DataTypes/DataTypesBinaryEncoding.cpp | 95 +- src/DataTypes/DataTypesBinaryEncoding.h | 105 +- src/DataTypes/FieldToDataType.cpp | 3 +- src/DataTypes/IDataType.cpp | 3 +- src/DataTypes/IDataType.h | 6 +- src/DataTypes/ObjectUtils.cpp | 58 +- src/DataTypes/ObjectUtils.h | 4 +- .../Serializations/ISerialization.cpp | 18 +- src/DataTypes/Serializations/ISerialization.h | 85 +- .../Serializations/SerializationDynamic.cpp | 45 +- .../Serializations/SerializationDynamic.h | 12 +- .../SerializationDynamicElement.cpp | 1 + .../Serializations/SerializationJSON.cpp | 409 ++++ .../Serializations/SerializationJSON.h | 50 + .../SerializationJSONElement.cpp | 3 + .../Serializations/SerializationJSONElement.h | 8 + .../SerializationLowCardinality.cpp | 9 +- .../Serializations/SerializationObject.cpp | 1045 +++++---- .../Serializations/SerializationObject.h | 129 +- .../SerializationObjectDeprecated.cpp | 586 +++++ .../SerializationObjectDeprecated.h | 121 + .../SerializationObjectDynamicPath.cpp | 180 ++ .../SerializationObjectDynamicPath.h | 58 + .../SerializationObjectElement.cpp | 3 + .../SerializationObjectElement.h | 8 + .../SerializationObjectTypedPath.cpp | 78 + .../SerializationObjectTypedPath.h | 57 + .../Serializations/SerializationSubObject.cpp | 259 +++ .../Serializations/SerializationSubObject.h | 72 + .../Serializations/SerializationVariant.cpp | 22 +- .../Serializations/SerializationVariant.h | 4 +- .../gtest_deprecated_object_serialization.cpp | 80 + .../tests/gtest_object_serialization.cpp | 160 +- src/DataTypes/Utils.cpp | 1 + .../gtest_data_types_binary_encoding.cpp | 5 +- src/Databases/DatabaseReplicated.cpp | 1 + src/Formats/EscapingRuleUtils.cpp | 2 +- src/Formats/FormatFactory.cpp | 4 +- src/Formats/FormatSettings.h | 4 +- src/Formats/JSONExtractTree.cpp | 303 ++- src/Formats/JSONUtils.cpp | 12 +- src/Formats/SchemaInferenceUtils.cpp | 46 +- src/Functions/FunctionsConversion.cpp | 40 +- src/Functions/JSONEmpty.cpp | 126 ++ src/Functions/JSONPaths.cpp | 339 +++ src/IO/ReadHelpers.cpp | 87 + src/IO/ReadHelpers.h | 7 + src/IO/WriteHelpers.h | 6 + src/Interpreters/TreeRewriter.cpp | 10 +- src/Interpreters/convertFieldToType.cpp | 2 +- .../parseColumnsListForTableFunction.cpp | 16 +- .../parseColumnsListForTableFunction.h | 2 + src/Parsers/ASTObjectTypeArgument.cpp | 75 + src/Parsers/ASTObjectTypeArgument.h | 35 + src/Parsers/CommonParsers.h | 28 + src/Parsers/ExpressionElementParsers.cpp | 37 +- src/Parsers/ExpressionElementParsers.h | 12 +- src/Parsers/Lexer.cpp | 2 + src/Parsers/Lexer.h | 1 + src/Parsers/ParserCreateQuery.cpp | 2 +- src/Parsers/ParserDataType.cpp | 103 +- .../Impl/JSONAsStringRowInputFormat.cpp | 15 +- .../Formats/Impl/JSONAsStringRowInputFormat.h | 18 +- src/Processors/QueryPlan/PartsSplitter.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 17 +- .../MergeTreeDataPartWriterCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 14 +- .../MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 2 +- 91 files changed, 8000 insertions(+), 2111 deletions(-) create mode 100644 src/Columns/ColumnObjectDeprecated.cpp create mode 100644 src/Columns/ColumnObjectDeprecated.h create mode 100644 src/Columns/tests/gtest_column_object.cpp create mode 100644 src/DataTypes/DataTypeObjectDeprecated.cpp create mode 100644 src/DataTypes/DataTypeObjectDeprecated.h create mode 100644 src/DataTypes/Serializations/SerializationJSON.cpp create mode 100644 src/DataTypes/Serializations/SerializationJSON.h create mode 100644 src/DataTypes/Serializations/SerializationJSONElement.cpp create mode 100644 src/DataTypes/Serializations/SerializationJSONElement.h create mode 100644 src/DataTypes/Serializations/SerializationObjectDeprecated.cpp create mode 100644 src/DataTypes/Serializations/SerializationObjectDeprecated.h create mode 100644 src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp create mode 100644 src/DataTypes/Serializations/SerializationObjectDynamicPath.h create mode 100644 src/DataTypes/Serializations/SerializationObjectElement.cpp create mode 100644 src/DataTypes/Serializations/SerializationObjectElement.h create mode 100644 src/DataTypes/Serializations/SerializationObjectTypedPath.cpp create mode 100644 src/DataTypes/Serializations/SerializationObjectTypedPath.h create mode 100644 src/DataTypes/Serializations/SerializationSubObject.cpp create mode 100644 src/DataTypes/Serializations/SerializationSubObject.h create mode 100644 src/DataTypes/Serializations/tests/gtest_deprecated_object_serialization.cpp create mode 100644 src/Functions/JSONEmpty.cpp create mode 100644 src/Functions/JSONPaths.cpp create mode 100644 src/Parsers/ASTObjectTypeArgument.cpp create mode 100644 src/Parsers/ASTObjectTypeArgument.h diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 90051779a26..395ebbfa408 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -52,6 +52,8 @@ void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, Col /// Replace `notEmpty(argument)` with `notEquals(argument.size0, 0)` if not positive /// `argument` may be Array or Map. + std::cerr << "optimizeFunctionEmpty " << ctx.column.name << "\n"; + NameAndTypePair column{ctx.column.name + ".size0", std::make_shared()}; auto & function_arguments_nodes = function_node.getArguments().getNodes(); @@ -234,9 +236,18 @@ std::tuple getTypedNodesForOptimizati return {}; auto column_in_table = storage_snapshot->tryGetColumn(GetColumnsOptions::All, column.name); + std::cerr << "getTypedNodesForOptimization " << column.name << "\n"; if (!column_in_table || !column_in_table->type->equals(*column.type)) + { + std::cerr << "getTypedNodesForOptimization FAIL\n"; + if (column_in_table) + std::cerr << column_in_table->type->getName() << "/" << column.type->getName() << "\n"; + else + std::cerr << "null\n"; return {}; + } + std::cerr << "getTypedNodesForOptimization OK\n"; return std::make_tuple(function_node, first_argument_column_node, table_node); } @@ -422,9 +433,15 @@ public: auto table_name = table_node->getStorage()->getStorageID().getFullTableName(); Identifier qualified_name({table_name, column.name}); - if (!identifiers_to_optimize.contains(qualified_name)) - return; + std::cerr << "FunctionToSubcolumnsVisitorSecondPass " << column.name << "\n"; + if (!identifiers_to_optimize.contains(qualified_name)) + { + std::cerr << "FunctionToSubcolumnsVisitorSecondPass FAIL\n"; + return; + } + + std::cerr << "FunctionToSubcolumnsVisitorSecondPass OK\n"; auto transformer_it = node_transformers.find({column.type->getTypeId(), function_node->getFunctionName()}); if (transformer_it != node_transformers.end()) { diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 692a31b66ba..a3c4d0aad65 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -1,6 +1,7 @@ #include -#include +#include #include +#include #include #include @@ -449,10 +450,10 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromCompoundExpression( if (auto * column = compound_expression->as()) { const DataTypePtr & column_type = column->getColumn().getTypeInStorage(); - if (column_type->getTypeId() == TypeIndex::Object) + if (column_type->getTypeId() == TypeIndex::ObjectDeprecated) { - const auto * object_type = checkAndGetDataType(column_type.get()); - if (object_type->getSchemaFormat() == "json" && object_type->hasNullableSubcolumns()) + const auto & object_type = checkAndGetDataType(*column_type); + if (object_type.getSchemaFormat() == "json" && object_type.hasNullableSubcolumns()) { QueryTreeNodePtr constant_node_null = std::make_shared(Field()); return constant_node_null; @@ -678,9 +679,33 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( bool match_full_identifier = false; const auto & identifier_full_name = identifier_without_column_qualifier.getFullName(); - auto it = table_expression_data.column_name_to_column_node.find(identifier_full_name); - bool can_resolve_directly_from_storage = it != table_expression_data.column_name_to_column_node.end(); - if (can_resolve_directly_from_storage && table_expression_data.subcolumn_names.contains(identifier_full_name)) + + ColumnNodePtr result_column_node; + bool can_resolve_directly_from_storage = false; + bool is_subcolumn = false; + if (auto it = table_expression_data.column_name_to_column_node.find(identifier_full_name); it != table_expression_data.column_name_to_column_node.end()) + { + can_resolve_directly_from_storage = true; + is_subcolumn = table_expression_data.subcolumn_names.contains(identifier_full_name); + result_column_node = it->second; + } + /// Check if it's a dynamic subcolumn + else + { + auto [column_name, dynamic_subcolumn_name] = Nested::splitName(identifier_full_name); + auto jt = table_expression_data.column_name_to_column_node.find(column_name); + if (jt != table_expression_data.column_name_to_column_node.end() && jt->second->getColumnType()->hasDynamicSubcolumns()) + { + if (auto dynamic_subcolumn_type = jt->second->getColumnType()->tryGetSubcolumnType(dynamic_subcolumn_name)) + { + result_column_node = std::make_shared(NameAndTypePair{identifier_full_name, dynamic_subcolumn_type}, jt->second->getColumnSource()); + can_resolve_directly_from_storage = true; + is_subcolumn = true; + } + } + } + + if (can_resolve_directly_from_storage && is_subcolumn) { /** In the case when we have an ARRAY JOIN, we should not resolve subcolumns directly from storage. * For example, consider the following SQL query: @@ -696,11 +721,11 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( if (can_resolve_directly_from_storage) { match_full_identifier = true; - result_expression = it->second; + result_expression = result_column_node; } else { - it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0)); + auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0)); if (it != table_expression_data.column_name_to_column_node.end()) result_expression = it->second; } @@ -973,7 +998,6 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromJoin(const Identifi if (!join_node_in_resolve_process && from_join_node.isUsingJoinExpression()) { auto & join_using_list = from_join_node.getJoinExpression()->as(); - for (auto & join_using_node : join_using_list.getNodes()) { auto & column_node = join_using_node->as(); diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 576c4943ccb..b8d1e44fca7 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -1,14 +1,14 @@ #include -#include -#include -#include -#include -#include #include -#include #include +#include +#include +#include #include +#include +#include +#include #include #include diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 27ad0dd583f..eaf4f85c9a7 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -22,7 +22,6 @@ namespace DB class ColumnDynamic final : public COWHelper, ColumnDynamic> { public: - /// struct Statistics { enum class Source @@ -77,7 +76,7 @@ public: return create(variant_column_->assumeMutable(), variant_type, max_dynamic_types_, statistics_); } - static MutablePtr create(size_t max_dynamic_types_) + static MutablePtr create(size_t max_dynamic_types_ = ColumnVariant::MAX_NESTED_COLUMNS) { return Base::create(max_dynamic_types_); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 90ef974010c..3d1ab63d219 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1,1096 +1,1186 @@ -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - +#include +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int DUPLICATE_COLUMN; - extern const int EXPERIMENTAL_FEATURE_ERROR; - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } namespace { -/// Recreates column with default scalar values and keeps sizes of arrays. -ColumnPtr recreateColumnWithDefaultValues( - const ColumnPtr & column, const DataTypePtr & scalar_type, size_t num_dimensions) +static const FormatSettings & getFormatSettings() { - const auto * column_array = checkAndGetColumn(column.get()); - if (column_array && num_dimensions) + static const FormatSettings settings; + return settings; +} + +static const std::shared_ptr & getDynamicSerialization() +{ + static const std::shared_ptr dynamic_serialization = std::make_shared(); + return dynamic_serialization; +} + +} + +ColumnObject::ColumnObject( + std::unordered_map typed_paths_, + std::unordered_map dynamic_paths_, + MutableColumnPtr shared_data_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_, + const Statistics & statistics_) + : shared_data(std::move(shared_data_)) + , max_dynamic_paths(max_dynamic_paths_) + , max_dynamic_types(max_dynamic_types_) + , statistics(statistics_) +{ + typed_paths.reserve(typed_paths_.size()); + for (auto & [path, column] : typed_paths_) + typed_paths[path] = std::move(column); + + dynamic_paths.reserve(dynamic_paths_.size()); + for (auto & [path, column] : dynamic_paths_) + dynamic_paths[path] = std::move(column); +} + +ColumnObject::ColumnObject( + std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_) + : max_dynamic_paths(max_dynamic_paths_), max_dynamic_types(max_dynamic_types_) +{ + typed_paths.reserve(typed_paths_.size()); + for (auto & [path, column] : typed_paths_) { - return ColumnArray::create( - recreateColumnWithDefaultValues( - column_array->getDataPtr(), scalar_type, num_dimensions - 1), - IColumn::mutate(column_array->getOffsetsPtr())); + if (!column->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected non-empty typed path column in ColumnObject constructor"); + typed_paths[path] = std::move(column); } - return createArrayOfType(scalar_type, num_dimensions)->createColumn()->cloneResized(column->size()); + MutableColumns paths_and_values; + paths_and_values.emplace_back(ColumnString::create()); + paths_and_values.emplace_back(ColumnString::create()); + shared_data = ColumnArray::create(ColumnTuple::create(std::move(paths_and_values))); } -/// Replaces NULL fields to given field or empty array. -class FieldVisitorReplaceNull : public StaticVisitor +ColumnObject::Ptr ColumnObject::create( + const std::unordered_map & typed_paths_, + const std::unordered_map & dynamic_paths_, + const ColumnPtr & shared_data_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_, + const ColumnObject::Statistics & statistics_) { -public: - explicit FieldVisitorReplaceNull( - const Field & replacement_, size_t num_dimensions_) - : replacement(replacement_) - , num_dimensions(num_dimensions_) - { - } + std::unordered_map mutable_typed_paths; + mutable_typed_paths.reserve(typed_paths_.size()); + for (const auto & [path, column] : typed_paths_) + mutable_typed_paths[path] = typed_paths_.at(path)->assumeMutable(); - Field operator()(const Null &) const - { - return num_dimensions ? Array() : replacement; - } - - Field operator()(const Array & x) const - { - assert(num_dimensions > 0); - const size_t size = x.size(); - Array res(size); - for (size_t i = 0; i < size; ++i) - res[i] = applyVisitor(FieldVisitorReplaceNull(replacement, num_dimensions - 1), x[i]); - return res; - } - - template - Field operator()(const T & x) const { return x; } - -private: - const Field & replacement; - size_t num_dimensions; -}; - -/// Visitor that allows to get type of scalar field -/// or least common type of scalars in array. -/// More optimized version of FieldToDataType. -class FieldVisitorToScalarType : public StaticVisitor<> -{ -public: - using FieldType = Field::Types::Which; - - void operator()(const Array & x) - { - size_t size = x.size(); - for (size_t i = 0; i < size; ++i) - applyVisitor(*this, x[i]); - } - - void operator()(const UInt64 & x) - { - field_types.insert(FieldType::UInt64); - if (x <= std::numeric_limits::max()) - type_indexes.insert(TypeIndex::UInt8); - else if (x <= std::numeric_limits::max()) - type_indexes.insert(TypeIndex::UInt16); - else if (x <= std::numeric_limits::max()) - type_indexes.insert(TypeIndex::UInt32); - else - type_indexes.insert(TypeIndex::UInt64); - } - - void operator()(const Int64 & x) - { - field_types.insert(FieldType::Int64); - if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) - type_indexes.insert(TypeIndex::Int8); - else if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) - type_indexes.insert(TypeIndex::Int16); - else if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) - type_indexes.insert(TypeIndex::Int32); - else - type_indexes.insert(TypeIndex::Int64); - } - - void operator()(const bool &) - { - field_types.insert(FieldType::UInt64); - type_indexes.insert(TypeIndex::UInt8); - } - - void operator()(const Null &) - { - have_nulls = true; - } - - template - void operator()(const T &) - { - field_types.insert(Field::TypeToEnum>::value); - type_indexes.insert(TypeToTypeIndex>); - } - - DataTypePtr getScalarType() const { return getLeastSupertypeOrString(type_indexes); } - bool haveNulls() const { return have_nulls; } - bool needConvertField() const { return field_types.size() > 1; } - -private: - TypeIndexSet type_indexes; - std::unordered_set field_types; - bool have_nulls = false; -}; + std::unordered_map mutable_dynamic_paths; + mutable_dynamic_paths.reserve(dynamic_paths_.size()); + for (const auto & [path, column] : dynamic_paths_) + mutable_dynamic_paths[path] = dynamic_paths_.at(path)->assumeMutable(); + return ColumnObject::create(std::move(mutable_typed_paths), std::move(mutable_dynamic_paths), shared_data_->assumeMutable(), max_dynamic_paths_, max_dynamic_types_, statistics_); } -FieldInfo getFieldInfo(const Field & field) +ColumnObject::MutablePtr ColumnObject::create( + std::unordered_map typed_paths_, + std::unordered_map dynamic_paths_, + MutableColumnPtr shared_data_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_, + const ColumnObject::Statistics & statistics_) { - FieldVisitorToScalarType to_scalar_type_visitor; - applyVisitor(to_scalar_type_visitor, field); - FieldVisitorToNumberOfDimensions to_number_dimension_visitor; - - return - { - to_scalar_type_visitor.getScalarType(), - to_scalar_type_visitor.haveNulls(), - to_scalar_type_visitor.needConvertField(), - applyVisitor(to_number_dimension_visitor, field), - to_number_dimension_visitor.need_fold_dimension - }; + return Base::create(std::move(typed_paths_), std::move(dynamic_paths_), std::move(shared_data_), max_dynamic_paths_, max_dynamic_types_, statistics_); } -ColumnObject::Subcolumn::Subcolumn(MutableColumnPtr && data_, bool is_nullable_) - : least_common_type(getDataTypeByColumn(*data_)) - , is_nullable(is_nullable_) - , num_rows(data_->size()) +ColumnObject::MutablePtr ColumnObject::create(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_) { - data.push_back(std::move(data_)); + return Base::create(std::move(typed_paths_), max_dynamic_paths_, max_dynamic_types_); } -ColumnObject::Subcolumn::Subcolumn( - size_t size_, bool is_nullable_) - : least_common_type(std::make_shared()) - , is_nullable(is_nullable_) - , num_of_defaults_in_prefix(size_) - , num_rows(size_) +std::string ColumnObject::getName() const { + WriteBufferFromOwnString ss; + ss << "Object("; + ss << "max_dynamic_paths=" << max_dynamic_paths; + ss << ", max_dynamic_types=" << max_dynamic_types; + std::vector sorted_typed_paths; + sorted_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + sorted_typed_paths.push_back(path); + std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); + for (const auto & path : sorted_typed_paths) + ss << ", " << path << " " << typed_paths.at(path)->getName(); + ss << ")"; + return ss.str(); } -size_t ColumnObject::Subcolumn::size() const +MutableColumnPtr ColumnObject::cloneEmpty() const { - return num_rows; + std::unordered_map empty_typed_paths; + empty_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + empty_typed_paths[path] = column->cloneEmpty(); + + std::unordered_map empty_dynamic_paths; + empty_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + empty_dynamic_paths[path] = column->cloneEmpty(); + + return ColumnObject::create(std::move(empty_typed_paths), std::move(empty_dynamic_paths), shared_data->cloneEmpty(), max_dynamic_paths, max_dynamic_types, statistics); } -size_t ColumnObject::Subcolumn::byteSize() const +MutableColumnPtr ColumnObject::cloneResized(size_t size) const { - size_t res = 0; - for (const auto & part : data) - res += part->byteSize(); - return res; -} + std::unordered_map resized_typed_paths; + resized_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + resized_typed_paths[path] = column->cloneResized(size); -size_t ColumnObject::Subcolumn::allocatedBytes() const -{ - size_t res = 0; - for (const auto & part : data) - res += part->allocatedBytes(); - return res; -} + std::unordered_map resized_dynamic_paths; + resized_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + resized_dynamic_paths[path] = column->cloneResized(size); -void ColumnObject::Subcolumn::get(size_t n, Field & res) const -{ - if (isFinalized()) - { - getFinalizedColumn().get(n, res); - return; - } - - size_t ind = n; - if (ind < num_of_defaults_in_prefix) - { - res = least_common_type.get()->getDefault(); - return; - } - - ind -= num_of_defaults_in_prefix; - for (const auto & part : data) - { - if (ind < part->size()) - { - part->get(ind, res); - res = convertFieldToTypeOrThrow(res, *least_common_type.get()); - return; - } - - ind -= part->size(); - } - - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for getting field is out of range", n); -} - -void ColumnObject::Subcolumn::checkTypes() const -{ - DataTypes prefix_types; - prefix_types.reserve(data.size()); - for (size_t i = 0; i < data.size(); ++i) - { - auto current_type = getDataTypeByColumn(*data[i]); - prefix_types.push_back(current_type); - auto prefix_common_type = getLeastSupertype(prefix_types); - if (!prefix_common_type->equals(*current_type)) - throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, - "Data type {} of column at position {} cannot represent all columns from i-th prefix", - current_type->getName(), i); - } -} - -void ColumnObject::Subcolumn::insert(Field field) -{ - auto info = DB::getFieldInfo(field); - insert(std::move(field), std::move(info)); -} - -void ColumnObject::Subcolumn::addNewColumnPart(DataTypePtr type) -{ - auto serialization = type->getSerialization(ISerialization::Kind::SPARSE); - data.push_back(type->createColumn(*serialization)); - least_common_type = LeastCommonType{std::move(type)}; -} - -static bool isConversionRequiredBetweenIntegers(const IDataType & lhs, const IDataType & rhs) -{ - /// If both of types are signed/unsigned integers and size of left field type - /// is less than right type, we don't need to convert field, - /// because all integer fields are stored in Int64/UInt64. - - WhichDataType which_lhs(lhs); - WhichDataType which_rhs(rhs); - - bool is_native_int = which_lhs.isNativeInt() && which_rhs.isNativeInt(); - bool is_native_uint = which_lhs.isNativeUInt() && which_rhs.isNativeUInt(); - - return (!is_native_int && !is_native_uint) - || lhs.getSizeOfValueInMemory() > rhs.getSizeOfValueInMemory(); -} - -void ColumnObject::Subcolumn::insert(Field field, FieldInfo info) -{ - auto base_type = std::move(info.scalar_type); - - if (isNothing(base_type) && info.num_dimensions == 0) - { - insertDefault(); - return; - } - - auto column_dim = least_common_type.getNumberOfDimensions(); - auto value_dim = info.num_dimensions; - - if (isNothing(least_common_type.get())) - column_dim = value_dim; - - if (isNothing(base_type)) - value_dim = column_dim; - - if (value_dim != column_dim) - throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, - "Dimension of types mismatched between inserted value and column. " - "Dimension of value: {}. Dimension of column: {}", - value_dim, column_dim); - - if (is_nullable) - base_type = makeNullable(base_type); - - if (!is_nullable && info.have_nulls) - field = applyVisitor(FieldVisitorReplaceNull(base_type->getDefault(), value_dim), std::move(field)); - - bool type_changed = false; - const auto & least_common_base_type = least_common_type.getBase(); - - if (data.empty()) - { - addNewColumnPart(createArrayOfType(std::move(base_type), value_dim)); - } - else if (!least_common_base_type->equals(*base_type) && !isNothing(base_type)) - { - if (isConversionRequiredBetweenIntegers(*base_type, *least_common_base_type)) - { - base_type = getLeastSupertypeOrString(DataTypes{std::move(base_type), least_common_base_type}); - type_changed = true; - if (!least_common_base_type->equals(*base_type)) - addNewColumnPart(createArrayOfType(std::move(base_type), value_dim)); - } - } - - if (type_changed || info.need_convert) - field = convertFieldToTypeOrThrow(field, *least_common_type.get()); - - if (!data.back()->tryInsert(field)) - { - /** Normalization of the field above is pretty complicated (it uses several FieldVisitors), - * so in the case of a bug, we may get mismatched types. - * The `IColumn::insert` method does not check the type of the inserted field, and it can lead to a segmentation fault. - * Therefore, we use the safer `tryInsert` method to get an exception instead of a segmentation fault. - */ - throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, - "Cannot insert field {} to column {}", - field.dump(), data.back()->dumpStructure()); - } - - ++num_rows; -} - -void ColumnObject::Subcolumn::insertRangeFrom(const Subcolumn & src, size_t start, size_t length) -{ - assert(start + length <= src.size()); - size_t end = start + length; - num_rows += length; - - if (data.empty()) - { - addNewColumnPart(src.getLeastCommonType()); - } - else if (!least_common_type.get()->equals(*src.getLeastCommonType())) - { - auto new_least_common_type = getLeastSupertypeOrString(DataTypes{least_common_type.get(), src.getLeastCommonType()}); - if (!new_least_common_type->equals(*least_common_type.get())) - addNewColumnPart(std::move(new_least_common_type)); - } - - if (end <= src.num_of_defaults_in_prefix) - { - data.back()->insertManyDefaults(length); - return; - } - - if (start < src.num_of_defaults_in_prefix) - data.back()->insertManyDefaults(src.num_of_defaults_in_prefix - start); - - auto insert_from_part = [&](const auto & column, size_t from, size_t n) - { - assert(from + n <= column->size()); - auto column_type = getDataTypeByColumn(*column); - - if (column_type->equals(*least_common_type.get())) - { - data.back()->insertRangeFrom(*column, from, n); - return; - } - - /// If we need to insert large range, there is no sense to cut part of column and cast it. - /// Casting of all column and inserting from it can be faster. - /// Threshold is just a guess. - - if (n * 3 >= column->size()) - { - auto casted_column = castColumn({column, column_type, ""}, least_common_type.get()); - data.back()->insertRangeFrom(*casted_column, from, n); - return; - } - - auto casted_column = column->cut(from, n); - casted_column = castColumn({casted_column, column_type, ""}, least_common_type.get()); - data.back()->insertRangeFrom(*casted_column, 0, n); - }; - - size_t pos = 0; - size_t processed_rows = src.num_of_defaults_in_prefix; - - /// Find the first part of the column that intersects the range. - while (pos < src.data.size() && processed_rows + src.data[pos]->size() < start) - { - processed_rows += src.data[pos]->size(); - ++pos; - } - - /// Insert from the first part of column. - if (pos < src.data.size() && processed_rows < start) - { - size_t part_start = start - processed_rows; - size_t part_length = std::min(src.data[pos]->size() - part_start, end - start); - insert_from_part(src.data[pos], part_start, part_length); - processed_rows += src.data[pos]->size(); - ++pos; - } - - /// Insert from the parts of column in the middle of range. - while (pos < src.data.size() && processed_rows + src.data[pos]->size() < end) - { - insert_from_part(src.data[pos], 0, src.data[pos]->size()); - processed_rows += src.data[pos]->size(); - ++pos; - } - - /// Insert from the last part of column if needed. - if (pos < src.data.size() && processed_rows < end) - { - size_t part_end = end - processed_rows; - insert_from_part(src.data[pos], 0, part_end); - } -} - -bool ColumnObject::Subcolumn::isFinalized() const -{ - return num_of_defaults_in_prefix == 0 && - (data.empty() || (data.size() == 1 && !data[0]->isSparse())); -} - -void ColumnObject::Subcolumn::finalize() -{ - if (isFinalized()) - return; - - if (data.size() == 1 && num_of_defaults_in_prefix == 0) - { - data[0] = data[0]->convertToFullColumnIfSparse(); - return; - } - - const auto & to_type = least_common_type.get(); - auto result_column = to_type->createColumn(); - - if (num_of_defaults_in_prefix) - result_column->insertManyDefaults(num_of_defaults_in_prefix); - - for (auto & part : data) - { - part = part->convertToFullColumnIfSparse(); - auto from_type = getDataTypeByColumn(*part); - size_t part_size = part->size(); - - if (!from_type->equals(*to_type)) - { - auto offsets = ColumnUInt64::create(); - auto & offsets_data = offsets->getData(); - - /// We need to convert only non-default values and then recreate column - /// with default value of new type, because default values (which represents misses in data) - /// may be inconsistent between types (e.g "0" in UInt64 and empty string in String). - - part->getIndicesOfNonDefaultRows(offsets_data, 0, part_size); - - if (offsets->size() == part_size) - { - part = castColumn({part, from_type, ""}, to_type); - } - else - { - auto values = part->index(*offsets, offsets->size()); - values = castColumn({values, from_type, ""}, to_type); - part = values->createWithOffsets(offsets_data, *createColumnConstWithDefaultValue(result_column->getPtr()), part_size, /*shift=*/ 0); - } - } - - result_column->insertRangeFrom(*part, 0, part_size); - } - - data = { std::move(result_column) }; - num_of_defaults_in_prefix = 0; -} - -void ColumnObject::Subcolumn::insertDefault() -{ - if (data.empty()) - ++num_of_defaults_in_prefix; - else - data.back()->insertDefault(); - - ++num_rows; -} - -void ColumnObject::Subcolumn::insertManyDefaults(size_t length) -{ - if (data.empty()) - num_of_defaults_in_prefix += length; - else - data.back()->insertManyDefaults(length); - - num_rows += length; -} - -void ColumnObject::Subcolumn::popBack(size_t n) -{ - assert(n <= size()); - - num_rows -= n; - size_t num_removed = 0; - for (auto it = data.rbegin(); it != data.rend(); ++it) - { - if (n == 0) - break; - - auto & column = *it; - if (n < column->size()) - { - column->popBack(n); - n = 0; - } - else - { - ++num_removed; - n -= column->size(); - } - } - - data.resize(data.size() - num_removed); - num_of_defaults_in_prefix -= n; -} - -ColumnObject::Subcolumn ColumnObject::Subcolumn::cut(size_t start, size_t length) const -{ - Subcolumn new_subcolumn(0, is_nullable); - new_subcolumn.insertRangeFrom(*this, start, length); - return new_subcolumn; -} - -Field ColumnObject::Subcolumn::getLastField() const -{ - if (data.empty()) - return Field(); - - const auto & last_part = data.back(); - assert(!last_part->empty()); - return (*last_part)[last_part->size() - 1]; -} - -FieldInfo ColumnObject::Subcolumn::getFieldInfo() const -{ - const auto & base_type = least_common_type.getBase(); - return FieldInfo - { - .scalar_type = base_type, - .have_nulls = base_type->isNullable(), - .need_convert = false, - .num_dimensions = least_common_type.getNumberOfDimensions(), - .need_fold_dimension = false, - }; -} - -ColumnObject::Subcolumn ColumnObject::Subcolumn::recreateWithDefaultValues(const FieldInfo & field_info) const -{ - auto scalar_type = field_info.scalar_type; - if (is_nullable) - scalar_type = makeNullable(scalar_type); - - Subcolumn new_subcolumn(*this); - new_subcolumn.least_common_type = LeastCommonType{createArrayOfType(scalar_type, field_info.num_dimensions)}; - - for (auto & part : new_subcolumn.data) - part = recreateColumnWithDefaultValues(part, scalar_type, field_info.num_dimensions); - - return new_subcolumn; -} - -IColumn & ColumnObject::Subcolumn::getFinalizedColumn() -{ - assert(isFinalized()); - return *data[0]; -} - -const IColumn & ColumnObject::Subcolumn::getFinalizedColumn() const -{ - assert(isFinalized()); - return *data[0]; -} - -const ColumnPtr & ColumnObject::Subcolumn::getFinalizedColumnPtr() const -{ - assert(isFinalized()); - return data[0]; -} - -ColumnObject::Subcolumn::LeastCommonType::LeastCommonType() - : type(std::make_shared()) - , base_type(type) - , num_dimensions(0) -{ -} - -ColumnObject::Subcolumn::LeastCommonType::LeastCommonType(DataTypePtr type_) - : type(std::move(type_)) - , base_type(getBaseTypeOfArray(type)) - , num_dimensions(DB::getNumberOfDimensions(*type)) -{ -} - -ColumnObject::ColumnObject(bool is_nullable_) - : is_nullable(is_nullable_) - , num_rows(0) -{ -} - -ColumnObject::ColumnObject(Subcolumns && subcolumns_, bool is_nullable_) - : is_nullable(is_nullable_) - , subcolumns(std::move(subcolumns_)) - , num_rows(subcolumns.empty() ? 0 : (*subcolumns.begin())->data.size()) - -{ - checkConsistency(); -} - -void ColumnObject::checkConsistency() const -{ - if (subcolumns.empty()) - return; - - for (const auto & leaf : subcolumns) - { - if (num_rows != leaf->data.size()) - { - throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject." - " Subcolumn '{}' has {} rows, but expected size is {}", - leaf->path.getPath(), leaf->data.size(), num_rows); - } - } -} - -size_t ColumnObject::size() const -{ -#ifndef NDEBUG - checkConsistency(); -#endif - return num_rows; -} - -size_t ColumnObject::byteSize() const -{ - size_t res = 0; - for (const auto & entry : subcolumns) - res += entry->data.byteSize(); - return res; -} - -size_t ColumnObject::allocatedBytes() const -{ - size_t res = 0; - for (const auto & entry : subcolumns) - res += entry->data.allocatedBytes(); - return res; -} - -void ColumnObject::forEachSubcolumn(MutableColumnCallback callback) -{ - for (auto & entry : subcolumns) - for (auto & part : entry->data.data) - callback(part); -} - -void ColumnObject::forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) -{ - for (auto & entry : subcolumns) - { - for (auto & part : entry->data.data) - { - callback(*part); - part->forEachSubcolumnRecursively(callback); - } - } -} - -void ColumnObject::insert(const Field & field) -{ - const auto & object = field.get(); - - HashSet inserted_paths; - size_t old_size = size(); - for (const auto & [key_str, value] : object) - { - PathInData key(key_str); - inserted_paths.insert(key_str); - if (!hasSubcolumn(key)) - addSubcolumn(key, old_size); - - auto & subcolumn = getSubcolumn(key); - subcolumn.insert(value); - } - - for (auto & entry : subcolumns) - { - if (!inserted_paths.has(entry->path.getPath())) - { - bool inserted = tryInsertDefaultFromNested(entry); - if (!inserted) - entry->data.insertDefault(); - } - } - - ++num_rows; -} - -bool ColumnObject::tryInsert(const Field & field) -{ - if (field.getType() != Field::Types::Which::Object) - return false; - - insert(field); - return true; -} - -void ColumnObject::insertDefault() -{ - for (auto & entry : subcolumns) - entry->data.insertDefault(); - - ++num_rows; + return ColumnObject::create(std::move(resized_typed_paths), std::move(resized_dynamic_paths), shared_data->cloneResized(size), max_dynamic_paths, max_dynamic_types, statistics); } Field ColumnObject::operator[](size_t n) const { - Field object; - get(n, object); + Object object; + + for (const auto & [path, column] : typed_paths) + object[path] = (*column)[n]; + for (const auto & [path, column] : dynamic_paths) + { + /// Output only non-null values from dynamic paths. We cannot distinguish cases when + /// dynamic path has Null value and when it's absent in the row and consider them equivalent. + if (!column->isNullAt(n)) + object[path] = (*column)[n]; + } + + const auto & shared_data_offsets = getSharedDataOffsets(); + const auto [shared_paths, shared_values] = getSharedDataPathsAndValues(); + size_t start = shared_data_offsets[ssize_t(n) - 1]; + size_t end = shared_data_offsets[n]; + for (size_t i = start; i != end; ++i) + { + String path = shared_paths->getDataAt(i).toString(); + auto value_data = shared_values->getDataAt(i); + ReadBufferFromMemory buf(value_data.data, value_data.size); + Field value; + getDynamicSerialization()->deserializeBinary(value, buf, getFormatSettings()); + object[path] = value; + } + return object; } void ColumnObject::get(size_t n, Field & res) const { - assert(n < size()); - res = Object(); - auto & object = res.get(); + res = (*this)[n]; +} - for (const auto & entry : subcolumns) +bool ColumnObject::isDefaultAt(size_t n) const +{ + for (const auto & [path, column] : typed_paths) { - auto it = object.try_emplace(entry->path.getPath()).first; - entry->data.get(n, it->second); + if (!column->isDefaultAt(n)) + return false; } + + for (const auto & [path, column] : dynamic_paths) + { + if (!column->isDefaultAt(n)) + return false; + } + + if (!shared_data->isDefaultAt(n)) + return false; + + return true; +} + +StringRef ColumnObject::getDataAt(size_t) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDataAt is not supported for {}", getName()); +} + +void ColumnObject::insertData(const char *, size_t) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertData is not supported for {}", getName()); +} + +IColumn * ColumnObject::tryToAddNewDynamicPath(const String & path) +{ + if (dynamic_paths.size() == max_dynamic_paths) + return nullptr; + + auto new_dynamic_column = ColumnDynamic::create(max_dynamic_types); + new_dynamic_column->insertManyDefaults(size()); + auto it = dynamic_paths.emplace(path, std::move(new_dynamic_column)).first; + return it->second.get(); +} + +void ColumnObject::setDynamicPaths(const std::vector & paths) +{ + if (paths.size() > max_dynamic_paths) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set dynamic paths to Object column, the number of paths ({}) exceeds the limit ({})", paths.size(), max_dynamic_paths); + + size_t size = this->size(); + for (const auto & path : paths) + { + auto new_dynamic_column = ColumnDynamic::create(max_dynamic_types); + if (size) + new_dynamic_column->insertManyDefaults(size); + dynamic_paths[path] = std::move(new_dynamic_column); + } +} + +void ColumnObject::insert(const Field & x) +{ + const auto & object = x.get(); + auto & shared_data_offsets = getSharedDataOffsets(); + auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + size_t current_size = size(); + for (const auto & [path, value_field] : object) + { + if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) + { + typed_it->second->insert(value_field); + } + else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) + { + dynamic_it->second->insert(value_field); + } + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) + { + dynamic_path_column->insert(value_field); + } + /// We reached the limit on dynamic paths. Add this path to the common data if the value is not Null. + /// (we cannot distinguish cases when path has Null value or is absent in the row and consider them equivalent). + /// Object is actually std::map, so all paths are already sorted and we can add it right now. + else if (!value_field.isNull()) + { + shared_data_paths->insertData(path.data(), path.size()); + auto & shared_data_values_chars = shared_data_values->getChars(); + WriteBufferFromVector value_buf(shared_data_values_chars, AppendModeTag()); + getDynamicSerialization()->serializeBinary(value_field, value_buf, getFormatSettings()); + value_buf.finalize(); + shared_data_values_chars.push_back(0); + shared_data_values->getOffsets().push_back(shared_data_values_chars.size()); + } + } + + shared_data_offsets.push_back(shared_data_paths->size()); + + /// Fill all remaining typed and dynamic paths with default values. + for (auto & [_, column] : typed_paths) + { + if (column->size() == current_size) + column->insertDefault(); + } + + for (auto & [_, column] : dynamic_paths) + { + if (column->size() == current_size) + column->insertDefault(); + } +} + +bool ColumnObject::tryInsert(const Field & x) +{ + if (x.getType() != Field::Types::Which::Object) + return false; + + const auto & object = x.get(); + auto & shared_data_offsets = getSharedDataOffsets(); + auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + size_t prev_size = size(); + size_t prev_paths_size = shared_data_paths->size(); + size_t prev_values_size = shared_data_values->size(); + auto restore_sizes = [&]() + { + for (auto & [_, column] : typed_paths) + { + if (column->size() != prev_size) + column->popBack(column->size() - prev_size); + } + + for (auto & [_, column] : dynamic_paths) + { + if (column->size() != prev_size) + column->popBack(column->size() - prev_size); + } + + if (shared_data_paths->size() != prev_paths_size) + shared_data_paths->popBack(shared_data_paths->size() - prev_paths_size); + if (shared_data_values->size() != prev_values_size) + shared_data_values->popBack(shared_data_values->size() - prev_values_size); + }; + + for (const auto & [path, value_field] : object) + { + if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) + { + if (!typed_it->second->tryInsert(value_field)) + { + restore_sizes(); + return false; + } + } + else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) + { + if (!dynamic_it->second->tryInsert(value_field)) + { + restore_sizes(); + return false; + } + } + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) + { + if (!dynamic_path_column->tryInsert(value_field)) + { + restore_sizes(); + return false; + } + } + /// We reached the limit on dynamic paths. Add this path to the common data if the value is not Null. + /// (we cannot distinguish cases when path has Null value or is absent in the row and consider them equivalent). + /// Object is actually std::map, so all paths are already sorted and we can add it right now. + else if (!value_field.isNull()) + { + WriteBufferFromOwnString value_buf; + getDynamicSerialization()->serializeBinary(value_field, value_buf, getFormatSettings()); + shared_data_paths->insertData(path.data(), path.size()); + shared_data_values->insertData(value_buf.str().data(), value_buf.str().size()); + } + } + + shared_data_offsets.push_back(shared_data_paths->size()); + + /// Fill all remaining typed and dynamic paths with default values. + for (auto & [_, column] : typed_paths) + { + if (column->size() == prev_size) + column->insertDefault(); + } + + for (auto & [_, column] : dynamic_paths) + { + if (column->size() == prev_size) + column->insertDefault(); + } + + return true; } void ColumnObject::insertFrom(const IColumn & src, size_t n) { - insert(src[n]); + const auto & src_object_column = assert_cast(src); + + /// First, insert typed paths, they must be the same for both columns. + for (auto & [path, column] : src_object_column.typed_paths) + typed_paths[path]->insertFrom(*column, n); + + /// Second, insert dynamic paths and extend them if needed. + /// We can reach the limit of dynamic paths, and in this case + /// the rest of dynamic paths will be inserted into shared data. + std::vector src_dynamic_paths_for_shared_data; + for (const auto & [path, column] : src_object_column.dynamic_paths) + { + /// Check if we already have such dynamic path. + if (auto it = dynamic_paths.find(path); it != dynamic_paths.end()) + it->second->insertFrom(*column, n); + /// Try to add a new dynamic path. + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) + dynamic_path_column->insertFrom(*column, n); + /// Limit on dynamic paths is reached, add path to shared data later. + else + src_dynamic_paths_for_shared_data.push_back(path); + } + + /// Finally, insert paths from shared data. + insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, src_dynamic_paths_for_shared_data, n, 1); } void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - const auto & src_object = assert_cast(src); + const auto & src_object_column = assert_cast(src); - for (const auto & entry : src_object.subcolumns) + /// First, insert typed paths, they must be the same for both columns. + for (auto & [path, column] : src_object_column.typed_paths) + typed_paths[path]->insertRangeFrom(*column, start, length); + + /// Second, insert dynamic paths and extend them if needed. + /// We can reach the limit of dynamic paths, and in this case + /// the rest of dynamic paths will be inserted into shared data. + std::vector src_dynamic_paths_for_shared_data; + for (const auto & [path, column] : src_object_column.dynamic_paths) { - if (!hasSubcolumn(entry->path)) + /// Check if we already have such dynamic path. + if (auto it = dynamic_paths.find(path); it != dynamic_paths.end()) + it->second->insertRangeFrom(*column, start, length); + /// Try to add a new dynamic path. + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) + dynamic_path_column->insertRangeFrom(*column, start, length); + /// Limit on dynamic paths is reached, add path to shared data later. + else + src_dynamic_paths_for_shared_data.push_back(path); + } + + /// Finally, insert paths from shared data. + insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, src_dynamic_paths_for_shared_data, start, length); +} + +void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::ColumnObject & src_object_column, std::vector & src_dynamic_paths_for_shared_data, size_t start, size_t length) +{ + /// Paths in shared data are sorted, so paths from src_dynamic_paths_for_shared_data should be inserted properly + /// to keep paths sorted. Let's sort them in advance. + std::sort(src_dynamic_paths_for_shared_data.begin(), src_dynamic_paths_for_shared_data.end()); + + /// Check if src object doesn't have any paths in shared data in specified range. + const auto & src_shared_data_offsets = src_object_column.getSharedDataOffsets(); + if (src_shared_data_offsets[ssize_t(start) - 1] == src_shared_data_offsets[ssize_t(start) + length - 1]) + { + size_t current_size = size(); + + /// If no src dynamic columns should be inserted into shared data, insert defaults. + if (src_dynamic_paths_for_shared_data.empty()) { - if (entry->path.hasNested()) - addNestedSubcolumn(entry->path, entry->data.getFieldInfo(), num_rows); + shared_data->insertManyDefaults(length); + } + /// Otherwise insert required src dynamic columns into shared data. + else + { + const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + auto & shared_data_offsets = getSharedDataOffsets(); + for (size_t i = start; i != start + length; ++i) + { + /// Paths in src_dynamic_paths_for_shared_data are already sorted. + for (const auto & path : src_dynamic_paths_for_shared_data) + serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, path, *src_object_column.dynamic_paths.at(path), i); + shared_data_offsets.push_back(shared_data_paths->size()); + } + } + + /// Insert default values in all remaining dynamic paths. + for (auto & [_, column] : dynamic_paths) + { + if (column->size() == current_size) + column->insertManyDefaults(length); + } + return; + } + + /// Src object column contains some paths in shared data in specified range. + /// Iterate over this range and insert all required paths into shared data or dynamic paths. + const auto [src_shared_data_paths, src_shared_data_values] = src_object_column.getSharedDataPathsAndValues(); + const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + auto & shared_data_offsets = getSharedDataOffsets(); + for (size_t row = start; row != start + length; ++row) + { + size_t current_size = shared_data_offsets.size(); + /// Use separate index to iterate over sorted src_dynamic_paths_for_shared_data. + size_t src_dynamic_paths_for_shared_data_index = 0; + size_t offset = src_shared_data_offsets[ssize_t(row) - 1]; + size_t end = src_shared_data_offsets[row]; + for (size_t i = offset; i != end; ++i) + { + auto path = src_shared_data_paths->getDataAt(i); + /// Check if we have this path in dynamic paths. + if (auto it = dynamic_paths.find(path.toString()); it != dynamic_paths.end()) + { + /// Deserialize binary value into dynamic column from shared data. + deserializeValueFromSharedData(src_shared_data_values, i, *it->second); + } else - addSubcolumn(entry->path, num_rows); + { + /// Before inserting this path into shared data check if we need to + /// insert dynamic paths from src_dynamic_paths_for_shared_data before. + while (src_dynamic_paths_for_shared_data_index < src_dynamic_paths_for_shared_data.size() + && src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index] < path) + { + auto dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; + serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.at(dynamic_path), row); + ++src_dynamic_paths_for_shared_data_index; + } + + /// Insert path and value from src shared data to our shared data. + shared_data_paths->insertFrom(*src_shared_data_paths, i); + shared_data_values->insertFrom(*src_shared_data_values, i); + } } - auto & subcolumn = getSubcolumn(entry->path); - subcolumn.insertRangeFrom(entry->data, start, length); - } - - for (auto & entry : subcolumns) - { - if (!src_object.hasSubcolumn(entry->path)) + /// Insert remaining dynamic paths from src_dynamic_paths_for_shared_data. + for (; src_dynamic_paths_for_shared_data_index != src_dynamic_paths_for_shared_data.size(); ++src_dynamic_paths_for_shared_data_index) { - bool inserted = tryInsertManyDefaultsFromNested(entry); - if (!inserted) - entry->data.insertManyDefaults(length); + auto dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; + serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.at(dynamic_path), row); + } + + shared_data_offsets.push_back(shared_data_paths->size()); + + /// Insert default value in all remaining dynamic paths. + for (auto & [_, column] : dynamic_paths) + { + if (column->size() == current_size) + column->insertDefault(); } } - - num_rows += length; - finalize(); } -void ColumnObject::popBack(size_t length) +void ColumnObject::serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const String & path, const IColumn & column, size_t n) { - for (auto & entry : subcolumns) - entry->data.popBack(length); + /// Don't store Null values in shared data. We consider Null value equivalent to the absense + /// of this path in the row because we cannot distinguish these 2 cases for dynamic paths. + if (column.isNullAt(n)) + return; - num_rows -= length; + shared_data_paths->insertData(path.data(), path.size()); + auto & shared_data_values_chars = shared_data_values->getChars(); + WriteBufferFromVector value_buf(shared_data_values_chars, AppendModeTag()); + getDynamicSerialization()->serializeBinary(column, n, value_buf, getFormatSettings()); + value_buf.finalize(); + shared_data_values_chars.push_back(0); + shared_data_values->getOffsets().push_back(shared_data_values_chars.size()); } -template -MutableColumnPtr ColumnObject::applyForSubcolumns(Func && func) const +void ColumnObject::deserializeValueFromSharedData(const ColumnString * shared_data_values, size_t n, IColumn & column) const { - if (!isFinalized()) + auto value_data = shared_data_values->getDataAt(n); + ReadBufferFromMemory buf(value_data.data, value_data.size); + getDynamicSerialization()->deserializeBinary(column, buf, getFormatSettings()); +} + +void ColumnObject::insertDefault() +{ + for (auto & [_, column] : typed_paths) + column->insertDefault(); + for (auto & [_, column] : dynamic_paths) + column->insertDefault(); + shared_data->insertDefault(); +} + +void ColumnObject::insertManyDefaults(size_t length) +{ + for (auto & [_, column] : typed_paths) + column->insertManyDefaults(length); + for (auto & [_, column] : dynamic_paths) + column->insertManyDefaults(length); + shared_data->insertManyDefaults(length); +} + +void ColumnObject::popBack(size_t n) +{ + for (auto & [_, column] : typed_paths) + column->popBack(n); + for (auto & [_, column] : dynamic_paths) + column->popBack(n); + shared_data->popBack(n); +} + +StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const +{ + StringRef res(begin, 0); + // Serialize all paths and values in binary format. + const auto & shared_data_offsets = getSharedDataOffsets(); + size_t offset = shared_data_offsets[ssize_t(n) - 1]; + size_t end = shared_data_offsets[ssize_t(n)]; + size_t num_paths = typed_paths.size() + dynamic_paths.size() + (end - offset); + char * pos = arena.allocContinue(sizeof(size_t), begin); + memcpy(pos, &num_paths, sizeof(size_t)); + res.data = pos - res.size; + res.size += sizeof(size_t); + /// Serialize paths and values from typed paths. + for (const auto & [path, column] : typed_paths) { - auto finalized = cloneFinalized(); - auto & finalized_object = assert_cast(*finalized); - return finalized_object.applyForSubcolumns(std::forward(func)); + size_t path_size = path.size(); + pos = arena.allocContinue(sizeof(size_t) + path_size, begin); + memcpy(pos, &path_size, sizeof(size_t)); + memcpy(pos + sizeof(size_t), path.data(), path_size); + auto data_ref = column->serializeValueIntoArena(n, arena, begin); + res.data = data_ref.data - res.size - sizeof(size_t) - path_size; + res.size += data_ref.size + sizeof(size_t) + path_size; } - auto res = ColumnObject::create(is_nullable); - for (const auto & subcolumn : subcolumns) + /// Serialize paths and values from dynamic paths. + for (const auto & [path, column] : dynamic_paths) { - auto new_subcolumn = func(subcolumn->data.getFinalizedColumn()); - res->addSubcolumn(subcolumn->path, new_subcolumn->assumeMutable()); + WriteBufferFromOwnString buf; + getDynamicSerialization()->serializeBinary(*column, n, buf, getFormatSettings()); + serializePathAndValueIntoArena(arena, begin, path, buf.str(), res); } + /// Serialize paths and values from shared data. + auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + for (size_t i = offset; i != end; ++i) + serializePathAndValueIntoArena(arena, begin, shared_data_paths->getDataAt(i), shared_data_values->getDataAt(i), res); + return res; } -ColumnPtr ColumnObject::permute(const Permutation & perm, size_t limit) const +void ColumnObject::serializePathAndValueIntoArena(DB::Arena & arena, const char *& begin, StringRef path, StringRef value, StringRef & res) const { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.permute(perm, limit); }); + size_t value_size = value.size; + size_t path_size = path.size; + char * pos = arena.allocContinue(sizeof(size_t) + path_size + sizeof(size_t) + value_size, begin); + memcpy(pos, &path_size, sizeof(size_t)); + memcpy(pos + sizeof(size_t), path.data, path_size); + memcpy(pos + sizeof(size_t) + path_size, &value_size, sizeof(size_t)); + memcpy(pos + sizeof(size_t) + path_size + sizeof(size_t), value.data, value_size); + res.data = pos - res.size; + res.size += sizeof(size_t) + path_size + sizeof(size_t) + value_size; } -ColumnPtr ColumnObject::filter(const Filter & filter, ssize_t result_size_hint) const +const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.filter(filter, result_size_hint); }); + /// 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> paths_and_values_for_shared_data; + auto num_paths = unalignedLoad(pos); + pos += sizeof(size_t); + for (size_t i = 0; i != num_paths; ++i) + { + auto path_size = unalignedLoad(pos); + pos += sizeof(size_t); + StringRef path(pos, path_size); + String path_str = path.toString(); + pos += path_size; + /// Check if it's a typed path. In this case we should use + /// deserializeAndInsertFromArena of corresponding column. + if (auto typed_it = typed_paths.find(path_str); typed_it != typed_paths.end()) + { + pos = typed_it->second->deserializeAndInsertFromArena(pos); + } + /// If it's not a typed path, deserialize binary value and try to insert it + /// to dynamic paths or shared data. + else + { + + auto value_size = unalignedLoad(pos); + pos += sizeof(size_t); + StringRef value(pos, value_size); + pos += value_size; + /// Check if we have this path in dynamic paths. + if (auto dynamic_it = dynamic_paths.find(path_str); dynamic_it != dynamic_paths.end()) + { + ReadBufferFromMemory buf(value.data, value.size); + getDynamicSerialization()->deserializeBinary(*dynamic_it->second, buf, getFormatSettings()); + } + /// Try to add a new dynamic path. + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path_str)) + { + ReadBufferFromMemory buf(value.data, value.size); + getDynamicSerialization()->deserializeBinary(*dynamic_path_column, buf, getFormatSettings()); + } + /// Limit on dynamic paths is reached, add this path to shared data later. + else + { + paths_and_values_for_shared_data.emplace_back(path, value); + } + } + } + + /// Sort and insert all paths from paths_and_values_for_shared_data into shared data. + std::sort(paths_and_values_for_shared_data.begin(), paths_and_values_for_shared_data.end()); + const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + for (const auto & [path, value] : paths_and_values_for_shared_data) + { + shared_data_paths->insertData(path.data, path.size); + shared_data_values->insertData(value.data, value.size); + } + + getSharedDataOffsets().push_back(shared_data_paths->size()); + return pos; +} + +const char * ColumnObject::skipSerializedInArena(const char * pos) const +{ + auto num_paths = unalignedLoad(pos); + pos += sizeof(size_t); + for (size_t i = 0; i != num_paths; ++i) + { + auto path_size = unalignedLoad(pos); + pos += sizeof(size_t); + String path(pos, path_size); + pos += path_size; + if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) + { + pos = typed_it->second->skipSerializedInArena(pos); + } + else + { + auto value_size = unalignedLoad(pos); + pos += sizeof(size_t) + value_size; + } + } + + return pos; +} + +void ColumnObject::updateHashWithValue(size_t n, SipHash & hash) const +{ + for (const auto & [_, column] : typed_paths) + column->updateHashWithValue(n, hash); + for (const auto & [_, column] : dynamic_paths) + column->updateHashWithValue(n, hash); + shared_data->updateHashWithValue(n, hash); +} + +void ColumnObject::updateWeakHash32(WeakHash32 & hash) const +{ + for (const auto & [_, column] : typed_paths) + column->updateWeakHash32(hash); + for (const auto & [_, column] : dynamic_paths) + column->updateWeakHash32(hash); + shared_data->updateWeakHash32(hash); +} + +void ColumnObject::updateHashFast(SipHash & hash) const +{ + for (const auto & [_, column] : typed_paths) + column->updateHashFast(hash); + for (const auto & [_, column] : dynamic_paths) + column->updateHashFast(hash); + shared_data->updateHashFast(hash); +} + +ColumnPtr ColumnObject::filter(const Filter & filt, ssize_t result_size_hint) const +{ + std::unordered_map filtered_typed_paths; + filtered_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + filtered_typed_paths[path] = column->filter(filt, result_size_hint); + + std::unordered_map filtered_dynamic_paths; + filtered_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + filtered_dynamic_paths[path] = column->filter(filt, result_size_hint); + + auto filtered_shared_data = shared_data->filter(filt, result_size_hint); + return ColumnObject::create(filtered_typed_paths, filtered_dynamic_paths, filtered_shared_data, max_dynamic_paths, max_dynamic_types); +} + +void ColumnObject::expand(const Filter & mask, bool inverted) +{ + for (auto & [_, column] : typed_paths) + column->expand(mask, inverted); + for (auto & [_, column] : dynamic_paths) + column->expand(mask, inverted); + shared_data->expand(mask, inverted); +} + +ColumnPtr ColumnObject::permute(const Permutation & perm, size_t limit) const +{ + std::unordered_map permuted_typed_paths; + permuted_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + permuted_typed_paths[path] = column->permute(perm, limit); + + std::unordered_map permuted_dynamic_paths; + permuted_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + permuted_dynamic_paths[path] = column->permute(perm, limit); + + auto permuted_shared_data = shared_data->permute(perm, limit); + return ColumnObject::create(permuted_typed_paths, permuted_dynamic_paths, permuted_shared_data, max_dynamic_paths, max_dynamic_types); } ColumnPtr ColumnObject::index(const IColumn & indexes, size_t limit) const { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.index(indexes, limit); }); + std::unordered_map indexed_typed_paths; + indexed_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + indexed_typed_paths[path] = column->index(indexes, limit); + + std::unordered_map indexed_dynamic_paths; + indexed_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + indexed_dynamic_paths[path] = column->index(indexes, limit); + + auto indexed_shared_data = shared_data->index(indexes, limit); + return ColumnObject::create(indexed_typed_paths, indexed_dynamic_paths, indexed_shared_data, max_dynamic_paths, max_dynamic_types); } -ColumnPtr ColumnObject::replicate(const Offsets & offsets) const +ColumnPtr ColumnObject::replicate(const Offsets & replicate_offsets) const { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.replicate(offsets); }); + std::unordered_map replicated_typed_paths; + replicated_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, column] : typed_paths) + replicated_typed_paths[path] = column->replicate(replicate_offsets); + + std::unordered_map replicated_dynamic_paths; + replicated_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + replicated_dynamic_paths[path] = column->replicate(replicate_offsets); + + auto replicated_shared_data = shared_data->replicate(replicate_offsets); + return ColumnObject::create(replicated_typed_paths, replicated_dynamic_paths, replicated_shared_data, max_dynamic_paths, max_dynamic_types); } -MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const +MutableColumns ColumnObject::scatter(ColumnIndex num_columns, const Selector & selector) const { - if (new_size == 0) - return ColumnObject::create(is_nullable); + std::vector> scattered_typed_paths(num_columns); + for (auto & typed_paths_ : scattered_typed_paths) + typed_paths_.reserve(typed_paths.size()); - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.cloneResized(new_size); }); + for (const auto & [path, column] : typed_paths) + { + auto scattered_columns = column->scatter(num_columns, selector); + for (size_t i = 0; i != num_columns; ++i) + scattered_typed_paths[i][path] = std::move(scattered_columns[i]); + } + + std::vector> scattered_dynamic_paths(num_columns); + for (auto & dynamic_paths_ : scattered_dynamic_paths) + dynamic_paths_.reserve(dynamic_paths.size()); + + for (const auto & [path, column] : dynamic_paths) + { + auto scattered_columns = column->scatter(num_columns, selector); + for (size_t i = 0; i != num_columns; ++i) + scattered_dynamic_paths[i][path] = std::move(scattered_columns[i]); + } + + auto scattered_shared_data_columns = shared_data->scatter(num_columns, selector); + MutableColumns result_columns; + result_columns.reserve(num_columns); + for (size_t i = 0; i != num_columns; ++i) + result_columns.emplace_back(ColumnObject::create(std::move(scattered_typed_paths[i]), std::move(scattered_dynamic_paths[i]), std::move(scattered_shared_data_columns[i]), max_dynamic_paths, max_dynamic_types)); + return result_columns; } void ColumnObject::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const { - res.resize(num_rows); + /// Values in ColumnObject are not comparable. + res.resize(size()); iota(res.data(), res.size(), size_t(0)); } -void ColumnObject::getExtremes(Field & min, Field & max) const +void ColumnObject::reserve(size_t n) { - if (num_rows == 0) + for (auto & [_, column] : typed_paths) + column->reserve(n); + for (auto & [_, column] : dynamic_paths) + column->reserve(n); + shared_data->reserve(n); +} + +void ColumnObject::ensureOwnership() +{ + for (auto & [_, column] : typed_paths) + column->ensureOwnership(); + for (auto & [_, column] : dynamic_paths) + column->ensureOwnership(); + shared_data->ensureOwnership(); +} + +size_t ColumnObject::byteSize() const +{ + size_t size = 0; + for (auto & [_, column] : typed_paths) + size += column->byteSize(); + for (auto & [_, column] : dynamic_paths) + size += column->byteSize(); + size += shared_data->byteSize(); + return size; +} + +size_t ColumnObject::byteSizeAt(size_t n) const +{ + size_t size = 0; + for (auto & [_, column] : typed_paths) + size += column->byteSizeAt(n); + for (auto & [_, column] : dynamic_paths) + size += column->byteSizeAt(n); + size += shared_data->byteSizeAt(n); + return size; +} + +size_t ColumnObject::allocatedBytes() const +{ + size_t size = 0; + for (auto & [_, column] : typed_paths) + size += column->allocatedBytes(); + for (auto & [_, column] : dynamic_paths) + size += column->allocatedBytes(); + size += shared_data->allocatedBytes(); + return size; +} + +void ColumnObject::protect() +{ + for (auto & [_, column] : typed_paths) + column->protect(); + for (auto & [_, column] : dynamic_paths) + column->protect(); + shared_data->protect(); +} + +void ColumnObject::forEachSubcolumn(DB::IColumn::MutableColumnCallback callback) +{ + for (auto & [_, column] : typed_paths) + callback(column); + for (auto & [_, column] : dynamic_paths) + callback(column); + callback(shared_data); +} + +void ColumnObject::forEachSubcolumnRecursively(DB::IColumn::RecursiveMutableColumnCallback callback) +{ + for (auto & [_, column] : typed_paths) { - min = Object(); - max = Object(); + callback(*column); + column->forEachSubcolumnRecursively(callback); } - else + for (auto & [_, column] : dynamic_paths) { - get(0, min); - get(0, max); + callback(*column); + column->forEachSubcolumnRecursively(callback); } + callback(*shared_data); + shared_data->forEachSubcolumnRecursively(callback); } -const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) const +bool ColumnObject::structureEquals(const IColumn & rhs) const { - if (const auto * node = subcolumns.findLeaf(key)) - return node->data; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObject", key.getPath()); -} - -ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) -{ - if (const auto * node = subcolumns.findLeaf(key)) - return const_cast(node)->data; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObject", key.getPath()); -} - -bool ColumnObject::hasSubcolumn(const PathInData & key) const -{ - return subcolumns.findLeaf(key) != nullptr; -} - -void ColumnObject::addSubcolumn(const PathInData & key, MutableColumnPtr && subcolumn) -{ - size_t new_size = subcolumn->size(); - bool inserted = subcolumns.add(key, Subcolumn(std::move(subcolumn), is_nullable)); - - if (!inserted) - throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath()); - - if (num_rows == 0) - num_rows = new_size; - else if (new_size != num_rows) - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, - "Size of subcolumn {} ({}) is inconsistent with column size ({})", - key.getPath(), new_size, num_rows); -} - -void ColumnObject::addSubcolumn(const PathInData & key, size_t new_size) -{ - bool inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable)); - if (!inserted) - throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath()); - - if (num_rows == 0) - num_rows = new_size; - else if (new_size != num_rows) - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, - "Required size of subcolumn {} ({}) is inconsistent with column size ({})", - key.getPath(), new_size, num_rows); -} - -void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size) -{ - if (!key.hasNested()) - throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, - "Cannot add Nested subcolumn, because path doesn't contain Nested"); - - bool inserted = false; - /// We find node that represents the same Nested type as @key. - const auto * nested_node = subcolumns.findBestMatch(key); - - if (nested_node) - { - /// Find any leaf of Nested subcolumn. - const auto * leaf = Subcolumns::findLeaf(nested_node, [&](const auto &) { return true; }); - assert(leaf); - - /// Recreate subcolumn with default values and the same sizes of arrays. - auto new_subcolumn = leaf->data.recreateWithDefaultValues(field_info); - - /// It's possible that we have already inserted value from current row - /// to this subcolumn. So, adjust size to expected. - if (new_subcolumn.size() > new_size) - new_subcolumn.popBack(new_subcolumn.size() - new_size); - - assert(new_subcolumn.size() == new_size); - inserted = subcolumns.add(key, new_subcolumn); - } - else - { - /// If node was not found just add subcolumn with empty arrays. - inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable)); - } - - if (!inserted) - throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath()); - - if (num_rows == 0) - num_rows = new_size; - else if (new_size != num_rows) - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, - "Required size of subcolumn {} ({}) is inconsistent with column size ({})", - key.getPath(), new_size, num_rows); -} - -const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(const Subcolumns::NodePtr & entry) const -{ - if (!entry->path.hasNested()) - return nullptr; - - size_t old_size = entry->data.size(); - const auto * current_node = subcolumns.findLeaf(entry->path); - const Subcolumns::Node * leaf = nullptr; - - while (current_node) - { - /// Try to find the first Nested up to the current node. - const auto * node_nested = Subcolumns::findParent(current_node, - [](const auto & candidate) { return candidate.isNested(); }); - - if (!node_nested) - break; - - /// Find the leaf with subcolumn that contains values - /// for the last rows. - /// If there are no leaves, skip current node and find - /// the next node up to the current. - leaf = Subcolumns::findLeaf(node_nested, - [&](const auto & candidate) - { - return candidate.data.size() > old_size; - }); - - if (leaf) - break; - - current_node = node_nested->parent; - } - - if (leaf && isNothing(leaf->data.getLeastCommonTypeBase())) - return nullptr; - - return leaf; -} - -bool ColumnObject::tryInsertManyDefaultsFromNested(const Subcolumns::NodePtr & entry) const -{ - const auto * leaf = getLeafOfTheSameNested(entry); - if (!leaf) + /// 2 Object columns have equal structure if they have the same typed paths and max_dynamic_paths/max_dynamic_types. + const auto * rhs_object = typeid_cast(&rhs); + if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || max_dynamic_paths != rhs_object->max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types) return false; - size_t old_size = entry->data.size(); - auto field_info = entry->data.getFieldInfo(); + for (const auto & [path, column] : typed_paths) + { + auto it = rhs_object->typed_paths.find(path); + if (it == rhs_object->typed_paths.end() || !it->second->structureEquals(*column)) + return false; + } - /// Cut the needed range from the found leaf - /// and replace scalar values to the correct - /// default values for given entry. - auto new_subcolumn = leaf->data - .cut(old_size, leaf->data.size() - old_size) - .recreateWithDefaultValues(field_info); - - entry->data.insertRangeFrom(new_subcolumn, 0, new_subcolumn.size()); return true; } -bool ColumnObject::tryInsertDefaultFromNested(const Subcolumns::NodePtr & entry) const +ColumnPtr ColumnObject::compress() const { - const auto * leaf = getLeafOfTheSameNested(entry); - if (!leaf) - return false; + std::unordered_map compressed_typed_paths; + compressed_typed_paths.reserve(typed_paths.size()); + size_t byte_size = 0; + for (const auto & [path, column] : typed_paths) + { + auto compressed_column = column->compress(); + byte_size += compressed_column->byteSize(); + compressed_typed_paths[path] = std::move(compressed_column); + } - auto last_field = leaf->data.getLastField(); - if (last_field.isNull()) - return false; + std::unordered_map compressed_dynamic_paths; + compressed_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, column] : dynamic_paths) + { + auto compressed_column = column->compress(); + byte_size += compressed_column->byteSize(); + compressed_dynamic_paths[path] = std::move(compressed_column); + } - size_t leaf_num_dimensions = leaf->data.getNumberOfDimensions(); - size_t entry_num_dimensions = entry->data.getNumberOfDimensions(); + auto compressed_shared_data = shared_data->compress(); + byte_size += compressed_shared_data->byteSize(); - auto default_scalar = entry_num_dimensions > leaf_num_dimensions - ? createEmptyArrayField(entry_num_dimensions - leaf_num_dimensions) - : entry->data.getLeastCommonTypeBase()->getDefault(); + auto decompress = + [my_compressed_typed_paths = std::move(compressed_typed_paths), + my_compressed_dynamic_paths = std::move(compressed_dynamic_paths), + my_compressed_shared_data = std::move(compressed_shared_data), + my_max_dynamic_paths = max_dynamic_paths, + my_max_dynamic_types = max_dynamic_types, + my_statistics = statistics]() mutable + { + std::unordered_map decompressed_typed_paths; + decompressed_typed_paths.reserve(my_compressed_typed_paths.size()); + for (const auto & [path, column] : my_compressed_typed_paths) + decompressed_typed_paths[path] = column->decompress(); - auto default_field = applyVisitor(FieldVisitorReplaceScalars(default_scalar, leaf_num_dimensions), last_field); - entry->data.insert(std::move(default_field)); - return true; -} + std::unordered_map decompressed_dynamic_paths; + decompressed_dynamic_paths.reserve(my_compressed_dynamic_paths.size()); + for (const auto & [path, column] : my_compressed_dynamic_paths) + decompressed_dynamic_paths[path] = column->decompress(); -PathsInData ColumnObject::getKeys() const -{ - PathsInData keys; - keys.reserve(subcolumns.size()); - for (const auto & entry : subcolumns) - keys.emplace_back(entry->path); - return keys; -} + auto decompressed_shared_data = my_compressed_shared_data->decompress(); + return ColumnObject::create(decompressed_typed_paths, decompressed_dynamic_paths, decompressed_shared_data, my_max_dynamic_paths, my_max_dynamic_types, my_statistics); + }; -bool ColumnObject::isFinalized() const -{ - return std::all_of(subcolumns.begin(), subcolumns.end(), - [](const auto & entry) { return entry->data.isFinalized(); }); + return ColumnCompressed::create(size(), byte_size, decompress); } void ColumnObject::finalize() { - size_t old_size = size(); - Subcolumns new_subcolumns; - for (auto && entry : subcolumns) + for (auto & [_, column] : typed_paths) + column->finalize(); + for (auto & [_, column] : dynamic_paths) + column->finalize(); + shared_data->finalize(); +} + +bool ColumnObject::isFinalized() const +{ + bool finalized = true; + for (auto & [_, column] : typed_paths) + finalized &= column->isFinalized(); + for (auto & [_, column] : dynamic_paths) + finalized &= column->isFinalized(); + finalized &= shared_data->isFinalized(); + return finalized; +} + +void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & source_columns) +{ + if (!empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Object column"); + + /// During serialization of Object column in MergeTree all Object columns + /// in single part must have the same structure (the same dynamic paths). During merge + /// resulting column is constructed by inserting from source columns, + /// but it may happen that resulting column doesn't have rows from all source parts + /// but only from subset of them, and as a result some dynamic paths could be missing + /// and structures of resulting column may differ. + /// To solve this problem, before merge we create empty resulting column and use this method + /// to take dynamic structure from all source columns even if we won't insert + /// rows from some of them. + + /// We want to construct resulting set of dynamic paths with paths that have least number of null values in source columns + /// and insert the rest paths into shared data if we exceed the limit of dynamic paths. + /// First, collect all dynamic paths from all source columns and calculate total number of non-null values. + std::unordered_map path_to_total_number_of_non_null_values; + for (const auto & source_column : source_columns) { - const auto & least_common_type = entry->data.getLeastCommonType(); - - /// Do not add subcolumns, which consist only from NULLs. - if (isNothing(getBaseTypeOfArray(least_common_type))) - continue; - - entry->data.finalize(); - new_subcolumns.add(entry->path, entry->data); + const auto & source_object = assert_cast(*source_column); + /// During deserialization from MergeTree we will have statistics from the whole + /// data part with number of non null values for each dynamic path. + const auto & source_statistics = source_object.getStatistics(); + for (const auto & [path, column] : source_object.dynamic_paths) + { + auto it = path_to_total_number_of_non_null_values.find(path); + if (it == path_to_total_number_of_non_null_values.end()) + it = path_to_total_number_of_non_null_values.emplace(path, 0).first; + auto statistics_it = source_statistics.data.find(path); + size_t size = statistics_it == source_statistics.data.end() ? (column->size() - column->getNumberOfDefaultRows()) : statistics_it->second; + it->second += size; + } } - /// If all subcolumns were skipped add a dummy subcolumn, - /// because Tuple type must have at least one element. - if (new_subcolumns.empty()) - new_subcolumns.add(PathInData{COLUMN_NAME_DUMMY}, Subcolumn{ColumnUInt8::create(old_size, 0), is_nullable}); + dynamic_paths.clear(); - std::swap(subcolumns, new_subcolumns); - checkObjectHasNoAmbiguosPaths(getKeys()); + /// Check if the number of all dynamic paths exceeds the limit. + if (path_to_total_number_of_non_null_values.size() > max_dynamic_paths) + { + /// Sort paths by total_number_of_non_null_values. + std::vector> paths_with_sizes; + paths_with_sizes.reserve(path_to_total_number_of_non_null_values.size()); + for (const auto & [path, size] : path_to_total_number_of_non_null_values) + paths_with_sizes.emplace_back(size, path); + std::sort(paths_with_sizes.begin(), paths_with_sizes.end(), std::greater()); + + /// Fill dynamic_paths with first max_dynamic_paths paths in sorted list. + for (size_t i = 0; i != max_dynamic_paths; ++i) + dynamic_paths[paths_with_sizes[i].second] = ColumnDynamic::create(max_dynamic_types); + } + /// Use all dynamic paths from all source columns. + else + { + for (const auto & [path, _] : path_to_total_number_of_non_null_values) + dynamic_paths[path] = ColumnDynamic::create(max_dynamic_types); + } + + /// Fill statistics for the merged part. + statistics.data.clear(); + statistics.source = Statistics::Source::MERGE; + for (const auto & [path, _] : dynamic_paths) + statistics.data[path] = path_to_total_number_of_non_null_values[path]; + + /// Now we have the resulting set of dynamic paths that will be used in all merged columns. + /// As we use Dynamic column for dynamic paths, we should call takeDynamicStructureFromSourceColumns + /// on all resulting dynamic columns. + for (auto & [path, column] : dynamic_paths) + { + Columns dynamic_path_source_columns; + for (const auto & source_column : source_columns) + { + const auto & source_object = assert_cast(*source_column); + auto it = source_object.dynamic_paths.find(path); + if (it != source_object.dynamic_paths.end()) + dynamic_path_source_columns.push_back(it->second); + } + column->takeDynamicStructureFromSourceColumns(dynamic_path_source_columns); + } +} + +size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const ColumnString & shared_data_paths, size_t start, size_t end) +{ + /// Simple random access iterator over values in ColumnString in specified range. + class Iterator : public std::iterator + { + public: + using difference_type = size_t; + Iterator() = delete; + Iterator(const ColumnString * data_, size_t index_) : data(data_), index(index_) {} + Iterator(const Iterator & rhs) : data(rhs.data), index(rhs.index) {} + Iterator & operator=(const Iterator & rhs) { data = rhs.data; index = rhs.index; return *this; } + inline Iterator& operator+=(difference_type rhs) { index += rhs; return *this;} + inline StringRef operator*() const {return data->getDataAt(index);} + + inline Iterator& operator++() { ++index; return *this; } + inline difference_type operator-(const Iterator & rhs) const {return index - rhs.index; } + + const ColumnString * data; + size_t index; + }; + + Iterator start_it(&shared_data_paths, start); + Iterator end_it(&shared_data_paths, end); + auto it = std::lower_bound(start_it, end_it, path); + return it.index; +} + +void ColumnObject::fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end) +{ + const auto & shared_data_array = assert_cast(*shared_data_column); + const auto & shared_data_offsets = shared_data_array.getOffsets(); + size_t first_offset = shared_data_offsets[ssize_t(start) - 1]; + size_t last_offset = shared_data_offsets[ssize_t(end) - 1]; + /// Check if we have at least one row with data. + if (first_offset == last_offset) + { + path_column.insertManyDefaults(end - start); + return; + } + + const auto & shared_data_tuple = assert_cast(shared_data_array.getData()); + const auto & shared_data_paths = assert_cast(shared_data_tuple.getColumn(0)); + const auto & shared_data_values = assert_cast(shared_data_tuple.getColumn(1)); + const auto & dynamic_serialization = getDynamicSerialization(); + for (size_t i = start; i != end; ++i) + { + size_t paths_start = shared_data_offsets[ssize_t(i) - 1]; + size_t paths_end = shared_data_offsets[ssize_t(i)]; + auto lower_bound_path_index = ColumnObject::findPathLowerBoundInSharedData(path, shared_data_paths, paths_start, paths_end); + if (lower_bound_path_index != paths_end && shared_data_paths.getDataAt(lower_bound_path_index) == path) + { + auto value_data = shared_data_values.getDataAt(lower_bound_path_index); + ReadBufferFromMemory buf(value_data.data, value_data.size); + dynamic_serialization->deserializeBinary(path_column, buf, getFormatSettings()); + } + else + { + path_column.insertDefault(); + } + } } } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index e2936b27994..901af6a353d 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -1,265 +1,206 @@ #pragma once #include -#include -#include -#include -#include - +#include +#include +#include +#include #include +#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -/// Info that represents a scalar or array field in a decomposed view. -/// It allows to recreate field with different number -/// of dimensions or nullability. -struct FieldInfo -{ - /// The common type of of all scalars in field. - DataTypePtr scalar_type; - - /// Do we have NULL scalar in field. - bool have_nulls; - - /// If true then we have scalars with different types in array and - /// we need to convert scalars to the common type. - bool need_convert; - - /// Number of dimension in array. 0 if field is scalar. - size_t num_dimensions; - - /// If true then this field is an array of variadic dimension field - /// and we need to normalize the dimension - bool need_fold_dimension; -}; - -FieldInfo getFieldInfo(const Field & field); - -/** A column that represents object with dynamic set of subcolumns. - * Subcolumns are identified by paths in document and are stored in - * a trie-like structure. ColumnObject is not suitable for writing into tables - * and it should be converted to Tuple with fixed set of subcolumns before that. - */ class ColumnObject final : public COWHelper, ColumnObject> { public: - /** Class that represents one subcolumn. - * It stores values in several parts of column - * and keeps current common type of all parts. - * We add a new column part with a new type, when we insert a field, - * which can't be converted to the current common type. - * After insertion of all values subcolumn should be finalized - * for writing and other operations. - */ - class Subcolumn + struct Statistics { - public: - Subcolumn() = default; - Subcolumn(size_t size_, bool is_nullable_); - Subcolumn(MutableColumnPtr && data_, bool is_nullable_); - - size_t size() const; - size_t byteSize() const; - size_t allocatedBytes() const; - void get(size_t n, Field & res) const; - - bool isFinalized() const; - const DataTypePtr & getLeastCommonType() const { return least_common_type.get(); } - const DataTypePtr & getLeastCommonTypeBase() const { return least_common_type.getBase(); } - size_t getNumberOfDimensions() const { return least_common_type.getNumberOfDimensions(); } - - /// Checks the consistency of column's parts stored in @data. - void checkTypes() const; - - /// Inserts a field, which scalars can be arbitrary, but number of - /// dimensions should be consistent with current common type. - void insert(Field field); - void insert(Field field, FieldInfo info); - - void insertDefault(); - void insertManyDefaults(size_t length); - void insertRangeFrom(const Subcolumn & src, size_t start, size_t length); - void popBack(size_t n); - - Subcolumn cut(size_t start, size_t length) const; - - /// Converts all column's parts to the common type and - /// creates a single column that stores all values. - void finalize(); - - /// Returns last inserted field. - Field getLastField() const; - - FieldInfo getFieldInfo() const; - - /// Recreates subcolumn with default scalar values and keeps sizes of arrays. - /// Used to create columns of type Nested with consistent array sizes. - Subcolumn recreateWithDefaultValues(const FieldInfo & field_info) const; - - /// Returns single column if subcolumn in finalizes. - /// Otherwise -- undefined behaviour. - IColumn & getFinalizedColumn(); - const IColumn & getFinalizedColumn() const; - const ColumnPtr & getFinalizedColumnPtr() const; - - const std::vector & getData() const { return data; } - size_t getNumberOfDefaultsInPrefix() const { return num_of_defaults_in_prefix; } - - friend class ColumnObject; - - private: - class LeastCommonType + enum class Source { - public: - LeastCommonType(); - explicit LeastCommonType(DataTypePtr type_); - - const DataTypePtr & get() const { return type; } - const DataTypePtr & getBase() const { return base_type; } - size_t getNumberOfDimensions() const { return num_dimensions; } - - private: - DataTypePtr type; - DataTypePtr base_type; - size_t num_dimensions = 0; + READ, /// Statistics were loaded into column during reading from MergeTree. + MERGE, /// Statistics were calculated during merge of several MergeTree parts. }; - void addNewColumnPart(DataTypePtr type); - - /// Current least common type of all values inserted to this subcolumn. - LeastCommonType least_common_type; - - /// If true then common type type of subcolumn is Nullable - /// and default values are NULLs. - bool is_nullable = false; - - /// Parts of column. Parts should be in increasing order in terms of subtypes/supertypes. - /// That means that the least common type for i-th prefix is the type of i-th part - /// and it's the supertype for all type of column from 0 to i-1. - std::vector data; - - /// Until we insert any non-default field we don't know further - /// least common type and we count number of defaults in prefix, - /// which will be converted to the default type of final common type. - size_t num_of_defaults_in_prefix = 0; - - size_t num_rows = 0; + /// Source of the statistics. + Source source; + /// Statistics data: (path) -> (total number of not-null values). + std::unordered_map data; }; - using Subcolumns = SubcolumnsTree; - private: - /// If true then all subcolumns are nullable. - const bool is_nullable; + friend class COWHelper, ColumnObject>; - Subcolumns subcolumns; - size_t num_rows; + ColumnObject(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_); + ColumnObject(std::unordered_map typed_paths_, std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); public: - static constexpr auto COLUMN_NAME_DUMMY = "_dummy"; + /** Create immutable column using immutable arguments. This arguments may be shared with other columns. + * Use mutate in order to make mutable column and mutate shared nested columns. + */ + using Base = COWHelper, ColumnObject>; + static Ptr create(const std::unordered_map & typed_paths_, const std::unordered_map & dynamic_paths_, const ColumnPtr & shared_data_, size_t max_dynamic_paths_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); + static MutablePtr create(std::unordered_map typed_paths_, std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); + static MutablePtr create(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_); - explicit ColumnObject(bool is_nullable_); - ColumnObject(Subcolumns && subcolumns_, bool is_nullable_); + std::string getName() const override; - /// Checks that all subcolumns have consistent sizes. - void checkConsistency() const; + const char * getFamilyName() const override + { + return "Object"; + } - bool hasSubcolumn(const PathInData & key) const; + TypeIndex getDataType() const override + { + return TypeIndex::Object; + } - const Subcolumn & getSubcolumn(const PathInData & key) const; - Subcolumn & getSubcolumn(const PathInData & key); + MutableColumnPtr cloneEmpty() const override; + MutableColumnPtr cloneResized(size_t size) const override; - void incrementNumRows() { ++num_rows; } + size_t size() const override + { + return shared_data->size(); + } - /// Adds a subcolumn from existing IColumn. - void addSubcolumn(const PathInData & key, MutableColumnPtr && subcolumn); - - /// Adds a subcolumn of specific size with default values. - void addSubcolumn(const PathInData & key, size_t new_size); - - /// Adds a subcolumn of type Nested of specific size with default values. - /// It cares about consistency of sizes of Nested arrays. - void addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size); - - /// Finds a subcolumn from the same Nested type as @entry and inserts - /// an array with default values with consistent sizes as in Nested type. - bool tryInsertDefaultFromNested(const Subcolumns::NodePtr & entry) const; - bool tryInsertManyDefaultsFromNested(const Subcolumns::NodePtr & entry) const; - - const Subcolumns & getSubcolumns() const { return subcolumns; } - Subcolumns & getSubcolumns() { return subcolumns; } - PathsInData getKeys() const; - - /// Part of interface - - const char * getFamilyName() const override { return "Object"; } - TypeIndex getDataType() const override { return TypeIndex::Object; } - - size_t size() const override; - size_t byteSize() const override; - size_t allocatedBytes() const override; - void forEachSubcolumn(MutableColumnCallback callback) override; - void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; - void insert(const Field & field) override; - bool tryInsert(const Field & field) override; - void insertDefault() override; - void insertFrom(const IColumn & src, size_t n) override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; - void popBack(size_t length) override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; - ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; - ColumnPtr replicate(const Offsets & offsets) const override; - MutableColumnPtr cloneResized(size_t new_size) const override; + bool isDefaultAt(size_t n) const override; + StringRef getDataAt(size_t n) const override; + void insertData(const char * pos, size_t length) override; + + void insert(const Field & x) override; + bool tryInsert(const Field & x) override; + void insertFrom(const IColumn & src_, size_t n) override; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + /// TODO: implement more optimal insertManyFrom + void insertDefault() override; + void insertManyDefaults(size_t length) override; + + void popBack(size_t n) override; + + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; + const char * deserializeAndInsertFromArena(const char * pos) override; + const char * skipSerializedInArena(const char * pos) const override; + + void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; + + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; + void expand(const Filter & mask, bool inverted) override; + ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; + ColumnPtr replicate(const Offsets & replicate_offsets) const override; + MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; + + /// Values of ColumnObject are not comparable. + int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } + void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &) const override; + void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} + void getExtremes(Field & min, Field & max) const override + { + min = Object(); + max = Object(); + } + + void reserve(size_t n) override; + void ensureOwnership() override; + size_t byteSize() const override; + size_t byteSizeAt(size_t n) const override; + size_t allocatedBytes() const override; + void protect() override; + + void forEachSubcolumn(MutableColumnCallback callback) override; + + void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; + + bool structureEquals(const IColumn & rhs) const override; + + ColumnPtr compress() const override; - /// Finalizes all subcolumns. void finalize() override; bool isFinalized() const override; - /// Order of rows in ColumnObject is undefined. - void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const override; - void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} - int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } - void getExtremes(Field & min, Field & max) const override; + bool hasDynamicStructure() const override { return true; } + void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; - /// All other methods throw exception. + const std::unordered_map & getTypedPaths() const { return typed_paths; } + std::unordered_map & getTypedPaths() { return typed_paths; } - StringRef getDataAt(size_t) const override { throwMustBeConcrete(); } - bool isDefaultAt(size_t) const override { throwMustBeConcrete(); } - void insertData(const char *, size_t) override { throwMustBeConcrete(); } - StringRef serializeValueIntoArena(size_t, Arena &, char const *&) const override { throwMustBeConcrete(); } - char * serializeValueIntoMemory(size_t, char *) const override { throwMustBeConcrete(); } - const char * deserializeAndInsertFromArena(const char *) override { throwMustBeConcrete(); } - const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } - void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } - void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } - void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } - void expand(const Filter &, bool) override { throwMustBeConcrete(); } - bool hasEqualValues() const override { throwMustBeConcrete(); } - size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } - double getRatioOfDefaultRows(double) const override { throwMustBeConcrete(); } - UInt64 getNumberOfDefaultRows() const override { throwMustBeConcrete(); } - void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwMustBeConcrete(); } + const std::unordered_map & getDynamicPaths() const { return dynamic_paths; } + std::unordered_map & getDynamicPaths() { return dynamic_paths; } -private: - [[noreturn]] static void throwMustBeConcrete() + const Statistics & getStatistics() const { return statistics; } + + const ColumnPtr & getSharedDataPtr() const { return shared_data; } + ColumnPtr & getSharedDataPtr() { return shared_data; } + IColumn & getSharedDataColumn() { return *shared_data; } + + const ColumnArray & getSharedDataNestedColumn() const { return assert_cast(*shared_data); } + ColumnArray & getSharedDataNestedColumn() { return assert_cast(*shared_data); } + + ColumnArray::Offsets & getSharedDataOffsets() { return assert_cast(*shared_data).getOffsets(); } + const ColumnArray::Offsets & getSharedDataOffsets() const { return assert_cast(*shared_data).getOffsets(); } + + std::pair getSharedDataPathsAndValues() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ColumnObject must be converted to ColumnTuple before use"); + auto & column_array = assert_cast(*shared_data); + auto & column_tuple = assert_cast(column_array.getData()); + return {assert_cast(&column_tuple.getColumn(0)), assert_cast(&column_tuple.getColumn(1))}; } - template - MutableColumnPtr applyForSubcolumns(Func && func) const; + std::pair getSharedDataPathsAndValues() const + { + const auto & column_array = assert_cast(*shared_data); + const auto & column_tuple = assert_cast(column_array.getData()); + return {assert_cast(&column_tuple.getColumn(0)), assert_cast(&column_tuple.getColumn(1))}; + } - /// It's used to get shared sized of Nested to insert correct default values. - const Subcolumns::Node * getLeafOfTheSameNested(const Subcolumns::NodePtr & entry) const; + size_t getMaxDynamicTypes() const { return max_dynamic_types; } + size_t getMaxDynamicPaths() const { return max_dynamic_paths; } + + /// Try to add new dynamic path. Returns pointer to the new dynamic + /// path column or nullptr if limit on dynamic paths is reached. + IColumn * tryToAddNewDynamicPath(const String & path); + + void setDynamicPaths(const std::vector & paths); + void setStatistics(const Statistics & statistics_) { statistics = statistics_; } + + void serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const String & path, const IColumn & column, size_t n); + void deserializeValueFromSharedData(const ColumnString * shared_data_values, size_t n, IColumn & column) const; + + /// Paths in shared data are sorted in each row. Use this method to find the lower bound for specific path in the row. + static size_t findPathLowerBoundInSharedData(StringRef path, const ColumnString & shared_data_paths, size_t start, size_t end); + /// Insert all the data from shared data with specified path to dynamic column. + static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); + +private: + void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector & dynamic_paths_to_shared_data, size_t start, size_t length); + void serializePathAndValueIntoArena(Arena & arena, const char *& begin, StringRef path, StringRef value, StringRef & res) const; + + /// Map path -> column for paths with explicitly specified types. + /// This set of paths is constant and cannot be changed. + std::unordered_map typed_paths; + /// Map path -> column for dynamically added paths. All columns + /// here are Dynamic columns. This set of paths can be extended + /// during inerts into the column. + std::unordered_map dynamic_paths; + /// Shared storage for all other paths and values. It's filled + /// when the number of dynamic paths reaches the limit. + /// It has type Array(Tuple(String, String)) and stores + /// an array of pairs (path, binary serialized dynamic value) for each row. + WrappedPtr shared_data; + + /// Maximum number of dynamic paths. If this limit is reached, all new paths will be inserted into shared data. + size_t max_dynamic_paths; + /// Maximum number of dynamic types for each dynamic path. Used while creating Dynamic columns for new dynamic paths. + size_t max_dynamic_types; + /// Statistics on the number of non-null values for each dynamic path in the MergeTree data part. + /// Calculated during merges or reading from MergeTree. Used to determine the set of dynamic paths for the merged part. + Statistics statistics; }; + } diff --git a/src/Columns/ColumnObjectDeprecated.cpp b/src/Columns/ColumnObjectDeprecated.cpp new file mode 100644 index 00000000000..d56f2a10309 --- /dev/null +++ b/src/Columns/ColumnObjectDeprecated.cpp @@ -0,0 +1,1096 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int DUPLICATE_COLUMN; + extern const int EXPERIMENTAL_FEATURE_ERROR; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; +} + +namespace +{ + +/// Recreates column with default scalar values and keeps sizes of arrays. +ColumnPtr recreateColumnWithDefaultValues( + const ColumnPtr & column, const DataTypePtr & scalar_type, size_t num_dimensions) +{ + const auto * column_array = checkAndGetColumn(column.get()); + if (column_array && num_dimensions) + { + return ColumnArray::create( + recreateColumnWithDefaultValues( + column_array->getDataPtr(), scalar_type, num_dimensions - 1), + IColumn::mutate(column_array->getOffsetsPtr())); + } + + return createArrayOfType(scalar_type, num_dimensions)->createColumn()->cloneResized(column->size()); +} + +/// Replaces NULL fields to given field or empty array. +class FieldVisitorReplaceNull : public StaticVisitor +{ +public: + explicit FieldVisitorReplaceNull( + const Field & replacement_, size_t num_dimensions_) + : replacement(replacement_) + , num_dimensions(num_dimensions_) + { + } + + Field operator()(const Null &) const + { + return num_dimensions ? Array() : replacement; + } + + Field operator()(const Array & x) const + { + assert(num_dimensions > 0); + const size_t size = x.size(); + Array res(size); + for (size_t i = 0; i < size; ++i) + res[i] = applyVisitor(FieldVisitorReplaceNull(replacement, num_dimensions - 1), x[i]); + return res; + } + + template + Field operator()(const T & x) const { return x; } + +private: + const Field & replacement; + size_t num_dimensions; +}; + +/// Visitor that allows to get type of scalar field +/// or least common type of scalars in array. +/// More optimized version of FieldToDataType. +class FieldVisitorToScalarType : public StaticVisitor<> +{ +public: + using FieldType = Field::Types::Which; + + void operator()(const Array & x) + { + size_t size = x.size(); + for (size_t i = 0; i < size; ++i) + applyVisitor(*this, x[i]); + } + + void operator()(const UInt64 & x) + { + field_types.insert(FieldType::UInt64); + if (x <= std::numeric_limits::max()) + type_indexes.insert(TypeIndex::UInt8); + else if (x <= std::numeric_limits::max()) + type_indexes.insert(TypeIndex::UInt16); + else if (x <= std::numeric_limits::max()) + type_indexes.insert(TypeIndex::UInt32); + else + type_indexes.insert(TypeIndex::UInt64); + } + + void operator()(const Int64 & x) + { + field_types.insert(FieldType::Int64); + if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) + type_indexes.insert(TypeIndex::Int8); + else if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) + type_indexes.insert(TypeIndex::Int16); + else if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) + type_indexes.insert(TypeIndex::Int32); + else + type_indexes.insert(TypeIndex::Int64); + } + + void operator()(const bool &) + { + field_types.insert(FieldType::UInt64); + type_indexes.insert(TypeIndex::UInt8); + } + + void operator()(const Null &) + { + have_nulls = true; + } + + template + void operator()(const T &) + { + field_types.insert(Field::TypeToEnum>::value); + type_indexes.insert(TypeToTypeIndex>); + } + + DataTypePtr getScalarType() const { return getLeastSupertypeOrString(type_indexes); } + bool haveNulls() const { return have_nulls; } + bool needConvertField() const { return field_types.size() > 1; } + +private: + TypeIndexSet type_indexes; + std::unordered_set field_types; + bool have_nulls = false; +}; + +} + +FieldInfo getFieldInfo(const Field & field) +{ + FieldVisitorToScalarType to_scalar_type_visitor; + applyVisitor(to_scalar_type_visitor, field); + FieldVisitorToNumberOfDimensions to_number_dimension_visitor; + + return + { + to_scalar_type_visitor.getScalarType(), + to_scalar_type_visitor.haveNulls(), + to_scalar_type_visitor.needConvertField(), + applyVisitor(to_number_dimension_visitor, field), + to_number_dimension_visitor.need_fold_dimension + }; +} + +ColumnObjectDeprecated::Subcolumn::Subcolumn(MutableColumnPtr && data_, bool is_nullable_) + : least_common_type(getDataTypeByColumn(*data_)) + , is_nullable(is_nullable_) + , num_rows(data_->size()) +{ + data.push_back(std::move(data_)); +} + +ColumnObjectDeprecated::Subcolumn::Subcolumn( + size_t size_, bool is_nullable_) + : least_common_type(std::make_shared()) + , is_nullable(is_nullable_) + , num_of_defaults_in_prefix(size_) + , num_rows(size_) +{ +} + +size_t ColumnObjectDeprecated::Subcolumn::size() const +{ + return num_rows; +} + +size_t ColumnObjectDeprecated::Subcolumn::byteSize() const +{ + size_t res = 0; + for (const auto & part : data) + res += part->byteSize(); + return res; +} + +size_t ColumnObjectDeprecated::Subcolumn::allocatedBytes() const +{ + size_t res = 0; + for (const auto & part : data) + res += part->allocatedBytes(); + return res; +} + +void ColumnObjectDeprecated::Subcolumn::get(size_t n, Field & res) const +{ + if (isFinalized()) + { + getFinalizedColumn().get(n, res); + return; + } + + size_t ind = n; + if (ind < num_of_defaults_in_prefix) + { + res = least_common_type.get()->getDefault(); + return; + } + + ind -= num_of_defaults_in_prefix; + for (const auto & part : data) + { + if (ind < part->size()) + { + part->get(ind, res); + res = convertFieldToTypeOrThrow(res, *least_common_type.get()); + return; + } + + ind -= part->size(); + } + + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for getting field is out of range", n); +} + +void ColumnObjectDeprecated::Subcolumn::checkTypes() const +{ + DataTypes prefix_types; + prefix_types.reserve(data.size()); + for (size_t i = 0; i < data.size(); ++i) + { + auto current_type = getDataTypeByColumn(*data[i]); + prefix_types.push_back(current_type); + auto prefix_common_type = getLeastSupertype(prefix_types); + if (!prefix_common_type->equals(*current_type)) + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, + "Data type {} of column at position {} cannot represent all columns from i-th prefix", + current_type->getName(), i); + } +} + +void ColumnObjectDeprecated::Subcolumn::insert(Field field) +{ + auto info = DB::getFieldInfo(field); + insert(std::move(field), std::move(info)); +} + +void ColumnObjectDeprecated::Subcolumn::addNewColumnPart(DataTypePtr type) +{ + auto serialization = type->getSerialization(ISerialization::Kind::SPARSE); + data.push_back(type->createColumn(*serialization)); + least_common_type = LeastCommonType{std::move(type)}; +} + +static bool isConversionRequiredBetweenIntegers(const IDataType & lhs, const IDataType & rhs) +{ + /// If both of types are signed/unsigned integers and size of left field type + /// is less than right type, we don't need to convert field, + /// because all integer fields are stored in Int64/UInt64. + + WhichDataType which_lhs(lhs); + WhichDataType which_rhs(rhs); + + bool is_native_int = which_lhs.isNativeInt() && which_rhs.isNativeInt(); + bool is_native_uint = which_lhs.isNativeUInt() && which_rhs.isNativeUInt(); + + return (!is_native_int && !is_native_uint) + || lhs.getSizeOfValueInMemory() > rhs.getSizeOfValueInMemory(); +} + +void ColumnObjectDeprecated::Subcolumn::insert(Field field, FieldInfo info) +{ + auto base_type = std::move(info.scalar_type); + + if (isNothing(base_type) && info.num_dimensions == 0) + { + insertDefault(); + return; + } + + auto column_dim = least_common_type.getNumberOfDimensions(); + auto value_dim = info.num_dimensions; + + if (isNothing(least_common_type.get())) + column_dim = value_dim; + + if (isNothing(base_type)) + value_dim = column_dim; + + if (value_dim != column_dim) + throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, + "Dimension of types mismatched between inserted value and column. " + "Dimension of value: {}. Dimension of column: {}", + value_dim, column_dim); + + if (is_nullable) + base_type = makeNullable(base_type); + + if (!is_nullable && info.have_nulls) + field = applyVisitor(FieldVisitorReplaceNull(base_type->getDefault(), value_dim), std::move(field)); + + bool type_changed = false; + const auto & least_common_base_type = least_common_type.getBase(); + + if (data.empty()) + { + addNewColumnPart(createArrayOfType(std::move(base_type), value_dim)); + } + else if (!least_common_base_type->equals(*base_type) && !isNothing(base_type)) + { + if (isConversionRequiredBetweenIntegers(*base_type, *least_common_base_type)) + { + base_type = getLeastSupertypeOrString(DataTypes{std::move(base_type), least_common_base_type}); + type_changed = true; + if (!least_common_base_type->equals(*base_type)) + addNewColumnPart(createArrayOfType(std::move(base_type), value_dim)); + } + } + + if (type_changed || info.need_convert) + field = convertFieldToTypeOrThrow(field, *least_common_type.get()); + + if (!data.back()->tryInsert(field)) + { + /** Normalization of the field above is pretty complicated (it uses several FieldVisitors), + * so in the case of a bug, we may get mismatched types. + * The `IColumn::insert` method does not check the type of the inserted field, and it can lead to a segmentation fault. + * Therefore, we use the safer `tryInsert` method to get an exception instead of a segmentation fault. + */ + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, + "Cannot insert field {} to column {}", + field.dump(), data.back()->dumpStructure()); + } + + ++num_rows; +} + +void ColumnObjectDeprecated::Subcolumn::insertRangeFrom(const Subcolumn & src, size_t start, size_t length) +{ + assert(start + length <= src.size()); + size_t end = start + length; + num_rows += length; + + if (data.empty()) + { + addNewColumnPart(src.getLeastCommonType()); + } + else if (!least_common_type.get()->equals(*src.getLeastCommonType())) + { + auto new_least_common_type = getLeastSupertypeOrString(DataTypes{least_common_type.get(), src.getLeastCommonType()}); + if (!new_least_common_type->equals(*least_common_type.get())) + addNewColumnPart(std::move(new_least_common_type)); + } + + if (end <= src.num_of_defaults_in_prefix) + { + data.back()->insertManyDefaults(length); + return; + } + + if (start < src.num_of_defaults_in_prefix) + data.back()->insertManyDefaults(src.num_of_defaults_in_prefix - start); + + auto insert_from_part = [&](const auto & column, size_t from, size_t n) + { + assert(from + n <= column->size()); + auto column_type = getDataTypeByColumn(*column); + + if (column_type->equals(*least_common_type.get())) + { + data.back()->insertRangeFrom(*column, from, n); + return; + } + + /// If we need to insert large range, there is no sense to cut part of column and cast it. + /// Casting of all column and inserting from it can be faster. + /// Threshold is just a guess. + + if (n * 3 >= column->size()) + { + auto casted_column = castColumn({column, column_type, ""}, least_common_type.get()); + data.back()->insertRangeFrom(*casted_column, from, n); + return; + } + + auto casted_column = column->cut(from, n); + casted_column = castColumn({casted_column, column_type, ""}, least_common_type.get()); + data.back()->insertRangeFrom(*casted_column, 0, n); + }; + + size_t pos = 0; + size_t processed_rows = src.num_of_defaults_in_prefix; + + /// Find the first part of the column that intersects the range. + while (pos < src.data.size() && processed_rows + src.data[pos]->size() < start) + { + processed_rows += src.data[pos]->size(); + ++pos; + } + + /// Insert from the first part of column. + if (pos < src.data.size() && processed_rows < start) + { + size_t part_start = start - processed_rows; + size_t part_length = std::min(src.data[pos]->size() - part_start, end - start); + insert_from_part(src.data[pos], part_start, part_length); + processed_rows += src.data[pos]->size(); + ++pos; + } + + /// Insert from the parts of column in the middle of range. + while (pos < src.data.size() && processed_rows + src.data[pos]->size() < end) + { + insert_from_part(src.data[pos], 0, src.data[pos]->size()); + processed_rows += src.data[pos]->size(); + ++pos; + } + + /// Insert from the last part of column if needed. + if (pos < src.data.size() && processed_rows < end) + { + size_t part_end = end - processed_rows; + insert_from_part(src.data[pos], 0, part_end); + } +} + +bool ColumnObjectDeprecated::Subcolumn::isFinalized() const +{ + return num_of_defaults_in_prefix == 0 && + (data.empty() || (data.size() == 1 && !data[0]->isSparse())); +} + +void ColumnObjectDeprecated::Subcolumn::finalize() +{ + if (isFinalized()) + return; + + if (data.size() == 1 && num_of_defaults_in_prefix == 0) + { + data[0] = data[0]->convertToFullColumnIfSparse(); + return; + } + + const auto & to_type = least_common_type.get(); + auto result_column = to_type->createColumn(); + + if (num_of_defaults_in_prefix) + result_column->insertManyDefaults(num_of_defaults_in_prefix); + + for (auto & part : data) + { + part = part->convertToFullColumnIfSparse(); + auto from_type = getDataTypeByColumn(*part); + size_t part_size = part->size(); + + if (!from_type->equals(*to_type)) + { + auto offsets = ColumnUInt64::create(); + auto & offsets_data = offsets->getData(); + + /// We need to convert only non-default values and then recreate column + /// with default value of new type, because default values (which represents misses in data) + /// may be inconsistent between types (e.g "0" in UInt64 and empty string in String). + + part->getIndicesOfNonDefaultRows(offsets_data, 0, part_size); + + if (offsets->size() == part_size) + { + part = castColumn({part, from_type, ""}, to_type); + } + else + { + auto values = part->index(*offsets, offsets->size()); + values = castColumn({values, from_type, ""}, to_type); + part = values->createWithOffsets(offsets_data, *createColumnConstWithDefaultValue(result_column->getPtr()), part_size, /*shift=*/ 0); + } + } + + result_column->insertRangeFrom(*part, 0, part_size); + } + + data = { std::move(result_column) }; + num_of_defaults_in_prefix = 0; +} + +void ColumnObjectDeprecated::Subcolumn::insertDefault() +{ + if (data.empty()) + ++num_of_defaults_in_prefix; + else + data.back()->insertDefault(); + + ++num_rows; +} + +void ColumnObjectDeprecated::Subcolumn::insertManyDefaults(size_t length) +{ + if (data.empty()) + num_of_defaults_in_prefix += length; + else + data.back()->insertManyDefaults(length); + + num_rows += length; +} + +void ColumnObjectDeprecated::Subcolumn::popBack(size_t n) +{ + assert(n <= size()); + + num_rows -= n; + size_t num_removed = 0; + for (auto it = data.rbegin(); it != data.rend(); ++it) + { + if (n == 0) + break; + + auto & column = *it; + if (n < column->size()) + { + column->popBack(n); + n = 0; + } + else + { + ++num_removed; + n -= column->size(); + } + } + + data.resize(data.size() - num_removed); + num_of_defaults_in_prefix -= n; +} + +ColumnObjectDeprecated::Subcolumn ColumnObjectDeprecated::Subcolumn::cut(size_t start, size_t length) const +{ + Subcolumn new_subcolumn(0, is_nullable); + new_subcolumn.insertRangeFrom(*this, start, length); + return new_subcolumn; +} + +Field ColumnObjectDeprecated::Subcolumn::getLastField() const +{ + if (data.empty()) + return Field(); + + const auto & last_part = data.back(); + assert(!last_part->empty()); + return (*last_part)[last_part->size() - 1]; +} + +FieldInfo ColumnObjectDeprecated::Subcolumn::getFieldInfo() const +{ + const auto & base_type = least_common_type.getBase(); + return FieldInfo + { + .scalar_type = base_type, + .have_nulls = base_type->isNullable(), + .need_convert = false, + .num_dimensions = least_common_type.getNumberOfDimensions(), + .need_fold_dimension = false, + }; +} + +ColumnObjectDeprecated::Subcolumn ColumnObjectDeprecated::Subcolumn::recreateWithDefaultValues(const FieldInfo & field_info) const +{ + auto scalar_type = field_info.scalar_type; + if (is_nullable) + scalar_type = makeNullable(scalar_type); + + Subcolumn new_subcolumn(*this); + new_subcolumn.least_common_type = LeastCommonType{createArrayOfType(scalar_type, field_info.num_dimensions)}; + + for (auto & part : new_subcolumn.data) + part = recreateColumnWithDefaultValues(part, scalar_type, field_info.num_dimensions); + + return new_subcolumn; +} + +IColumn & ColumnObjectDeprecated::Subcolumn::getFinalizedColumn() +{ + assert(isFinalized()); + return *data[0]; +} + +const IColumn & ColumnObjectDeprecated::Subcolumn::getFinalizedColumn() const +{ + assert(isFinalized()); + return *data[0]; +} + +const ColumnPtr & ColumnObjectDeprecated::Subcolumn::getFinalizedColumnPtr() const +{ + assert(isFinalized()); + return data[0]; +} + +ColumnObjectDeprecated::Subcolumn::LeastCommonType::LeastCommonType() + : type(std::make_shared()) + , base_type(type) + , num_dimensions(0) +{ +} + +ColumnObjectDeprecated::Subcolumn::LeastCommonType::LeastCommonType(DataTypePtr type_) + : type(std::move(type_)) + , base_type(getBaseTypeOfArray(type)) + , num_dimensions(DB::getNumberOfDimensions(*type)) +{ +} + +ColumnObjectDeprecated::ColumnObjectDeprecated(bool is_nullable_) + : is_nullable(is_nullable_) + , num_rows(0) +{ +} + +ColumnObjectDeprecated::ColumnObjectDeprecated(Subcolumns && subcolumns_, bool is_nullable_) + : is_nullable(is_nullable_) + , subcolumns(std::move(subcolumns_)) + , num_rows(subcolumns.empty() ? 0 : (*subcolumns.begin())->data.size()) + +{ + checkConsistency(); +} + +void ColumnObjectDeprecated::checkConsistency() const +{ + if (subcolumns.empty()) + return; + + for (const auto & leaf : subcolumns) + { + if (num_rows != leaf->data.size()) + { + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Sizes of subcolumns are inconsistent in ColumnObjectDeprecated." + " Subcolumn '{}' has {} rows, but expected size is {}", + leaf->path.getPath(), leaf->data.size(), num_rows); + } + } +} + +size_t ColumnObjectDeprecated::size() const +{ +#ifndef NDEBUG + checkConsistency(); +#endif + return num_rows; +} + +size_t ColumnObjectDeprecated::byteSize() const +{ + size_t res = 0; + for (const auto & entry : subcolumns) + res += entry->data.byteSize(); + return res; +} + +size_t ColumnObjectDeprecated::allocatedBytes() const +{ + size_t res = 0; + for (const auto & entry : subcolumns) + res += entry->data.allocatedBytes(); + return res; +} + +void ColumnObjectDeprecated::forEachSubcolumn(MutableColumnCallback callback) +{ + for (auto & entry : subcolumns) + for (auto & part : entry->data.data) + callback(part); +} + +void ColumnObjectDeprecated::forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) +{ + for (auto & entry : subcolumns) + { + for (auto & part : entry->data.data) + { + callback(*part); + part->forEachSubcolumnRecursively(callback); + } + } +} + +void ColumnObjectDeprecated::insert(const Field & field) +{ + const auto & object = field.get(); + + HashSet inserted_paths; + size_t old_size = size(); + for (const auto & [key_str, value] : object) + { + PathInData key(key_str); + inserted_paths.insert(key_str); + if (!hasSubcolumn(key)) + addSubcolumn(key, old_size); + + auto & subcolumn = getSubcolumn(key); + subcolumn.insert(value); + } + + for (auto & entry : subcolumns) + { + if (!inserted_paths.has(entry->path.getPath())) + { + bool inserted = tryInsertDefaultFromNested(entry); + if (!inserted) + entry->data.insertDefault(); + } + } + + ++num_rows; +} + +bool ColumnObjectDeprecated::tryInsert(const Field & field) +{ + if (field.getType() != Field::Types::Which::Object) + return false; + + insert(field); + return true; +} + +void ColumnObjectDeprecated::insertDefault() +{ + for (auto & entry : subcolumns) + entry->data.insertDefault(); + + ++num_rows; +} + +Field ColumnObjectDeprecated::operator[](size_t n) const +{ + Field object; + get(n, object); + return object; +} + +void ColumnObjectDeprecated::get(size_t n, Field & res) const +{ + assert(n < size()); + res = Object(); + auto & object = res.get(); + + for (const auto & entry : subcolumns) + { + auto it = object.try_emplace(entry->path.getPath()).first; + entry->data.get(n, it->second); + } +} + +void ColumnObjectDeprecated::insertFrom(const IColumn & src, size_t n) +{ + insert(src[n]); +} + +void ColumnObjectDeprecated::insertRangeFrom(const IColumn & src, size_t start, size_t length) +{ + const auto & src_object = assert_cast(src); + + for (const auto & entry : src_object.subcolumns) + { + if (!hasSubcolumn(entry->path)) + { + if (entry->path.hasNested()) + addNestedSubcolumn(entry->path, entry->data.getFieldInfo(), num_rows); + else + addSubcolumn(entry->path, num_rows); + } + + auto & subcolumn = getSubcolumn(entry->path); + subcolumn.insertRangeFrom(entry->data, start, length); + } + + for (auto & entry : subcolumns) + { + if (!src_object.hasSubcolumn(entry->path)) + { + bool inserted = tryInsertManyDefaultsFromNested(entry); + if (!inserted) + entry->data.insertManyDefaults(length); + } + } + + num_rows += length; + finalize(); +} + +void ColumnObjectDeprecated::popBack(size_t length) +{ + for (auto & entry : subcolumns) + entry->data.popBack(length); + + num_rows -= length; +} + +template +MutableColumnPtr ColumnObjectDeprecated::applyForSubcolumns(Func && func) const +{ + if (!isFinalized()) + { + auto finalized = cloneFinalized(); + auto & finalized_object = assert_cast(*finalized); + return finalized_object.applyForSubcolumns(std::forward(func)); + } + + auto res = ColumnObjectDeprecated::create(is_nullable); + for (const auto & subcolumn : subcolumns) + { + auto new_subcolumn = func(subcolumn->data.getFinalizedColumn()); + res->addSubcolumn(subcolumn->path, new_subcolumn->assumeMutable()); + } + + return res; +} + +ColumnPtr ColumnObjectDeprecated::permute(const Permutation & perm, size_t limit) const +{ + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.permute(perm, limit); }); +} + +ColumnPtr ColumnObjectDeprecated::filter(const Filter & filter, ssize_t result_size_hint) const +{ + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.filter(filter, result_size_hint); }); +} + +ColumnPtr ColumnObjectDeprecated::index(const IColumn & indexes, size_t limit) const +{ + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.index(indexes, limit); }); +} + +ColumnPtr ColumnObjectDeprecated::replicate(const Offsets & offsets) const +{ + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.replicate(offsets); }); +} + +MutableColumnPtr ColumnObjectDeprecated::cloneResized(size_t new_size) const +{ + if (new_size == 0) + return ColumnObjectDeprecated::create(is_nullable); + + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.cloneResized(new_size); }); +} + +void ColumnObjectDeprecated::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const +{ + res.resize(num_rows); + iota(res.data(), res.size(), size_t(0)); +} + +void ColumnObjectDeprecated::getExtremes(Field & min, Field & max) const +{ + if (num_rows == 0) + { + min = Object(); + max = Object(); + } + else + { + get(0, min); + get(0, max); + } +} + +const ColumnObjectDeprecated::Subcolumn & ColumnObjectDeprecated::getSubcolumn(const PathInData & key) const +{ + if (const auto * node = subcolumns.findLeaf(key)) + return node->data; + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObjectDeprecated", key.getPath()); +} + +ColumnObjectDeprecated::Subcolumn & ColumnObjectDeprecated::getSubcolumn(const PathInData & key) +{ + if (const auto * node = subcolumns.findLeaf(key)) + return const_cast(node)->data; + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObjectDeprecated", key.getPath()); +} + +bool ColumnObjectDeprecated::hasSubcolumn(const PathInData & key) const +{ + return subcolumns.findLeaf(key) != nullptr; +} + +void ColumnObjectDeprecated::addSubcolumn(const PathInData & key, MutableColumnPtr && subcolumn) +{ + size_t new_size = subcolumn->size(); + bool inserted = subcolumns.add(key, Subcolumn(std::move(subcolumn), is_nullable)); + + if (!inserted) + throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath()); + + if (num_rows == 0) + num_rows = new_size; + else if (new_size != num_rows) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + "Size of subcolumn {} ({}) is inconsistent with column size ({})", + key.getPath(), new_size, num_rows); +} + +void ColumnObjectDeprecated::addSubcolumn(const PathInData & key, size_t new_size) +{ + bool inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable)); + if (!inserted) + throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath()); + + if (num_rows == 0) + num_rows = new_size; + else if (new_size != num_rows) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + "Required size of subcolumn {} ({}) is inconsistent with column size ({})", + key.getPath(), new_size, num_rows); +} + +void ColumnObjectDeprecated::addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size) +{ + if (!key.hasNested()) + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, + "Cannot add Nested subcolumn, because path doesn't contain Nested"); + + bool inserted = false; + /// We find node that represents the same Nested type as @key. + const auto * nested_node = subcolumns.findBestMatch(key); + + if (nested_node) + { + /// Find any leaf of Nested subcolumn. + const auto * leaf = Subcolumns::findLeaf(nested_node, [&](const auto &) { return true; }); + assert(leaf); + + /// Recreate subcolumn with default values and the same sizes of arrays. + auto new_subcolumn = leaf->data.recreateWithDefaultValues(field_info); + + /// It's possible that we have already inserted value from current row + /// to this subcolumn. So, adjust size to expected. + if (new_subcolumn.size() > new_size) + new_subcolumn.popBack(new_subcolumn.size() - new_size); + + assert(new_subcolumn.size() == new_size); + inserted = subcolumns.add(key, new_subcolumn); + } + else + { + /// If node was not found just add subcolumn with empty arrays. + inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable)); + } + + if (!inserted) + throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath()); + + if (num_rows == 0) + num_rows = new_size; + else if (new_size != num_rows) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + "Required size of subcolumn {} ({}) is inconsistent with column size ({})", + key.getPath(), new_size, num_rows); +} + +const ColumnObjectDeprecated::Subcolumns::Node * ColumnObjectDeprecated::getLeafOfTheSameNested(const Subcolumns::NodePtr & entry) const +{ + if (!entry->path.hasNested()) + return nullptr; + + size_t old_size = entry->data.size(); + const auto * current_node = subcolumns.findLeaf(entry->path); + const Subcolumns::Node * leaf = nullptr; + + while (current_node) + { + /// Try to find the first Nested up to the current node. + const auto * node_nested = Subcolumns::findParent(current_node, + [](const auto & candidate) { return candidate.isNested(); }); + + if (!node_nested) + break; + + /// Find the leaf with subcolumn that contains values + /// for the last rows. + /// If there are no leaves, skip current node and find + /// the next node up to the current. + leaf = Subcolumns::findLeaf(node_nested, + [&](const auto & candidate) + { + return candidate.data.size() > old_size; + }); + + if (leaf) + break; + + current_node = node_nested->parent; + } + + if (leaf && isNothing(leaf->data.getLeastCommonTypeBase())) + return nullptr; + + return leaf; +} + +bool ColumnObjectDeprecated::tryInsertManyDefaultsFromNested(const Subcolumns::NodePtr & entry) const +{ + const auto * leaf = getLeafOfTheSameNested(entry); + if (!leaf) + return false; + + size_t old_size = entry->data.size(); + auto field_info = entry->data.getFieldInfo(); + + /// Cut the needed range from the found leaf + /// and replace scalar values to the correct + /// default values for given entry. + auto new_subcolumn = leaf->data + .cut(old_size, leaf->data.size() - old_size) + .recreateWithDefaultValues(field_info); + + entry->data.insertRangeFrom(new_subcolumn, 0, new_subcolumn.size()); + return true; +} + +bool ColumnObjectDeprecated::tryInsertDefaultFromNested(const Subcolumns::NodePtr & entry) const +{ + const auto * leaf = getLeafOfTheSameNested(entry); + if (!leaf) + return false; + + auto last_field = leaf->data.getLastField(); + if (last_field.isNull()) + return false; + + size_t leaf_num_dimensions = leaf->data.getNumberOfDimensions(); + size_t entry_num_dimensions = entry->data.getNumberOfDimensions(); + + auto default_scalar = entry_num_dimensions > leaf_num_dimensions + ? createEmptyArrayField(entry_num_dimensions - leaf_num_dimensions) + : entry->data.getLeastCommonTypeBase()->getDefault(); + + auto default_field = applyVisitor(FieldVisitorReplaceScalars(default_scalar, leaf_num_dimensions), last_field); + entry->data.insert(std::move(default_field)); + return true; +} + +PathsInData ColumnObjectDeprecated::getKeys() const +{ + PathsInData keys; + keys.reserve(subcolumns.size()); + for (const auto & entry : subcolumns) + keys.emplace_back(entry->path); + return keys; +} + +bool ColumnObjectDeprecated::isFinalized() const +{ + return std::all_of(subcolumns.begin(), subcolumns.end(), + [](const auto & entry) { return entry->data.isFinalized(); }); +} + +void ColumnObjectDeprecated::finalize() +{ + size_t old_size = size(); + Subcolumns new_subcolumns; + for (auto && entry : subcolumns) + { + const auto & least_common_type = entry->data.getLeastCommonType(); + + /// Do not add subcolumns, which consist only from NULLs. + if (isNothing(getBaseTypeOfArray(least_common_type))) + continue; + + entry->data.finalize(); + new_subcolumns.add(entry->path, entry->data); + } + + /// If all subcolumns were skipped add a dummy subcolumn, + /// because Tuple type must have at least one element. + if (new_subcolumns.empty()) + new_subcolumns.add(PathInData{COLUMN_NAME_DUMMY}, Subcolumn{ColumnUInt8::create(old_size, 0), is_nullable}); + + std::swap(subcolumns, new_subcolumns); + checkObjectHasNoAmbiguosPaths(getKeys()); +} + +} diff --git a/src/Columns/ColumnObjectDeprecated.h b/src/Columns/ColumnObjectDeprecated.h new file mode 100644 index 00000000000..b8981e91f44 --- /dev/null +++ b/src/Columns/ColumnObjectDeprecated.h @@ -0,0 +1,265 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +/// Info that represents a scalar or array field in a decomposed view. +/// It allows to recreate field with different number +/// of dimensions or nullability. +struct FieldInfo +{ + /// The common type of of all scalars in field. + DataTypePtr scalar_type; + + /// Do we have NULL scalar in field. + bool have_nulls; + + /// If true then we have scalars with different types in array and + /// we need to convert scalars to the common type. + bool need_convert; + + /// Number of dimension in array. 0 if field is scalar. + size_t num_dimensions; + + /// If true then this field is an array of variadic dimension field + /// and we need to normalize the dimension + bool need_fold_dimension; +}; + +FieldInfo getFieldInfo(const Field & field); + +/** A column that represents object with dynamic set of subcolumns. + * Subcolumns are identified by paths in document and are stored in + * a trie-like structure. ColumnObjectDeprecated is not suitable for writing into tables + * and it should be converted to Tuple with fixed set of subcolumns before that. + */ +class ColumnObjectDeprecated final : public COWHelper, ColumnObjectDeprecated> +{ +public: + /** Class that represents one subcolumn. + * It stores values in several parts of column + * and keeps current common type of all parts. + * We add a new column part with a new type, when we insert a field, + * which can't be converted to the current common type. + * After insertion of all values subcolumn should be finalized + * for writing and other operations. + */ + class Subcolumn + { + public: + Subcolumn() = default; + Subcolumn(size_t size_, bool is_nullable_); + Subcolumn(MutableColumnPtr && data_, bool is_nullable_); + + size_t size() const; + size_t byteSize() const; + size_t allocatedBytes() const; + void get(size_t n, Field & res) const; + + bool isFinalized() const; + const DataTypePtr & getLeastCommonType() const { return least_common_type.get(); } + const DataTypePtr & getLeastCommonTypeBase() const { return least_common_type.getBase(); } + size_t getNumberOfDimensions() const { return least_common_type.getNumberOfDimensions(); } + + /// Checks the consistency of column's parts stored in @data. + void checkTypes() const; + + /// Inserts a field, which scalars can be arbitrary, but number of + /// dimensions should be consistent with current common type. + void insert(Field field); + void insert(Field field, FieldInfo info); + + void insertDefault(); + void insertManyDefaults(size_t length); + void insertRangeFrom(const Subcolumn & src, size_t start, size_t length); + void popBack(size_t n); + + Subcolumn cut(size_t start, size_t length) const; + + /// Converts all column's parts to the common type and + /// creates a single column that stores all values. + void finalize(); + + /// Returns last inserted field. + Field getLastField() const; + + FieldInfo getFieldInfo() const; + + /// Recreates subcolumn with default scalar values and keeps sizes of arrays. + /// Used to create columns of type Nested with consistent array sizes. + Subcolumn recreateWithDefaultValues(const FieldInfo & field_info) const; + + /// Returns single column if subcolumn in finalizes. + /// Otherwise -- undefined behaviour. + IColumn & getFinalizedColumn(); + const IColumn & getFinalizedColumn() const; + const ColumnPtr & getFinalizedColumnPtr() const; + + const std::vector & getData() const { return data; } + size_t getNumberOfDefaultsInPrefix() const { return num_of_defaults_in_prefix; } + + friend class ColumnObjectDeprecated; + + private: + class LeastCommonType + { + public: + LeastCommonType(); + explicit LeastCommonType(DataTypePtr type_); + + const DataTypePtr & get() const { return type; } + const DataTypePtr & getBase() const { return base_type; } + size_t getNumberOfDimensions() const { return num_dimensions; } + + private: + DataTypePtr type; + DataTypePtr base_type; + size_t num_dimensions = 0; + }; + + void addNewColumnPart(DataTypePtr type); + + /// Current least common type of all values inserted to this subcolumn. + LeastCommonType least_common_type; + + /// If true then common type type of subcolumn is Nullable + /// and default values are NULLs. + bool is_nullable = false; + + /// Parts of column. Parts should be in increasing order in terms of subtypes/supertypes. + /// That means that the least common type for i-th prefix is the type of i-th part + /// and it's the supertype for all type of column from 0 to i-1. + std::vector data; + + /// Until we insert any non-default field we don't know further + /// least common type and we count number of defaults in prefix, + /// which will be converted to the default type of final common type. + size_t num_of_defaults_in_prefix = 0; + + size_t num_rows = 0; + }; + + using Subcolumns = SubcolumnsTree; + +private: + /// If true then all subcolumns are nullable. + const bool is_nullable; + + Subcolumns subcolumns; + size_t num_rows; + +public: + static constexpr auto COLUMN_NAME_DUMMY = "_dummy"; + + explicit ColumnObjectDeprecated(bool is_nullable_); + ColumnObjectDeprecated(Subcolumns && subcolumns_, bool is_nullable_); + + /// Checks that all subcolumns have consistent sizes. + void checkConsistency() const; + + bool hasSubcolumn(const PathInData & key) const; + + const Subcolumn & getSubcolumn(const PathInData & key) const; + Subcolumn & getSubcolumn(const PathInData & key); + + void incrementNumRows() { ++num_rows; } + + /// Adds a subcolumn from existing IColumn. + void addSubcolumn(const PathInData & key, MutableColumnPtr && subcolumn); + + /// Adds a subcolumn of specific size with default values. + void addSubcolumn(const PathInData & key, size_t new_size); + + /// Adds a subcolumn of type Nested of specific size with default values. + /// It cares about consistency of sizes of Nested arrays. + void addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size); + + /// Finds a subcolumn from the same Nested type as @entry and inserts + /// an array with default values with consistent sizes as in Nested type. + bool tryInsertDefaultFromNested(const Subcolumns::NodePtr & entry) const; + bool tryInsertManyDefaultsFromNested(const Subcolumns::NodePtr & entry) const; + + const Subcolumns & getSubcolumns() const { return subcolumns; } + Subcolumns & getSubcolumns() { return subcolumns; } + PathsInData getKeys() const; + + /// Part of interface + + const char * getFamilyName() const override { return "Object"; } + TypeIndex getDataType() const override { return TypeIndex::ObjectDeprecated; } + + size_t size() const override; + size_t byteSize() const override; + size_t allocatedBytes() const override; + void forEachSubcolumn(MutableColumnCallback callback) override; + void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; + void insert(const Field & field) override; + bool tryInsert(const Field & field) override; + void insertDefault() override; + void insertFrom(const IColumn & src, size_t n) override; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void popBack(size_t length) override; + Field operator[](size_t n) const override; + void get(size_t n, Field & res) const override; + + ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; + ColumnPtr replicate(const Offsets & offsets) const override; + MutableColumnPtr cloneResized(size_t new_size) const override; + + /// Finalizes all subcolumns. + void finalize() override; + bool isFinalized() const override; + + /// Order of rows in ColumnObjectDeprecated is undefined. + void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const override; + void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} + int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } + void getExtremes(Field & min, Field & max) const override; + + /// All other methods throw exception. + + StringRef getDataAt(size_t) const override { throwMustBeConcrete(); } + bool isDefaultAt(size_t) const override { throwMustBeConcrete(); } + void insertData(const char *, size_t) override { throwMustBeConcrete(); } + StringRef serializeValueIntoArena(size_t, Arena &, char const *&) const override { throwMustBeConcrete(); } + char * serializeValueIntoMemory(size_t, char *) const override { throwMustBeConcrete(); } + const char * deserializeAndInsertFromArena(const char *) override { throwMustBeConcrete(); } + const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } + void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } + void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } + void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } + void expand(const Filter &, bool) override { throwMustBeConcrete(); } + bool hasEqualValues() const override { throwMustBeConcrete(); } + size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } + double getRatioOfDefaultRows(double) const override { throwMustBeConcrete(); } + UInt64 getNumberOfDefaultRows() const override { throwMustBeConcrete(); } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwMustBeConcrete(); } + +private: + [[noreturn]] static void throwMustBeConcrete() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ColumnObjectDeprecated must be converted to ColumnTuple before use"); + } + + template + MutableColumnPtr applyForSubcolumns(Func && func) const; + + /// It's used to get shared sized of Nested to insert correct default values. + const Subcolumns::Node * getLeafOfTheSameNested(const Subcolumns::NodePtr & entry) const; +}; +} diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 90cccef2b03..bccc52d4bb2 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -11,12 +11,13 @@ #include #include #include -#include +#include #include #include #include #include #include +#include #include #include #include @@ -462,12 +463,13 @@ template class IColumnHelper; template class IColumnHelper; template class IColumnHelper; template class IColumnHelper; -template class IColumnHelper; +template class IColumnHelper; template class IColumnHelper; template class IColumnHelper; template class IColumnHelper; template class IColumnHelper; template class IColumnHelper; +template class IColumnHelper; template class IColumnHelper; diff --git a/src/Columns/tests/gtest_column_object.cpp b/src/Columns/tests/gtest_column_object.cpp new file mode 100644 index 00000000000..f6a1da64ba3 --- /dev/null +++ b/src/Columns/tests/gtest_column_object.cpp @@ -0,0 +1,351 @@ +#include +#include +#include +#include +#include + +#include +#include + +using namespace DB; + +TEST(ColumnObject, CreateEmpty) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=20, a.b UInt32, a.c Array(String))"); + auto col = type->createColumn(); + const auto & col_object = assert_cast(*col); + const auto & typed_paths = col_object.getTypedPaths(); + ASSERT_TRUE(typed_paths.contains("a.b")); + ASSERT_EQ(typed_paths.at("a.b")->getName(), "UInt32"); + ASSERT_TRUE(typed_paths.contains("a.c")); + ASSERT_EQ(typed_paths.at("a.c")->getName(), "Array(String)"); + ASSERT_TRUE(col_object.getDynamicPaths().empty()); + ASSERT_TRUE(col_object.getSharedDataOffsets().empty()); + ASSERT_TRUE(col_object.getSharedDataPathsAndValues().first->empty()); + ASSERT_TRUE(col_object.getSharedDataPathsAndValues().second->empty()); + ASSERT_EQ(col_object.getMaxDynamicTypes(), 10); + ASSERT_EQ(col_object.getMaxDynamicPaths(), 20); +} + +TEST(ColumnObject, GetName) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=20, b.d UInt32, a.b Array(String))"); + auto col = type->createColumn(); + ASSERT_EQ(col->getName(), "Object(max_dynamic_paths=20, max_dynamic_types=10, a.b Array(String), b.d UInt32)"); +} + +Field deserializeFieldFromSharedData(ColumnString * values, size_t n) +{ + auto data = values->getDataAt(n); + ReadBufferFromMemory buf(data.data, data.size); + Field res; + std::make_shared()->deserializeBinary(res, buf, FormatSettings()); + return res; +} + +TEST(ColumnObject, InsertField) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, b.d UInt32, a.b Array(String))"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + const auto & typed_paths = col_object.getTypedPaths(); + const auto & dynamic_paths = col_object.getDynamicPaths(); + const auto & shared_data_nested_column = col_object.getSharedDataNestedColumn(); + const auto & shared_data_offsets = col_object.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = col_object.getSharedDataPathsAndValues(); + Object empty_object; + col_object.insert(empty_object); + ASSERT_EQ(col_object[0], (Object{{"a.b", Array{}}, {"b.d", Field(0u)}})); + ASSERT_EQ(typed_paths.at("a.b")->size(), 1); + ASSERT_TRUE(typed_paths.at("a.b")->isDefaultAt(0)); + ASSERT_EQ(typed_paths.at("b.d")->size(), 1); + ASSERT_TRUE(typed_paths.at("b.d")->isDefaultAt(0)); + ASSERT_TRUE(dynamic_paths.empty()); + ASSERT_EQ(shared_data_nested_column.size(), 1); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(0)); + + Object object1 = {{"a.b", Array{String("Hello"), String("World")}}, {"a.c", Field(42)}}; + col_object.insert(object1); + ASSERT_EQ(col_object[1], (Object{{"a.b", Array{String("Hello"), String("World")}}, {"b.d", Field(0u)}, {"a.c", Field(42)}})); + ASSERT_EQ(typed_paths.at("a.b")->size(), 2); + ASSERT_EQ((*typed_paths.at("a.b"))[1], (Array{String("Hello"), String("World")})); + ASSERT_EQ(typed_paths.at("b.d")->size(), 2); + ASSERT_TRUE(typed_paths.at("b.d")->isDefaultAt(1)); + ASSERT_EQ(dynamic_paths.size(), 1); + ASSERT_TRUE(dynamic_paths.contains("a.c")); + ASSERT_EQ(dynamic_paths.at("a.c")->size(), 2); + ASSERT_TRUE(dynamic_paths.at("a.c")->isDefaultAt(0)); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field(42)); + ASSERT_EQ(shared_data_nested_column.size(), 2); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(1)); + + Object object2 = {{"b.d", Field(142u)}, {"a.c", Field(43)}, {"a.d", Field("str")}, {"a.e", Field(242)}, {"a.f", Array{Field(42), Field(43)}}}; + col_object.insert(object2); + ASSERT_EQ(col_object[2], (Object{{"a.b", Array{}}, {"b.d", Field(142u)}, {"a.c", Field(43)}, {"a.d", Field("str")}, {"a.e", Field(242)}, {"a.f", Array{Field(42), Field(43)}}})); + ASSERT_EQ(typed_paths.at("a.b")->size(), 3); + ASSERT_TRUE(typed_paths.at("a.b")->isDefaultAt(2)); + ASSERT_EQ(typed_paths.at("b.d")->size(), 3); + ASSERT_EQ((*typed_paths.at("b.d"))[2], Field(142u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_TRUE(dynamic_paths.contains("a.c")); + ASSERT_EQ(dynamic_paths.at("a.c")->size(), 3); + ASSERT_EQ((*dynamic_paths.at("a.c"))[2], Field(43)); + ASSERT_TRUE(dynamic_paths.contains("a.d")); + ASSERT_EQ(dynamic_paths.at("a.d")->size(), 3); + ASSERT_EQ((*dynamic_paths.at("a.d"))[2], Field("str")); + + ASSERT_EQ(shared_data_nested_column.size(), 3); + ASSERT_EQ(shared_data_offsets[2] - shared_data_offsets[1], 2); + ASSERT_EQ((*shared_data_paths)[0], "a.e"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 0), Field(242)); + ASSERT_EQ((*shared_data_paths)[1], "a.f"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 1), (Array({Field(42), Field(43)}))); + + Object object3 = {{"b.a", Field("Str")}, {"b.b", Field(2)}, {"b.c", Field(Tuple{Field(42), Field("Str")})}}; + col_object.insert(object3); + ASSERT_EQ(col_object[3], (Object{{"a.b", Array{}}, {"b.d", Field(0u)}, {"b.a", Field("Str")}, {"b.b", Field(2)}, {"b.c", Field(Tuple{Field(42), Field("Str")})}})); + ASSERT_EQ(typed_paths.at("a.b")->size(), 4); + ASSERT_TRUE(typed_paths.at("a.b")->isDefaultAt(3)); + ASSERT_EQ(typed_paths.at("b.d")->size(), 4); + ASSERT_TRUE(typed_paths.at("b.d")->isDefaultAt(3)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ(dynamic_paths.at("a.c")->size(), 4); + ASSERT_TRUE(dynamic_paths.at("a.c")->isDefaultAt(3)); + ASSERT_EQ(dynamic_paths.at("a.d")->size(), 4); + ASSERT_TRUE(dynamic_paths.at("a.d")->isDefaultAt(3)); + + ASSERT_EQ(shared_data_nested_column.size(), 4); + ASSERT_EQ(shared_data_offsets[3] - shared_data_offsets[2], 3); + ASSERT_EQ((*shared_data_paths)[2], "b.a"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 2), Field("Str")); + ASSERT_EQ((*shared_data_paths)[3], "b.b"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 3), Field(2)); + ASSERT_EQ((*shared_data_paths)[4], "b.c"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 4), Field(Tuple{Field(42), Field("Str")})); + + Object object4 = {{"c.c", Field(Null())}, {"c.d", Field(Null())}}; + col_object.insert(object4); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(4)); +} + +TEST(ColumnObject, InsertFrom) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, b.d UInt32, a.b Array(String))"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + col_object.insert(Object{{"a.a", Field(42)}}); + + const auto & typed_paths = col_object.getTypedPaths(); + const auto & dynamic_paths = col_object.getDynamicPaths(); + const auto & shared_data_nested_column = col_object.getSharedDataNestedColumn(); + const auto & shared_data_offsets = col_object.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = col_object.getSharedDataPathsAndValues(); + + auto src_col1 = type->createColumn(); + auto & src_col_object1 = assert_cast(*src_col1); + src_col_object1.insert(Object{{"b.d", Field(43u)}, {"a.c", Field("Str1")}}); + col_object.insertFrom(src_col_object1, 0); + ASSERT_EQ((*typed_paths.at("a.b"))[1], Field(Array{})); + ASSERT_EQ((*typed_paths.at("b.d"))[1], Field(43u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ((*dynamic_paths.at("a.a"))[1], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field("Str1")); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(1)); + + auto src_col2 = type->createColumn(); + auto & src_col_object2 = assert_cast(*src_col2); + src_col_object2.insert(Object{{"a.b", Array{"Str4", "Str5"}}, {"b.d", Field(44u)}, {"a.d", Field("Str2")}, {"a.e", Field("Str3")}}); + col_object.insertFrom(src_col_object2, 0); + ASSERT_EQ((*typed_paths.at("a.b"))[2], Field(Array{"Str4", "Str5"})); + ASSERT_EQ((*typed_paths.at("b.d"))[2], Field(44u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ((*dynamic_paths.at("a.a"))[2], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[2], Field(Null())); + ASSERT_EQ(shared_data_offsets[2] - shared_data_offsets[1], 2); + ASSERT_EQ((*shared_data_paths)[0], "a.d"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 0), Field("Str2")); + ASSERT_EQ((*shared_data_paths)[1], "a.e"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 1), Field("Str3")); + + auto src_col3 = type->createColumn(); + auto & src_col_object3 = assert_cast(*src_col3); + src_col_object3.insert(Object{{"a.h", Field("Str6")}, {"h.h", Field("Str7")}}); + src_col_object3.insert(Object{{"a.a", Field("Str10")}, {"a.c", Field(45u)}, {"a.h", Field("Str6")}, {"h.h", Field("Str7")}, {"a.f", Field("Str8")}, {"a.g", Field("Str9")}, {"a.i", Field("Str11")}, {"a.u", Field(Null())}}); + col_object.insertFrom(src_col_object3, 1); + ASSERT_EQ((*typed_paths.at("a.b"))[3], Field(Array{})); + ASSERT_EQ((*typed_paths.at("b.d"))[3], Field(0u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ((*dynamic_paths.at("a.a"))[3], Field("Str10")); + ASSERT_EQ((*dynamic_paths.at("a.c"))[3], Field(45u)); + ASSERT_EQ(shared_data_offsets[3] - shared_data_offsets[2], 5); + ASSERT_EQ((*shared_data_paths)[2], "a.f"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 2), Field("Str8")); + ASSERT_EQ((*shared_data_paths)[3], "a.g"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 3), Field("Str9")); + ASSERT_EQ((*shared_data_paths)[4], "a.h"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 4), Field("Str6")); + ASSERT_EQ((*shared_data_paths)[5], "a.i"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 5), Field("Str11")); + ASSERT_EQ((*shared_data_paths)[6], "h.h"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 6), Field("Str7")); +} + + +TEST(ColumnObject, InsertRangeFrom) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, b.d UInt32, a.b Array(String))"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + col_object.insert(Object{{"a.a", Field(42)}}); + + const auto & typed_paths = col_object.getTypedPaths(); + const auto & dynamic_paths = col_object.getDynamicPaths(); + const auto & shared_data_nested_column = col_object.getSharedDataNestedColumn(); + const auto & shared_data_offsets = col_object.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = col_object.getSharedDataPathsAndValues(); + + auto src_col1 = type->createColumn(); + auto & src_col_object1 = assert_cast(*src_col1); + src_col_object1.insert(Object{{"b.d", Field(43u)}, {"a.c", Field("Str1")}}); + src_col_object1.insert(Object{{"a.b", Field(Array{"Str1", "Str2"})}, {"a.a", Field("Str1")}}); + src_col_object1.insert(Object{{"b.d", Field(45u)}, {"a.c", Field("Str2")}}); + col_object.insertRangeFrom(src_col_object1, 0, 3); + ASSERT_EQ((*typed_paths.at("a.b"))[1], Field(Array{})); + ASSERT_EQ((*typed_paths.at("a.b"))[2], Field(Array{"Str1", "Str2"})); + ASSERT_EQ((*typed_paths.at("a.b"))[3], Field(Array{})); + ASSERT_EQ((*typed_paths.at("b.d"))[1], Field(43u)); + ASSERT_EQ((*typed_paths.at("b.d"))[2], Field(0u)); + ASSERT_EQ((*typed_paths.at("b.d"))[3], Field(45u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ((*dynamic_paths.at("a.a"))[1], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.a"))[2], Field("Str1")); + ASSERT_EQ((*dynamic_paths.at("a.a"))[3], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field("Str1")); + ASSERT_EQ((*dynamic_paths.at("a.c"))[2], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[3], Field("Str2")); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(1)); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(2)); + ASSERT_TRUE(shared_data_nested_column.isDefaultAt(3)); + + auto src_col2 = type->createColumn(); + auto & src_col_object2 = assert_cast(*src_col2); + src_col_object2.insert(Object{{"a.b", Array{"Str4", "Str5"}}, {"a.d", Field("Str2")}, {"a.e", Field("Str3")}}); + src_col_object2.insert(Object{{"b.d", Field(44u)}, {"a.d", Field("Str22")}, {"a.e", Field("Str33")}}); + src_col_object2.insert(Object{{"a.b", Array{"Str44", "Str55"}}, {"a.d", Field("Str222")}, {"a.e", Field("Str333")}}); + col_object.insertRangeFrom(src_col_object2, 0, 3); + ASSERT_EQ((*typed_paths.at("a.b"))[4], Field(Array{"Str4", "Str5"})); + ASSERT_EQ((*typed_paths.at("a.b"))[5], Field(Array{})); + ASSERT_EQ((*typed_paths.at("a.b"))[6], Field(Array{"Str44", "Str55"})); + ASSERT_EQ((*typed_paths.at("b.d"))[4], Field(0u)); + ASSERT_EQ((*typed_paths.at("b.d"))[5], Field(44u)); + ASSERT_EQ((*typed_paths.at("b.d"))[6], Field(0u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ((*dynamic_paths.at("a.a"))[4], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.a"))[5], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.a"))[6], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[4], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[5], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[6], Field(Null())); + ASSERT_EQ(shared_data_offsets[4] - shared_data_offsets[3], 2); + ASSERT_EQ((*shared_data_paths)[0], "a.d"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 0), Field("Str2")); + ASSERT_EQ((*shared_data_paths)[1], "a.e"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 1), Field("Str3")); + ASSERT_EQ(shared_data_offsets[5] - shared_data_offsets[4], 2); + ASSERT_EQ((*shared_data_paths)[2], "a.d"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 2), Field("Str22")); + ASSERT_EQ((*shared_data_paths)[3], "a.e"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 3), Field("Str33")); + ASSERT_EQ(shared_data_offsets[6] - shared_data_offsets[5], 2); + ASSERT_EQ((*shared_data_paths)[4], "a.d"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 4), Field("Str222")); + ASSERT_EQ((*shared_data_paths)[5], "a.e"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 5), Field("Str333")); + + auto src_col3 = type->createColumn(); + auto & src_col_object3 = assert_cast(*src_col3); + src_col_object3.insert(Object{{"a.h", Field("Str6")}, {"h.h", Field("Str7")}}); + src_col_object3.insert(Object{{"a.h", Field("Str6")}, {"h.h", Field("Str7")}, {"a.f", Field("Str8")}, {"a.g", Field("Str9")}, {"a.i", Field("Str11")}}); + src_col_object3.insert(Object{{"a.a", Field("Str10")}}); + src_col_object3.insert(Object{{"a.h", Field("Str6")}, {"a.c", Field(45u)}, {"h.h", Field("Str7")}, {"a.i", Field("Str11")}}); + col_object.insertRangeFrom(src_col_object3, 1, 3); + ASSERT_EQ((*typed_paths.at("a.b"))[7], Field(Array{})); + ASSERT_EQ((*typed_paths.at("a.b"))[8], Field(Array{})); + ASSERT_EQ((*typed_paths.at("a.b"))[9], Field(Array{})); + ASSERT_EQ((*typed_paths.at("b.d"))[7], Field(0u)); + ASSERT_EQ((*typed_paths.at("b.d"))[8], Field(0u)); + ASSERT_EQ((*typed_paths.at("b.d"))[9], Field(0u)); + ASSERT_EQ(dynamic_paths.size(), 2); + ASSERT_EQ((*dynamic_paths.at("a.a"))[7], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.a"))[8], Field("Str10")); + ASSERT_EQ((*dynamic_paths.at("a.a"))[9], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[7], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[8], Field(Null())); + ASSERT_EQ((*dynamic_paths.at("a.c"))[9], Field(45u)); + ASSERT_EQ(shared_data_offsets[7] - shared_data_offsets[6], 5); + ASSERT_EQ((*shared_data_paths)[6], "a.f"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 6), Field("Str8")); + ASSERT_EQ((*shared_data_paths)[7], "a.g"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 7), Field("Str9")); + ASSERT_EQ((*shared_data_paths)[8], "a.h"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 8), Field("Str6")); + ASSERT_EQ((*shared_data_paths)[9], "a.i"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 9), Field("Str11")); + ASSERT_EQ((*shared_data_paths)[10], "h.h"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 10), Field("Str7")); + ASSERT_EQ(shared_data_offsets[8] - shared_data_offsets[7], 0); + ASSERT_EQ(shared_data_offsets[9] - shared_data_offsets[8], 3); + ASSERT_EQ((*shared_data_paths)[11], "a.h"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 11), Field("Str6")); + ASSERT_EQ((*shared_data_paths)[12], "a.i"); + ASSERT_EQ(deserializeFieldFromSharedData(shared_data_values, 12), Field("Str11")); +} + +TEST(ColumnObject, SerializeDeserializerFromArena) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, b.d UInt32, a.b Array(String))"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + col_object.insert(Object{{"b.d", Field(42u)}, {"a.b", Array{"Str1", "Str2"}}, {"a.a", Tuple{"Str3", 441u}}, {"a.c", Field("Str4")}, {"a.d", Array{Field(45), Field(46)}}, {"a.e", Field(47)}}); + col_object.insert(Object{{"b.a", Field(48)}, {"b.b", Array{Field(49), Field(50)}}}); + col_object.insert(Object{{"b.d", Field(442u)}, {"a.b", Array{"Str11", "Str22"}}, {"a.a", Tuple{"Str33", 444u}}, {"a.c", Field("Str44")}, {"a.d", Array{Field(445), Field(446)}}, {"a.e", Field(447)}}); + + Arena arena; + const char * pos = nullptr; + auto ref1 = col_object.serializeValueIntoArena(0, arena, pos); + col_object.serializeValueIntoArena(1, arena, pos); + col_object.serializeValueIntoArena(2, arena, pos); + + auto col2 = type->createColumn(); + auto & col_object2 = assert_cast(*col); + pos = col_object2.deserializeAndInsertFromArena(ref1.data); + pos = col_object2.deserializeAndInsertFromArena(pos); + col_object2.deserializeAndInsertFromArena(pos); + + ASSERT_EQ(col_object2[0], (Object{{"b.d", Field(42u)}, {"a.b", Array{"Str1", "Str2"}}, {"a.a", Tuple{"Str3", 441u}}, {"a.c", Field("Str4")}, {"a.d", Array{Field(45), Field(46)}}, {"a.e", Field(47)}})); + ASSERT_EQ(col_object2[1], (Object{{"b.d", Field{0u}}, {"a.b", Array{}}, {"b.a", Field(48)}, {"b.b", Array{Field(49), Field(50)}}})); + ASSERT_EQ(col_object2[2], (Object{{"b.d", Field(442u)}, {"a.b", Array{"Str11", "Str22"}}, {"a.a", Tuple{"Str33", 444u}}, {"a.c", Field("Str44")}, {"a.d", Array{Field(445), Field(446)}}, {"a.e", Field(447)}})); +} + +TEST(ColumnObject, SkipSerializedInArena) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, b.d UInt32, a.b Array(String))"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + col_object.insert(Object{{"b.d", Field(42u)}, {"a.b", Array{"Str1", "Str2"}}, {"a.a", Tuple{"Str3", 441u}}, {"a.c", Field("Str4")}, {"a.d", Array{Field(45), Field(46)}}, {"a.e", Field(47)}}); + col_object.insert(Object{{"b.a", Field(48)}, {"b.b", Array{Field(49), Field(50)}}}); + col_object.insert(Object{{"b.d", Field(442u)}, {"a.b", Array{"Str11", "Str22"}}, {"a.a", Tuple{"Str33", 444u}}, {"a.c", Field("Str44")}, {"a.d", Array{Field(445), Field(446)}}, {"a.e", Field(447)}}); + + Arena arena; + const char * pos = nullptr; + auto ref1 = col_object.serializeValueIntoArena(0, arena, pos); + col_object.serializeValueIntoArena(1, arena, pos); + auto ref3 = col_object.serializeValueIntoArena(2, arena, pos); + + const char * end = ref3.data + ref3.size; + auto col2 = type->createColumn(); + pos = col2->skipSerializedInArena(ref1.data); + pos = col2->skipSerializedInArena(pos); + pos = col2->skipSerializedInArena(pos); + ASSERT_EQ(pos, end); +} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e835aa3a4f2..2e808159926 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -871,6 +871,7 @@ class IColumn; M(Bool, allow_get_client_http_header, false, "Allow to use the function `getClientHTTPHeader` which lets to obtain a value of an the current HTTP request's header. It is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. Note that the `X-ClickHouse-*` and `Authentication` headers are always restricted and cannot be obtained with this function.", 0) \ M(Bool, cast_string_to_dynamic_use_inference, false, "Use types inference during String to Dynamic conversion", 0) \ M(Bool, enable_blob_storage_log, true, "Write information about blob storage operations to system.blob_storage_log table", 0) \ + M(Bool, use_json_alias_for_old_object_type, false, "When enabled, JSON type alias will create old experimental Object type instead of a new JSON type", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ @@ -900,6 +901,7 @@ class IColumn; M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ + M(Bool, allow_experimental_json_type, false, "Allow JSON data type", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ @@ -1105,7 +1107,7 @@ class IColumn; M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ - M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ + M(Bool, input_format_json_try_infer_numbers_from_strings, true, "Try to infer numbers from string fields while schema inference", 0) \ M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \ M(Bool, input_format_json_read_numbers_as_strings, true, "Allow to parse numbers as strings in JSON input formats", 0) \ M(Bool, input_format_json_read_objects_as_strings, true, "Allow to parse JSON objects as strings in JSON input formats", 0) \ @@ -1119,6 +1121,7 @@ class IColumn; M(Bool, input_format_json_throw_on_bad_escape_sequence, true, "Throw an exception if JSON string contains bad escape sequence in JSON input formats. If disabled, bad escape sequences will remain as is in the data", 0) \ M(Bool, input_format_json_ignore_unnecessary_fields, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields", 0) \ M(Bool, input_format_json_ignore_key_case, false, "Ignore json key case while read json field from string", 0) \ + M(Bool, type_json_skip_duplicated_paths, false, "When enabled, during parsing JSON object into JSON type duplicated paths will be ignored and only the first one will be inserted instead of an exception", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index df681425355..7abec285b37 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -75,7 +75,10 @@ static std::initializer_list #include #include +#include namespace DB { @@ -65,7 +66,11 @@ static DataTypePtr create(const ASTPtr & arguments) if (!argument || argument->name != "equals") throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Dynamic data type argument should be in a form 'max_types=N'"); - auto identifier_name = argument->arguments->children[0]->as()->name(); + const auto * identifier = argument->arguments->children[0]->as(); + if (!identifier) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected Dynamic type argument: {}. Expected expression 'max_types=N'", identifier->formatForErrorMessage()); + + auto identifier_name = identifier->name(); if (identifier_name != "max_types") throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected identifier: {}. Dynamic data type argument should be in a form 'max_types=N'", identifier_name); @@ -82,9 +87,51 @@ void registerDataTypeDynamic(DataTypeFactory & factory) factory.registerDataType("Dynamic", create); } +namespace +{ + +std::pair splitSubcolumnName(std::string_view subcolumn_name) +{ + bool inside_quotes = false; + const char * pos = subcolumn_name.data(); + const char * end = subcolumn_name.data() + subcolumn_name.size(); + while (true) + { + pos = find_first_symbols<'`', '.', '\\'>(pos, end); + if (pos == end) + break; + + if (*pos == '`') + { + inside_quotes = !inside_quotes; + ++pos; + } + else if (*pos == '\\') + { + ++pos; + } + else if (*pos == '.') + { + if (inside_quotes) + ++pos; + else + break; + } + } + + if (pos == end) + return {subcolumn_name, {}}; + + return {std::string_view(subcolumn_name.data(), pos), std::string_view(pos + 1, end)}; +} + +} + std::unique_ptr DataTypeDynamic::getDynamicSubcolumnData(std::string_view subcolumn_name, const DB::IDataType::SubstreamData & data, bool throw_if_null) const { - auto [subcolumn_type_name, subcolumn_nested_name] = Nested::splitName(subcolumn_name); + auto [subcolumn_type_name, subcolumn_nested_name] = splitSubcolumnName(subcolumn_name); +// std::cerr << "Dynamic subcolumn: " << subcolumn_name << ", " << subcolumn_nested_name << "\n"; + /// Check if requested subcolumn is a valid data type. auto subcolumn_type = DataTypeFactory::instance().tryGet(String(subcolumn_type_name)); if (!subcolumn_type) diff --git a/src/DataTypes/DataTypeDynamic.h b/src/DataTypes/DataTypeDynamic.h index d5e4c5261ce..2e7a23d314d 100644 --- a/src/DataTypes/DataTypeDynamic.h +++ b/src/DataTypes/DataTypeDynamic.h @@ -12,6 +12,9 @@ class DataTypeDynamic final : public IDataType public: static constexpr bool is_parametric = true; + /// Don't change this constant, it can break backward compatibility. + static constexpr size_t DEFAULT_MAX_DYNAMIC_TYPES = 32; + explicit DataTypeDynamic(size_t max_dynamic_types_ = DEFAULT_MAX_DYNAMIC_TYPES); TypeIndex getTypeId() const override { return TypeIndex::Dynamic; } @@ -43,8 +46,6 @@ public: size_t getMaxDynamicTypes() const { return max_dynamic_types; } private: - static constexpr size_t DEFAULT_MAX_DYNAMIC_TYPES = 32; - SerializationPtr doGetDefaultSerialization() const override; String doGetName() const override; diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 8c8f9999ada..6488e2d022b 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -290,9 +290,10 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainSimpleAggregateFunction(*this); registerDataTypeDomainGeo(*this); registerDataTypeMap(*this); - registerDataTypeObject(*this); + registerDataTypeObjectDeprecated(*this); registerDataTypeVariant(*this); registerDataTypeDynamic(*this); + registerDataTypeJSON(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 86e0203358d..ec0586f2f3c 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -98,8 +98,9 @@ void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainBool(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); -void registerDataTypeObject(DataTypeFactory & factory); +void registerDataTypeObjectDeprecated(DataTypeFactory & factory); void registerDataTypeVariant(DataTypeFactory & factory); void registerDataTypeDynamic(DataTypeFactory & factory); +void registerDataTypeJSON(DataTypeFactory & factory); } diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 720436d0e0d..338883c3ae4 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -1,11 +1,30 @@ -#include #include -#include +#include +#include +#include +#include +#include +#include #include #include #include +#include +#include +#include +#include +#include + +#if USE_SIMDJSON +#include +#endif +#if USE_RAPIDJSON +#include +#endif +#include + #include +#include namespace DB { @@ -14,69 +33,465 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int BAD_ARGUMENTS; } -DataTypeObject::DataTypeObject(const String & schema_format_, bool is_nullable_) - : schema_format(Poco::toLower(schema_format_)) - , is_nullable(is_nullable_) +DataTypeObject::DataTypeObject( + const SchemaFormat & schema_format_, + const std::unordered_map & typed_paths_, + const std::unordered_set & paths_to_skip_, + const std::vector & path_prefixes_to_skip_, + const std::vector & path_regexps_to_skip_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_) + : schema_format(schema_format_) + , typed_paths(typed_paths_) + , paths_to_skip(paths_to_skip_) + , path_prefixes_to_skip(path_prefixes_to_skip_) + , path_regexps_to_skip(path_regexps_to_skip_) + , max_dynamic_paths(max_dynamic_paths_) + , max_dynamic_types(max_dynamic_types_) +{ + for (const auto & path : paths_to_skip) + { + if (typed_paths.contains(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified both with the data type ('{}') and in the SKIP section", path, typed_paths[path]->getName()); + } +} + +DataTypeObject::DataTypeObject(const DB::DataTypeObject::SchemaFormat & schema_format_, size_t max_dynamic_paths_, size_t max_dynamic_types_) + : schema_format(schema_format_) + , max_dynamic_paths(max_dynamic_paths_) + , max_dynamic_types(max_dynamic_types_) { } bool DataTypeObject::equals(const IDataType & rhs) const { if (const auto * object = typeid_cast(&rhs)) - return schema_format == object->schema_format && is_nullable == object->is_nullable; + { + if (typed_paths.size() != object->typed_paths.size()) + return false; + + for (const auto & [path, type] : typed_paths) + { + auto it = object->typed_paths.find(path); + if (it == object->typed_paths.end()) + return false; + if (!type->equals(*it->second)) + return false; + } + + return schema_format == object->schema_format && paths_to_skip == object->paths_to_skip && path_prefixes_to_skip == object->path_prefixes_to_skip && path_regexps_to_skip == object->path_regexps_to_skip + && max_dynamic_types == object->max_dynamic_types && max_dynamic_paths == object->max_dynamic_paths; + } + return false; } SerializationPtr DataTypeObject::doGetDefaultSerialization() const { - return getObjectSerialization(schema_format); + std::unordered_map typed_path_serializations; + typed_path_serializations.reserve(typed_paths.size()); + for (const auto & [path, type] : typed_paths) + typed_path_serializations[path] = type->getDefaultSerialization(); + + switch (schema_format) + { + case SchemaFormat::JSON: +#ifdef USE_SIMDJSON + return std::make_shared>( + std::move(typed_path_serializations), + paths_to_skip, + path_prefixes_to_skip, + path_regexps_to_skip, + buildJSONExtractTree(getPtr(), "JSON serialization")); +#elif USE_RAPIDJSON + return std::make_shared>(std::move(typed_path_serializations), paths_to_skip, buildJSONExtractTree(getPtr(), "JSON serialization")); +#else + return std::make_shared>(std::move(typed_path_serializations), paths_to_skip, buildJSONExtractTree(getPtr(), "JSON serialization")); +#endif + } } String DataTypeObject::doGetName() const { WriteBufferFromOwnString out; - if (is_nullable) - out << "Object(Nullable(" << quote << schema_format << "))"; - else - out << "Object(" << quote << schema_format << ")"; + out << magic_enum::enum_name(schema_format); + bool first = true; + auto write_separator = [&]() + { + if (!first) + { + out << ", "; + } + else + { + out << "("; + first = false; + } + }; + + if (max_dynamic_types != DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES) + { + write_separator(); + out << "max_dynamic_types=" << max_dynamic_types; + } + + if (max_dynamic_paths != DEFAULT_MAX_SEPARATELY_STORED_PATHS) + { + write_separator(); + out << "max_dynamic_paths=" << max_dynamic_paths; + } + + std::vector sorted_typed_paths; + sorted_typed_paths.reserve(typed_paths.size()); + for (const auto & [path, _] : typed_paths) + sorted_typed_paths.push_back(path); + std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); + for (const auto & path : sorted_typed_paths) + { + write_separator(); + out << path << " " << typed_paths.at(path)->getName(); + } + + std::vector sorted_skip_paths; + sorted_skip_paths.reserve(paths_to_skip.size()); + for (const auto & skip_path : paths_to_skip) + sorted_skip_paths.push_back(skip_path); + std::sort(sorted_skip_paths.begin(), sorted_skip_paths.end()); + for (const auto & skip_path : sorted_skip_paths) + { + write_separator(); + out << "SKIP " << skip_path; + } + + for (const auto & skip_prefix : path_prefixes_to_skip) + { + write_separator(); + out << "SKIP PREFIX " << skip_prefix; + } + + for (const auto & skip_regexp : path_regexps_to_skip) + { + write_separator(); + out << "SKIP REGEXP " << skip_regexp; + } + + if (!first) + out << ")"; + return out.str(); } -static DataTypePtr create(const ASTPtr & arguments) +MutableColumnPtr DataTypeObject::createColumn() const { - if (!arguments || arguments->children.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Object data type family must have one argument - name of schema format"); + std::unordered_map typed_path_columns; + typed_path_columns.reserve(typed_paths.size()); + for (const auto & [path, type] : typed_paths) + typed_path_columns[path] = type->createColumn(); - ASTPtr schema_argument = arguments->children[0]; - bool is_nullable = false; + return ColumnObject::create(std::move(typed_path_columns), max_dynamic_paths, max_dynamic_types); +} - if (const auto * func = schema_argument->as()) +namespace +{ + +/// JSON subcolumn name with Dynamic type subcolumn looks like this: +/// "json.some.path.:`Type_name`.some.subcolumn". +/// We back quoted type name during identifier parsing so we can distinguish type subcolumn and path element ":TypeName". +std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subcolumn_name) +{ + /// Try to find dynamic type subcolumn in a form .:`Type`. + auto pos = subcolumn_name.find(".:`"); + if (pos == std::string_view::npos) + return {String(subcolumn_name), ""}; + + ReadBufferFromMemory buf(subcolumn_name.substr(pos + 2)); + String dynamic_subcolumn; + /// Try to read back quoted type name. + if (!tryReadBackQuotedString(dynamic_subcolumn, buf)) + return {String(subcolumn_name), ""}; + + /// If there is more data in the buffer - it's subcolumn of a type, append it to the type name. + if (!buf.eof()) + dynamic_subcolumn += String(buf.position(), buf.available()); + + return {String(subcolumn_name.substr(0, pos)), dynamic_subcolumn}; +} + +/// Sub-object subcolumn in JSON path always looks like "^`some`.path.path". +/// We back quote first path element after `^` so we can distinguish sub-object subcolumn and path element "^path". +std::optional tryGetSubObjectSubcolumn(std::string_view subcolumn_name) +{ + if (!subcolumn_name.starts_with("^`")) + return std::nullopt; + + ReadBufferFromMemory buf(subcolumn_name.data() + 1); + String path; + /// Try to read back-quoted first path element. + if (!tryReadBackQuotedString(path, buf)) + return std::nullopt; + + /// Add remaining path elements if any. + return path + String(buf.position(), buf.available()); +} + +/// Return sub-path by specified prefix. +/// For example, for prefix a.b: +/// a.b.c.d -> c.d, a.b.c -> c +String getSubPath(const String & path, const String & prefix) +{ + return path.substr(prefix.size() + 1); +} + +std::string_view getSubPath(const std::string_view & path, const String & prefix) +{ + return path.substr(prefix.size() + 1); +} + +} + +std::unique_ptr DataTypeObject::getDynamicSubcolumnData(std::string_view subcolumn_name, const SubstreamData & data, bool throw_if_null) const +{ + /// Check if it's sub-object subcolumn. + /// In this case we should return JSON column with all paths that are inside specified object prefix. + /// For example, if we have {"a" : {"b" : {"c" : {"d" : 10, "e" : "Hello"}, "f" : [1, 2, 3]}}} and subcolumn ^a.b + /// we should return JSON column with data {"c" : {"d" : 10, "e" : Hello}, "f" : [1, 2, 3]} + if (auto sub_object_subcolumn = tryGetSubObjectSubcolumn(subcolumn_name)) { - if (func->name != "Nullable" || func->arguments->children.size() != 1) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, - "Expected 'Nullable()' as parameter for type Object (function: {})", func->name); + const String & prefix = *sub_object_subcolumn; + /// Collect new typed paths. + std::unordered_map typed_sub_paths; + /// Collect serializations for typed paths. They will be needed for sub-object subcolumn deserialization. + std::unordered_map typed_paths_serializations; + for (const auto & [path, type] : typed_paths) + { + if (path.starts_with(prefix) && path.size() != prefix.size()) + { + typed_sub_paths[getSubPath(path, prefix)] = type; + typed_paths_serializations[path] = type->getDefaultSerialization(); + } + } - schema_argument = func->arguments->children[0]; - is_nullable = true; + std::unique_ptr res = std::make_unique(std::make_shared(prefix, typed_paths_serializations)); + /// Keep all current constrains like limits and skip paths/prefixes/regexps. + res->type = std::make_shared(schema_format, typed_sub_paths, paths_to_skip, path_prefixes_to_skip, path_prefixes_to_skip, max_dynamic_paths, max_dynamic_types); + /// If column was provided, we should create a column for the requested subcolumn. + if (data.column) + { + const auto & object_column = assert_cast(*data.column); + + auto result_column = res->type->createColumn(); + auto & result_object_column = assert_cast(*result_column); + + /// Iterate over all typed/dynamic/shared data paths and collect all paths with specified prefix. + auto & result_typed_columns = result_object_column.getTypedPaths(); + for (const auto & [path, column] : object_column.getTypedPaths()) + { + if (path.starts_with(prefix) && path.size() != prefix.size()) + result_typed_columns[getSubPath(path, prefix)] = column; + } + + auto & result_dynamic_columns = result_object_column.getDynamicPaths(); + for (const auto & [path, column] : object_column.getDynamicPaths()) + { + if (path.starts_with(prefix) && path.size() != prefix.size()) + result_dynamic_columns[getSubPath(path, prefix)] = column; + } + + const auto & shared_data_offsets = object_column.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = object_column.getSharedDataPathsAndValues(); + auto & result_shared_data_offsets = result_object_column.getSharedDataOffsets(); + result_shared_data_offsets.reserve(shared_data_offsets.size()); + auto [result_shared_data_paths, result_shared_data_values] = result_object_column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_offsets.size(); ++i) + { + size_t start = shared_data_offsets[ssize_t(i) - 1]; + size_t end = shared_data_offsets[ssize_t(i)]; + size_t lower_bound_index = ColumnObject::findPathLowerBoundInSharedData(prefix, *shared_data_paths, start, end); + for (; lower_bound_index != end; ++lower_bound_index) + { + auto path = shared_data_paths->getDataAt(lower_bound_index).toView(); + if (!path.starts_with(prefix)) + break; + + /// Don't include path that is equal to the prefix. + if (path.size() != prefix.size()) + { + auto sub_path = getSubPath(path, prefix); + result_shared_data_paths->insertData(sub_path.data(), sub_path.size()); + result_shared_data_values->insertFrom(*shared_data_values, lower_bound_index); + } + } + result_shared_data_offsets.push_back(result_shared_data_paths->size()); + } + + res->column = std::move(result_column); + } + + return res; } - const auto * literal = schema_argument->as(); - if (!literal || literal->value.getType() != Field::Types::String) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, - "Object data type family must have a const string as its schema name parameter"); + /// Split requested subcolumn to the JSON path and Dynamic type subcolumn. + auto [path, path_subcolumn] = splitPathAndDynamicTypeSubcolumn(subcolumn_name); + std::unique_ptr res; + if (auto it = typed_paths.find(path); it != typed_paths.end()) + { + res = std::make_unique(it->second->getDefaultSerialization()); + res->type = it->second; + } + else + { + res = std::make_unique(std::make_shared()); + res->type = std::make_shared(); + } - return std::make_shared(literal->value.get(), is_nullable); + /// If column was provided, we should create a column for requested subcolumn. + if (data.column) + { + const auto & object_column = assert_cast(*data.column); + /// Try to find requested path in typed paths. + if (auto typed_it = object_column.getTypedPaths().find(path); typed_it != object_column.getTypedPaths().end()) + { + res->column = typed_it->second; + } + /// Try to find requested path in dynamic paths. + else if (auto dynamic_it = object_column.getDynamicPaths().find(path); dynamic_it != object_column.getDynamicPaths().end()) + { + res->column = dynamic_it->second; + } + /// Extract values of requested path from shared data. + else + { + auto dynamic_column = ColumnDynamic::create(max_dynamic_types); + dynamic_column->reserve(object_column.size()); + ColumnObject::fillPathColumnFromSharedData(*dynamic_column, path, object_column.getSharedDataPtr(), 0, object_column.size()); + res->column = std::move(dynamic_column); + } + } + + /// Get subcolumn for Dynamic type if needed. + if (!path_subcolumn.empty()) + { + /// It is possible to have nested JSON object inside Dynamic. For example when we have an array of JSON objects. + /// During parsing in case of creating nested JSON objects, we reduce max_dynamic_paths/max_dynamic_types by NESTED_OBJECT_REDUCE_FACTOR factor. + /// So the type name will actually be JSON(max_dynamic_paths=N, max_dynamic_types=M). But we want the user to be able to query it + /// using json.array.:`Array(JSON)`.some.path without specifying max_dynamic_paths/max_dynamic_types. + /// To support it, we do a trick - we replace JSON name in subcolumn to JSON(max_dynamic_paths=N, max_dynamic_types=M), because we know + /// the exact values of max_dynamic_paths/max_dynamic_types for it. + auto pos = path_subcolumn.find("JSON"); + /// We want to replace JSON keyword only in the first subcolumn part before the first dot. + auto first_dot_pos = path_subcolumn.find('.'); + if (pos != path_subcolumn.npos && (first_dot_pos == path_subcolumn.npos || pos < first_dot_pos)) + path_subcolumn.replace(pos, 4, fmt::format("JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); + + res = res->type->getSubcolumnData(path_subcolumn, *res, throw_if_null); + if (!res) + return nullptr; + } + + if (typed_paths.contains(path)) + res->serialization = std::make_shared(res->serialization, path); + else + res->serialization = std::make_shared(res->serialization, path, path_subcolumn, max_dynamic_types); + + return res; } -void registerDataTypeObject(DataTypeFactory & factory) +static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject::SchemaFormat & schema_format) { - factory.registerDataType("Object", create); - factory.registerSimpleDataType("JSON", - [] { return std::make_shared("JSON", false); }, - DataTypeFactory::CaseInsensitive); + if (!arguments || arguments->children.empty()) + return std::make_shared(schema_format); + + std::unordered_map typed_paths; + std::unordered_set paths_to_skip; + /// Collect prefixes in unordered_set to avoid duplicate prefixes + std::unordered_set path_prefixes_to_skip_set; + std::vector path_regexps_to_skip; + + size_t max_dynamic_types = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES; + size_t max_dynamic_paths = DataTypeObject::DEFAULT_MAX_SEPARATELY_STORED_PATHS; + + for (const auto & argument : arguments->children) + { + const auto * object_type_argument = argument->as(); + if (object_type_argument->parameter) + { + const auto * function = object_type_argument->parameter->as(); + + if (!function || function->name != "equals") + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected parameter in {} type arguments: {}", magic_enum::enum_name(schema_format), function->formatForErrorMessage()); + + const auto * identifier = function->arguments->children[0]->as(); + if (!identifier) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected {} type argument: {}. Expected expression 'max_dynamic_types=N' or 'max_dynamic_paths=N'", magic_enum::enum_name(schema_format), function->formatForErrorMessage()); + + auto identifier_name = identifier->name(); + if (identifier_name != "max_dynamic_types" && identifier_name != "max_dynamic_paths") + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected parameter in {} type arguments: {}. Expected 'max_dynamic_types' or `max_dynamic_paths`", magic_enum::enum_name(schema_format), identifier_name); + + auto * literal = function->arguments->children[1]->as(); + /// Is 1000000 a good maximum for max paths? + size_t min_value = identifier_name == "max_dynamic_types" ? 1 : 0; + size_t max_value = identifier_name == "max_dynamic_types" ? 255 : 1000000; + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() < min_value || literal->value.get() > max_value) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'{}' parameter for {} type should be a positive integer between {} and {}. Got {}", identifier_name, magic_enum::enum_name(schema_format), min_value, max_value, function->arguments->children[1]->formatForErrorMessage()); + + if (identifier_name == "max_dynamic_types") + max_dynamic_types = literal->value.get(); + else + max_dynamic_paths = literal->value.get(); + } + else if (object_type_argument->path_with_type) + { + const auto * path_with_type = object_type_argument->path_with_type->as(); + auto data_type = DataTypeFactory::instance().get(path_with_type->type); + if (typed_paths.contains(path_with_type->name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Found duplicated path with type: {}", path_with_type->name); + typed_paths.emplace(path_with_type->name, data_type); + } + else if (object_type_argument->skip_path) + { + const auto * identifier = object_type_argument->skip_path->as(); + if (!identifier) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST in SKIP section of {} type arguments: {}. Expected identifier with path name", magic_enum::enum_name(schema_format), object_type_argument->skip_path->formatForErrorMessage()); + + paths_to_skip.insert(identifier->name()); + } + else if (object_type_argument->skip_path_prefix) + { + const auto * identifier = object_type_argument->skip_path_prefix->as(); + if (!identifier) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST in SKIP section of {} type arguments: {}. Expected identifier with path name", magic_enum::enum_name(schema_format), object_type_argument->skip_path->formatForErrorMessage()); + + path_prefixes_to_skip_set.insert(identifier->name()); + } + else if (object_type_argument->skip_path_regexp) + { + const auto * literal = object_type_argument->skip_path_regexp->as(); + if (!literal || literal->value.getType() != Field::Types::String) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST in SKIP section of {} type arguments: {}. Expected identifier with path name", magic_enum::enum_name(schema_format), object_type_argument->skip_path->formatForErrorMessage()); + + path_regexps_to_skip.push_back(literal->value.get()); + } + } + + std::vector path_prefixes_to_skip(path_prefixes_to_skip_set.begin(), path_prefixes_to_skip_set.end()); + std::sort(path_prefixes_to_skip.begin(), path_prefixes_to_skip.end()); + std::sort(path_regexps_to_skip.begin(), path_regexps_to_skip.end()); + return std::make_shared(schema_format, typed_paths, paths_to_skip, path_prefixes_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); +} + +static DataTypePtr createJSON(const ASTPtr & arguments) +{ + return createObject(arguments, DataTypeObject::SchemaFormat::JSON); +} + +void registerDataTypeJSON(DataTypeFactory & factory) +{ + if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) + factory.registerDataType("JSON", createJSON); } } diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index c610a1a8ba4..1a66222cc98 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include -#include +#include +#include namespace DB @@ -15,34 +17,70 @@ namespace ErrorCodes class DataTypeObject : public IDataType { -private: - String schema_format; - bool is_nullable; - public: - DataTypeObject(const String & schema_format_, bool is_nullable_); + enum class SchemaFormat + { + JSON = 0, + }; + + /// Don't change these constants, it can break backward compatibility. + static constexpr size_t DEFAULT_MAX_SEPARATELY_STORED_PATHS = 1000; + static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR = 4; + static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR = 2; + + DataTypeObject( + const SchemaFormat & schema_format_, + const std::unordered_map & typed_paths_ = {}, + const std::unordered_set & paths_to_skip_ = {}, + const std::vector & path_prefixes_to_skip_ = {}, + const std::vector & path_regexps_to_skip_ = {}, + size_t max_dynamic_paths_ = DEFAULT_MAX_SEPARATELY_STORED_PATHS, + size_t max_dynamic_types_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES); + + DataTypeObject(const SchemaFormat & schema_format_, size_t max_dynamic_paths_, size_t max_dynamic_types_); const char * getFamilyName() const override { return "Object"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::Object; } - MutableColumnPtr createColumn() const override { return ColumnObject::create(is_nullable); } + MutableColumnPtr createColumn() const override; Field getDefault() const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDefault() is not implemented for data type {}", getName()); } - bool haveSubtypes() const override { return false; } - bool equals(const IDataType & rhs) const override; bool isParametric() const override { return true; } - bool hasDynamicSubcolumnsDeprecated() const override { return true; } + bool canBeInsideNullable() const override { return false; } + bool supportsSparseSerialization() const override { return false; } + bool canBeInsideSparseColumns() const override { return false; } + bool isComparable() const override { return false; } + bool haveSubtypes() const override { return false; } + + bool equals(const IDataType & rhs) const override; + + bool hasDynamicSubcolumnsData() const override { return true; } + std::unique_ptr getDynamicSubcolumnData(std::string_view subcolumn_name, const SubstreamData & data, bool throw_if_null) const override; SerializationPtr doGetDefaultSerialization() const override; - bool hasNullableSubcolumns() const { return is_nullable; } + const SchemaFormat & getSchemaFormat() const { return schema_format; } + const std::unordered_map & getTypedPaths() const { return typed_paths; } + const std::unordered_set & getPathsToSkip() const { return paths_to_skip; } + const std::vector & getPathPrefixesToSkip() const { return path_prefixes_to_skip; } + const std::vector & getPathRegexpsToSkip() const { return path_regexps_to_skip; } - const String & getSchemaFormat() const { return schema_format; } + size_t getMaxDynamicTypes() const { return max_dynamic_types; } + size_t getMaxDynamicPaths() const { return max_dynamic_paths; } + +private: + SchemaFormat schema_format; + std::unordered_map typed_paths; + std::unordered_set paths_to_skip; + std::vector path_prefixes_to_skip; + std::vector path_regexps_to_skip; + size_t max_dynamic_paths; + size_t max_dynamic_types; }; } diff --git a/src/DataTypes/DataTypeObjectDeprecated.cpp b/src/DataTypes/DataTypeObjectDeprecated.cpp new file mode 100644 index 00000000000..8b2b5e72f6c --- /dev/null +++ b/src/DataTypes/DataTypeObjectDeprecated.cpp @@ -0,0 +1,85 @@ +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int UNEXPECTED_AST_STRUCTURE; +} + +DataTypeObjectDeprecated::DataTypeObjectDeprecated(const String & schema_format_, bool is_nullable_) + : schema_format(Poco::toLower(schema_format_)) + , is_nullable(is_nullable_) +{ +} + +bool DataTypeObjectDeprecated::equals(const IDataType & rhs) const +{ + if (const auto * object = typeid_cast(&rhs)) + return schema_format == object->schema_format && is_nullable == object->is_nullable; + return false; +} + +SerializationPtr DataTypeObjectDeprecated::doGetDefaultSerialization() const +{ + return getObjectSerialization(schema_format); +} + +String DataTypeObjectDeprecated::doGetName() const +{ + WriteBufferFromOwnString out; + if (is_nullable) + out << "Object(Nullable(" << quote << schema_format << "))"; + else + out << "Object(" << quote << schema_format << ")"; + return out.str(); +} + +static DataTypePtr create(const ASTPtr & arguments) +{ + if (!arguments || arguments->children.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Object data type family must have one argument - name of schema format"); + + ASTPtr schema_argument = arguments->children[0]; + bool is_nullable = false; + + if (const auto * func = schema_argument->as()) + { + if (func->name != "Nullable" || func->arguments->children.size() != 1) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, + "Expected 'Nullable()' as parameter for type Object (function: {})", func->name); + + schema_argument = func->arguments->children[0]; + is_nullable = true; + } + + const auto * literal = schema_argument->as(); + if (!literal || literal->value.getType() != Field::Types::String) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, + "Object data type family must have a const string as its schema name parameter"); + + return std::make_shared(literal->value.get(), is_nullable); +} + +void registerDataTypeObjectDeprecated(DataTypeFactory & factory) +{ + factory.registerDataType("Object", create); + if (Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) + factory.registerSimpleDataType("JSON", + [] { return std::make_shared("JSON", false); }, + DataTypeFactory::CaseInsensitive); +} + +} diff --git a/src/DataTypes/DataTypeObjectDeprecated.h b/src/DataTypes/DataTypeObjectDeprecated.h new file mode 100644 index 00000000000..e1f81caaa4f --- /dev/null +++ b/src/DataTypes/DataTypeObjectDeprecated.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class DataTypeObjectDeprecated : public IDataType +{ +private: + String schema_format; + bool is_nullable; + +public: + DataTypeObjectDeprecated(const String & schema_format_, bool is_nullable_); + + const char * getFamilyName() const override { return "Object"; } + String doGetName() const override; + TypeIndex getTypeId() const override { return TypeIndex::ObjectDeprecated; } + + MutableColumnPtr createColumn() const override { return ColumnObjectDeprecated::create(is_nullable); } + + Field getDefault() const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDefault() is not implemented for data type {}", getName()); + } + + bool haveSubtypes() const override { return false; } + bool equals(const IDataType & rhs) const override; + bool isParametric() const override { return true; } + bool hasDynamicSubcolumnsDeprecated() const override { return true; } + + SerializationPtr doGetDefaultSerialization() const override; + + bool hasNullableSubcolumns() const { return is_nullable; } + + const String & getSchemaFormat() const { return schema_format; } +}; + +} diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index e96937d522d..75556ed4090 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -192,17 +192,12 @@ MutableColumnPtr DataTypeTuple::createColumn() const MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serialization) const { - /// If we read Tuple as Variant subcolumn, it may be wrapped to SerializationVariantElement. - /// Here we don't need it, so we drop this wrapper. - const auto * current_serialization = &serialization; - while (const auto * serialization_variant_element = typeid_cast(current_serialization)) - current_serialization = serialization_variant_element->getNested().get(); - - /// If we read subcolumn of nested Tuple, it may be wrapped to SerializationNamed + /// If we read subcolumn of nested Tuple or this Tuple is a subcolumn, it may be wrapped to SerializationWrapper /// several times to allow to reconstruct the substream path name. /// Here we don't need substream path name, so we drop first several wrapper serializations. - while (const auto * serialization_named = typeid_cast(current_serialization)) - current_serialization = serialization_named->getNested().get(); + const auto * current_serialization = &serialization; + while (const auto * serialization_wrapper = dynamic_cast(current_serialization)) + current_serialization = serialization_wrapper->getNested().get(); const auto * serialization_tuple = typeid_cast(current_serialization); if (!serialization_tuple) diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index bd994e313ba..102e5eaede0 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -94,6 +95,7 @@ enum class BinaryTypeIndex : uint8_t Bool = 0x2D, SimpleAggregateFunction = 0x2E, Nested = 0x2F, + JSON = 0x30, }; BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) @@ -202,7 +204,7 @@ BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) return BinaryTypeIndex::LowCardinality; case TypeIndex::Map: return BinaryTypeIndex::Map; - case TypeIndex::Object: + case TypeIndex::ObjectDeprecated: /// Object type will be deprecated and replaced by new implementation. No need to support it here. throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Binary encoding of type Object is not supported"); case TypeIndex::IPv4: @@ -216,6 +218,15 @@ BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) /// JSONPaths is used only during schema inference and cannot be used anywhere else. case TypeIndex::JSONPaths: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Binary encoding of type JSONPaths is not supported"); + case TypeIndex::Object: + { + const auto & object_type = assert_cast(*type); + switch (object_type.getSchemaFormat()) + { + case DataTypeObject::SchemaFormat::JSON: + return BinaryTypeIndex::JSON; + } + } } } @@ -480,6 +491,32 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) writeStringBinary(type_name, buf); break; } + case BinaryTypeIndex::JSON: + { + const auto & object_type = assert_cast(*type); + writeVarUInt(object_type.getMaxDynamicPaths(), buf); + writeBinary(UInt8(object_type.getMaxDynamicTypes()), buf); + const auto & typed_paths = object_type.getTypedPaths(); + writeVarUInt(typed_paths.size(), buf); + for (const auto & [path, path_type] : typed_paths) + { + writeStringBinary(path, buf); + encodeDataType(path_type, buf); + } + const auto & paths_to_skip = object_type.getPathsToSkip(); + writeVarUInt(paths_to_skip.size(), buf); + for (const auto & path : paths_to_skip) + writeStringBinary(path, buf); + const auto & path_prefixes_to_skip = object_type.getPathPrefixesToSkip(); + writeVarUInt(path_prefixes_to_skip.size(), buf); + for (const auto & prefix : path_prefixes_to_skip) + writeStringBinary(prefix, buf); + const auto & path_regexps_to_skip = object_type.getPathRegexpsToSkip(); + writeVarUInt(path_regexps_to_skip.size(), buf); + for (const auto & regexp : path_regexps_to_skip) + writeStringBinary(regexp, buf); + break; + } default: break; } @@ -691,6 +728,62 @@ DataTypePtr decodeDataType(ReadBuffer & buf) readStringBinary(type_name, buf); return DataTypeFactory::instance().get(type_name); } + case BinaryTypeIndex::JSON: + { + size_t max_dynamic_paths; + readVarUInt(max_dynamic_paths, buf); + UInt8 max_dynamic_types; + readBinary(max_dynamic_types, buf); + size_t typed_paths_size; + readVarUInt(typed_paths_size, buf); + std::unordered_map typed_paths; + for (size_t i = 0; i != typed_paths_size; ++i) + { + String path; + readStringBinary(path, buf); + typed_paths[path] = decodeDataType(buf); + } + size_t paths_to_skip_size; + readVarUInt(paths_to_skip_size, buf); + std::unordered_set paths_to_skip; + paths_to_skip.reserve(paths_to_skip_size); + for (size_t i = 0; i != paths_to_skip_size; ++i) + { + String path; + readStringBinary(path, buf); + paths_to_skip.insert(path); + } + + size_t path_prefixes_to_skip_size; + readVarUInt(path_prefixes_to_skip_size, buf); + std::vector path_prefixes_to_skip; + path_prefixes_to_skip.reserve(path_prefixes_to_skip_size); + for (size_t i = 0; i != path_prefixes_to_skip_size; ++i) + { + String prefix; + readStringBinary(prefix, buf); + path_prefixes_to_skip.push_back(prefix); + } + + size_t path_regexps_to_skip_size; + readVarUInt(path_regexps_to_skip_size, buf); + std::vector path_regexps_to_skip; + path_regexps_to_skip.reserve(path_regexps_to_skip_size); + for (size_t i = 0; i != path_regexps_to_skip_size; ++i) + { + String regexp; + readStringBinary(regexp, buf); + path_regexps_to_skip.push_back(regexp); + } + return std::make_shared( + DataTypeObject::SchemaFormat::JSON, + typed_paths, + paths_to_skip, + path_prefixes_to_skip, + path_regexps_to_skip, + max_dynamic_paths, + max_dynamic_types); + } } throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown type code: {0:#04x}", UInt64(type)); diff --git a/src/DataTypes/DataTypesBinaryEncoding.h b/src/DataTypes/DataTypesBinaryEncoding.h index d02e7f85942..de001966aee 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.h +++ b/src/DataTypes/DataTypesBinaryEncoding.h @@ -8,58 +8,59 @@ namespace DB /** Binary encoding for ClickHouse data types: -|------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| ClickHouse data type | Binary encoding | -|------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| Nothing | 0x00 | -| UInt8 | 0x01 | -| UInt16 | 0x02 | -| UInt32 | 0x03 | -| UInt64 | 0x04 | -| UInt128 | 0x05 | -| UInt256 | 0x06 | -| Int8 | 0x07 | -| Int16 | 0x08 | -| Int32 | 0x09 | -| Int64 | 0x0A | -| Int128 | 0x0B | -| Int256 | 0x0C | -| Float32 | 0x0D | -| Float64 | 0x0E | -| Date | 0x0F | -| Date32 | 0x10 | -| DateTime | 0x11 | -| DateTime(time_zone) | 0x12 | -| DateTime64(P) | 0x13 | -| DateTime64(P, time_zone) | 0x14 | -| String | 0x15 | -| FixedString(N) | 0x16 | -| Enum8 | 0x17... | -| Enum16 | 0x18...> | -| Decimal32(P, S) | 0x19 | -| Decimal64(P, S) | 0x1A | -| Decimal128(P, S) | 0x1B | -| Decimal256(P, S) | 0x1C | -| UUID | 0x1D | -| Array(T) | 0x1E | -| Tuple(T1, ..., TN) | 0x1F... | -| Tuple(name1 T1, ..., nameN TN) | 0x20... | -| Set | 0x21 | -| Interval | 0x22 | -| Nullable(T) | 0x23 | -| Function | 0x24... | -| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | -| LowCardinality(T) | 0x26 | -| Map(K, V) | 0x27 | -| IPv4 | 0x28 | -| IPv6 | 0x29 | -| Variant(T1, ..., TN) | 0x2A... | -| Dynamic(max_types=N) | 0x2B | -| Custom type (Ring, Polygon, etc) | 0x2C | -| Bool | 0x2D | -| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | -| Nested(name1 T1, ..., nameN TN) | 0x2F... | -|------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +|---------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| ClickHouse data type | Binary encoding | +|---------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Nothing | 0x00 | +| UInt8 | 0x01 | +| UInt16 | 0x02 | +| UInt32 | 0x03 | +| UInt64 | 0x04 | +| UInt128 | 0x05 | +| UInt256 | 0x06 | +| Int8 | 0x07 | +| Int16 | 0x08 | +| Int32 | 0x09 | +| Int64 | 0x0A | +| Int128 | 0x0B | +| Int256 | 0x0C | +| Float32 | 0x0D | +| Float64 | 0x0E | +| Date | 0x0F | +| Date32 | 0x10 | +| DateTime | 0x11 | +| DateTime(time_zone) | 0x12 | +| DateTime64(P) | 0x13 | +| DateTime64(P, time_zone) | 0x14 | +| String | 0x15 | +| FixedString(N) | 0x16 | +| Enum8 | 0x17... | +| Enum16 | 0x18...> | +| Decimal32(P, S) | 0x19 | +| Decimal64(P, S) | 0x1A | +| Decimal128(P, S) | 0x1B | +| Decimal256(P, S) | 0x1C | +| UUID | 0x1D | +| Array(T) | 0x1E | +| Tuple(T1, ..., TN) | 0x1F... | +| Tuple(name1 T1, ..., nameN TN) | 0x20... | +| Set | 0x21 | +| Interval | 0x22 | +| Nullable(T) | 0x23 | +| Function | 0x24... | +| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | +| LowCardinality(T) | 0x26 | +| Map(K, V) | 0x27 | +| IPv4 | 0x28 | +| IPv6 | 0x29 | +| Variant(T1, ..., TN) | 0x2A... | +| Dynamic(max_types=N) | 0x2B | +| Custom type (Ring, Polygon, etc) | 0x2C | +| Bool | 0x2D | +| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | +| Nested(name1 T1, ..., nameN TN) | 0x2F... | +| JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP PREFIX skip_path_prefix, SKIP REGEXP skip_path_regexp) | 0x30............ | +|---------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| Interval kind binary encoding: |---------------|-----------------| diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 03874279a0b..536d2656021 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -178,8 +178,7 @@ DataTypePtr FieldToDataType::operator() (const Map & map) const template DataTypePtr FieldToDataType::operator() (const Object &) const { - /// TODO: Do we need different parameters for type Object? - return std::make_shared("json", false); + return std::make_shared(DataTypeObject::SchemaFormat::JSON); } template diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 1cb64b65d3a..9938bb8849a 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -361,9 +361,10 @@ bool isArray(TYPE data_type) { return WhichDataType(data_type).isArray(); } \ bool isTuple(TYPE data_type) { return WhichDataType(data_type).isTuple(); } \ bool isMap(TYPE data_type) {return WhichDataType(data_type).isMap(); } \ bool isInterval(TYPE data_type) {return WhichDataType(data_type).isInterval(); } \ -bool isObject(TYPE data_type) { return WhichDataType(data_type).isObject(); } \ +bool isObjectDeprecated(TYPE data_type) { return WhichDataType(data_type).isObjectDeprecated(); } \ bool isVariant(TYPE data_type) { return WhichDataType(data_type).isVariant(); } \ bool isDynamic(TYPE data_type) { return WhichDataType(data_type).isDynamic(); } \ +bool isObject(TYPE data_type) { return WhichDataType(data_type).isObject(); } \ bool isNothing(TYPE data_type) { return WhichDataType(data_type).isNothing(); } \ \ bool isColumnedAsNumber(TYPE data_type) \ diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 397ae3d8be9..a7665e610ab 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -432,7 +432,7 @@ struct WhichDataType constexpr bool isMap() const {return idx == TypeIndex::Map; } constexpr bool isSet() const { return idx == TypeIndex::Set; } constexpr bool isInterval() const { return idx == TypeIndex::Interval; } - constexpr bool isObject() const { return idx == TypeIndex::Object; } + constexpr bool isObjectDeprecated() const { return idx == TypeIndex::ObjectDeprecated; } constexpr bool isNothing() const { return idx == TypeIndex::Nothing; } constexpr bool isNullable() const { return idx == TypeIndex::Nullable; } @@ -444,6 +444,7 @@ struct WhichDataType constexpr bool isVariant() const { return idx == TypeIndex::Variant; } constexpr bool isDynamic() const { return idx == TypeIndex::Dynamic; } + constexpr bool isObject() const { return idx == TypeIndex::Object; } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) @@ -502,9 +503,10 @@ bool isArray(TYPE data_type); \ bool isTuple(TYPE data_type); \ bool isMap(TYPE data_type); \ bool isInterval(TYPE data_type); \ -bool isObject(TYPE data_type); \ +bool isObjectDeprecated(TYPE data_type); \ bool isVariant(TYPE data_type); \ bool isDynamic(TYPE data_type); \ +bool isObject(TYPE data_type); \ bool isNothing(TYPE data_type); \ \ bool isColumnedAsNumber(TYPE data_type); \ diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 1d525e5987f..4df240daa8f 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -3,28 +3,28 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include namespace DB @@ -149,12 +149,12 @@ static DataTypePtr recreateTupleWithElements(const DataTypeTuple & type_tuple, c } static std::pair convertObjectColumnToTuple( - const ColumnObject & column_object, const DataTypeObject & type_object) + const ColumnObjectDeprecated & column_object, const DataTypeObjectDeprecated & type_object) { if (!column_object.isFinalized()) { auto finalized = column_object.cloneFinalized(); - const auto & finalized_object = assert_cast(*finalized); + const auto & finalized_object = assert_cast(*finalized); return convertObjectColumnToTuple(finalized_object, type_object); } @@ -180,9 +180,9 @@ static std::pair recursivlyConvertDynamicColumnToTuple( if (!type->hasDynamicSubcolumnsDeprecated()) return {column, type}; - if (const auto * type_object = typeid_cast(type.get())) + if (const auto * type_object = typeid_cast(type.get())) { - const auto & column_object = assert_cast(*column); + const auto & column_object = assert_cast(*column); return convertObjectColumnToTuple(column_object, *type_object); } @@ -338,7 +338,7 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che for (const auto & [key, subtypes] : subcolumns_types) { assert(!subtypes.empty()); - if (key.getPath() == ColumnObject::COLUMN_NAME_DUMMY) + if (key.getPath() == ColumnObjectDeprecated::COLUMN_NAME_DUMMY) continue; size_t first_dim = getNumberOfDimensions(*subtypes[0]); @@ -354,7 +354,7 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che if (tuple_paths.empty()) { - tuple_paths.emplace_back(ColumnObject::COLUMN_NAME_DUMMY); + tuple_paths.emplace_back(ColumnObjectDeprecated::COLUMN_NAME_DUMMY); tuple_types.emplace_back(std::make_shared()); } @@ -421,7 +421,7 @@ static DataTypePtr getLeastCommonTypeForDynamicColumnsImpl( if (!type_in_storage->hasDynamicSubcolumnsDeprecated()) return type_in_storage; - if (isObject(type_in_storage)) + if (isObjectDeprecated(type_in_storage)) return getLeastCommonTypeForObject(concrete_types, check_ambiguos_paths); if (const auto * type_array = typeid_cast(type_in_storage.get())) @@ -463,9 +463,9 @@ DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage if (!type_in_storage->hasDynamicSubcolumnsDeprecated()) return type_in_storage; - if (isObject(type_in_storage)) + if (isObjectDeprecated(type_in_storage)) return std::make_shared( - DataTypes{std::make_shared()}, Names{ColumnObject::COLUMN_NAME_DUMMY}); + DataTypes{std::make_shared()}, Names{ColumnObjectDeprecated::COLUMN_NAME_DUMMY}); if (const auto * type_array = typeid_cast(type_in_storage.get())) return std::make_shared( @@ -807,7 +807,7 @@ DataTypePtr unflattenTuple(const PathsInData & paths, const DataTypes & tuple_ty return unflattenTuple(paths, tuple_types, tuple_columns).second; } -std::pair unflattenObjectToTuple(const ColumnObject & column) +std::pair unflattenObjectToTuple(const ColumnObjectDeprecated & column) { const auto & subcolumns = column.getSubcolumns(); @@ -815,7 +815,7 @@ std::pair unflattenObjectToTuple(const ColumnObject & co { auto type = std::make_shared( DataTypes{std::make_shared()}, - Names{ColumnObject::COLUMN_NAME_DUMMY}); + Names{ColumnObjectDeprecated::COLUMN_NAME_DUMMY}); return {type->createColumn()->cloneResized(column.size()), type}; } diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 6599d8adef1..80c3c4a14c3 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { @@ -85,7 +85,7 @@ DataTypePtr unflattenTuple( const PathsInData & paths, const DataTypes & tuple_types); -std::pair unflattenObjectToTuple(const ColumnObject & column); +std::pair unflattenObjectToTuple(const ColumnObjectDeprecated & column); std::pair unflattenTuple( const PathsInData & paths, diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 7642a6619b3..9d0d9f7c6eb 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -202,6 +202,12 @@ String getNameForSubstreamPath( stream_name += "." + it->variant_element_name + ".null"; else if (it->type == SubstreamType::DynamicStructure) stream_name += ".dynamic_structure"; + else if (it->type == SubstreamType::ObjectStructure) + stream_name += ".object_structure"; + else if (it->type == SubstreamType::ObjectSharedData) + stream_name += ".object_shared_data"; + else if (it->type == SubstreamType::ObjectTypedPath || it->type == SubstreamType::ObjectDynamicPath) + stream_name += "." + it->object_path_name; } return stream_name; @@ -401,7 +407,17 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref || path[last_elem].type == Substream::TupleElement || path[last_elem].type == Substream::ArraySizes || path[last_elem].type == Substream::VariantElement - || path[last_elem].type == Substream::VariantElementNullMap; + || path[last_elem].type == Substream::VariantElementNullMap + || path[last_elem].type == Substream::ObjectTypedPath; +} + +bool ISerialization::isFictitiousSubcolumn(const DB::ISerialization::SubstreamPath & path, size_t prefix_len) +{ + if (prefix_len == 0 || prefix_len > path.size()) + return false; + + size_t last_elem = prefix_len - 1; + return path[last_elem].type == Substream::VariantElementNullMap; } ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 255dbbfadd2..51fcdaec6ab 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -176,8 +176,8 @@ public: SparseElements, SparseOffsets, - ObjectStructure, - ObjectData, + DeprecatedObjectStructure, + DeprecatedObjectData, VariantDiscriminators, NamedVariantDiscriminators, @@ -189,6 +189,12 @@ public: DynamicData, DynamicStructure, + ObjectData, + ObjectTypedPath, + ObjectDynamicPath, + ObjectSharedData, + ObjectStructure, + Regular, }; @@ -203,6 +209,9 @@ public: /// Name of substream for type from 'named_types'. String name_of_substream; + /// Path name for Object type elements. + String object_path_name; + /// Data for current substream. SubstreamData data; @@ -262,13 +271,13 @@ public: bool use_compact_variant_discriminators_serialization = false; - enum class DynamicStatisticsMode + enum class ObjectAndDynamicStatisticsMode { NONE, /// Don't write statistics. PREFIX, /// Write statistics in prefix. SUFFIX, /// Write statistics in suffix. }; - DynamicStatisticsMode dynamic_write_statistics = DynamicStatisticsMode::NONE; + ObjectAndDynamicStatisticsMode object_and_dynamic_write_statistics = ObjectAndDynamicStatisticsMode::NONE; }; struct DeserializeBinaryBulkSettings @@ -289,7 +298,7 @@ public: /// If not zero, may be used to avoid reallocations while reading column of String type. double avg_value_size_hint = 0; - bool dynamic_read_statistics = false; + bool object_and_dynamic_read_statistics = false; }; /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. @@ -438,6 +447,7 @@ public: static size_t getArrayLevel(const SubstreamPath & path); static bool hasSubcolumnForPath(const SubstreamPath & path, size_t prefix_len); static SubstreamData createFromPath(const SubstreamPath & path, size_t prefix_len); + static bool isFictitiousSubcolumn(const SubstreamPath & path, size_t prefix_len); protected: template @@ -481,4 +491,69 @@ State * ISerialization::checkAndGetState(const StatePtr & state, const ISerializ return state_concrete; } + +WITH snaps AS ( + SELECT + rs.brokerfk brokerfk, + rs.brokerinstancefk brokerinstancefk, + rs.tradedt tradedt, + min(rs.snapdttm) snapdttmmin, + max(rs.snapdttm) snapdttmmax + FROM risksnapshothistory rs + WHERE rs.snapdttm >= now() - interval '10 minutes' + AND rs.snapdttm <= now() + GROUP BY rs.brokerfk, rs.brokerinstancefk, rs.tradedt + ORDER BY rs.brokerfk, rs.brokerinstancefk, rs.tradedt + ), + start_snaps_by_trade_dt AS ( + SELECT + rs.* + FROM risksnapshothistory rs + WHERE (rs.brokerfk, rs.brokerinstancefk, rs.tradedt, rs.snapdttm) IN ( + SELECT + s.brokerfk s_brokerfk, + s.brokerinstancefk s_brokerinstancefk, + s.tradedt s_tradedt, + s.snapdttmmin s_snapdttmmin + FROM snaps s) + ORDER BY rs.brokerfk, rs.brokerinstancefk, rs.snapdttm + ), + end_snaps_by_trade_dt AS ( + SELECT + rs.* + FROM risksnapshothistory rs + WHERE (rs.brokerfk, rs.brokerinstancefk, rs.tradedt, rs.snapdttm) IN ( + SELECT + s.brokerfk s_brokerfk, + s.brokerinstancefk s_brokerinstancefk, + s.tradedt s_tradedt, + s.snapdttmmax s_snapdttmmax + FROM snaps s) + ORDER BY rs.brokerfk, rs.brokerinstancefk, rs.snapdttm + ), + data as ( + SELECT + ssbtd.snapdttm startdttm, + esbtd.snapdttm enddttm, + case when(ssbtd.snapdttm = '1970-01-01 00:00:00.000000') then 0 else 1 end hasstart, + case when(esbtd.snapdttm = '1970-01-01 00:00:00.000000') then 0 else 1 end hasend, + IF(esbtd.brokerfk = 0, ssbtd.brokerfk, esbtd.brokerfk) brokerfk, + IF(esbtd.brokerinstancefk = 0, ssbtd.brokerinstancefk, esbtd.brokerinstancefk) brokerinstancefk + FROM start_snaps_by_trade_dt ssbtd + FULL OUTER JOIN end_snaps_by_trade_dt esbtd ON ssbtd.brokerfk = esbtd.brokerfk AND ssbtd.brokerinstancefk = esbtd.brokerinstancefk AND ssbtd.tradedt = esbtd.tradedt AND ssbtd.traderfk = esbtd.traderfk AND ssbtd.brokersymbolfk = esbtd.brokersymbolfk + ) + SELECT + d.brokerfk, d.brokerinstancefk, + max(startdttm) startdttm, + max(enddttm) enddttm, + countIf(d.hasstart =1 and d.hasend=1) matches, + countIf(d.hasstart =1 and d.hasend=0) beforeonly, + countIf(d.hasstart =0 and d.hasend=1) afteronly, + (select array_agg(toJSONString(map( + 'snapdttmmax', toString(snapdttmmax), 'snapdttmmin', toString(snapdttmmin), + 'tradedt', toString(tradedt), 'brokerfk', toString(brokerfk), 'brokerinstancefk', toString(brokerinstancefk)))) + from snaps) snaps + FROM data d + GROUP BY d.brokerfk, d.brokerinstancefk + } diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 7609ffc91ca..6cc7a7e2e74 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -117,7 +117,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( writeStringBinary(dynamic_state->variant_type->getName(), *stream); /// Write statistics in prefix if needed. - if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::PREFIX) + if (settings.object_and_dynamic_write_statistics == SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::PREFIX) { const auto & statistics = column_dynamic.getStatistics(); for (size_t i = 0; i != variant_info.variant_names.size(); ++i) @@ -156,8 +156,8 @@ void SerializationDynamic::deserializeBinaryBulkStatePrefix( return; auto dynamic_state = std::make_shared(); - dynamic_state->structure_state = structure_state; - dynamic_state->variant_serialization = checkAndGetState(structure_state)->variant_type->getDefaultSerialization(); + dynamic_state->structure_state = std::move(structure_state); + dynamic_state->variant_serialization = checkAndGetState(dynamic_state->structure_state)->variant_type->getDefaultSerialization(); settings.path.push_back(Substream::DynamicData); dynamic_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_state->variant_state, cache); @@ -174,7 +174,7 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD DeserializeBinaryBulkStatePtr state = nullptr; if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) { - state = cached_state; + state = std::move(cached_state); } else if (auto * structure_stream = settings.getter(settings.path)) { @@ -198,18 +198,13 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect type of Dynamic nested column, expected Variant, got {}", structure_state->variant_type->getName()); /// Read statistics. - if (settings.dynamic_read_statistics) + if (settings.object_and_dynamic_read_statistics) { - const auto & variants = variant_type->getVariants(); - size_t variant_size; - for (const auto & variant : variants) - { - readVarUInt(variant_size, *structure_stream); - structure_state->statistics.data[variant->getName()] = variant_size; - } + for (const auto & variant : variant_type->getVariants()) + readVarUInt(structure_state->statistics.data[variant->getName()], *structure_stream); } - state = structure_state; + state = std::move(structure_state); addToSubstreamsDeserializeStatesCache(cache, settings.path, state); } @@ -226,10 +221,10 @@ void SerializationDynamic::serializeBinaryBulkStateSuffix( settings.path.pop_back(); if (!stream) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for Dynamic column structure during serialization of binary bulk state prefix"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for Dynamic column structure during serialization of binary bulk state suffix"); /// Write statistics in suffix if needed. - if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::SUFFIX) + if (settings.object_and_dynamic_write_statistics == SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::SUFFIX) { for (const auto & variant_name : dynamic_state->variant_names) writeVarUInt(dynamic_state->statistics.data[variant_name], *stream); @@ -246,6 +241,18 @@ void SerializationDynamic::serializeBinaryBulkWithMultipleStreams( size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + size_t tmp_size; + serializeBinaryBulkWithMultipleStreamsAndCountTotalSizeOfVariants(column, offset, limit, settings, state, tmp_size); +} + +void SerializationDynamic::serializeBinaryBulkWithMultipleStreamsAndCountTotalSizeOfVariants( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state, + size_t & total_size_of_variants) const { const auto & column_dynamic = assert_cast(column); auto * dynamic_state = checkAndGetState(state); @@ -257,7 +264,7 @@ void SerializationDynamic::serializeBinaryBulkWithMultipleStreams( settings.path.push_back(Substream::DynamicData); assert_cast(*dynamic_state->variant_serialization) - .serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(*variant_column, offset, limit, settings, dynamic_state->variant_state, dynamic_state->statistics.data); + .serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(*variant_column, offset, limit, settings, dynamic_state->variant_state, dynamic_state->statistics.data, total_size_of_variants); settings.path.pop_back(); } @@ -538,6 +545,12 @@ void SerializationDynamic::serializeTextJSON(const IColumn & column, size_t row_ dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextJSON(dynamic_column.getVariantColumn(), row_num, ostr, settings); } +void SerializationDynamic::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const +{ + const auto & dynamic_column = assert_cast(column); + dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextJSONPretty(dynamic_column.getVariantColumn(), row_num, ostr, settings, indent); +} + void SerializationDynamic::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { auto read_field = [&settings](ReadBuffer & buf) diff --git a/src/DataTypes/Serializations/SerializationDynamic.h b/src/DataTypes/Serializations/SerializationDynamic.h index 001a3cf87ce..50593c79b0f 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.h +++ b/src/DataTypes/Serializations/SerializationDynamic.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include namespace DB @@ -11,7 +12,7 @@ class SerializationDynamicElement; class SerializationDynamic : public ISerialization { public: - explicit SerializationDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_dynamic_types_) + explicit SerializationDynamic(size_t max_dynamic_types_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES) : max_dynamic_types(max_dynamic_types_) { } @@ -59,6 +60,14 @@ public: SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkWithMultipleStreamsAndCountTotalSizeOfVariants( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state, + size_t & total_size_of_variants) const; + void deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, size_t limit, @@ -89,6 +98,7 @@ public: bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.cpp b/src/DataTypes/Serializations/SerializationDynamicElement.cpp index 211f0ac9377..9c698f96888 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.cpp +++ b/src/DataTypes/Serializations/SerializationDynamicElement.cpp @@ -48,6 +48,7 @@ void SerializationDynamicElement::enumerateStreams( .withColumn(data.column) .withSerializationInfo(data.serialization_info) .withDeserializeState(deserialize_state->variant_element_state); + settings.path.back().data = variant_data; deserialize_state->variant_serialization->enumerateStreams(settings, callback, variant_data); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationJSON.cpp b/src/DataTypes/Serializations/SerializationJSON.cpp new file mode 100644 index 00000000000..b5bf8d1421b --- /dev/null +++ b/src/DataTypes/Serializations/SerializationJSON.cpp @@ -0,0 +1,409 @@ +#include +#include +#include + +#if USE_SIMDJSON +#include +#endif +#if USE_RAPIDJSON +#include +#endif +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + +template +SerializationJSON::SerializationJSON( + std::unordered_map typed_paths_serializations_, + const std::unordered_set & paths_to_skip_, + const std::vector & path_prefixes_to_skip_, + const std::vector & path_regexps_to_skip_, + std::unique_ptr> json_extract_tree_) + : SerializationObject(std::move(typed_paths_serializations_), paths_to_skip_, path_prefixes_to_skip_, path_regexps_to_skip_) + , json_extract_tree(std::move(json_extract_tree_)) +{ +} + +namespace +{ + +/// Struct that represents elements of the JSON path. +/// "a.b.c" -> ["a", "b", "c"] +struct PathElements +{ + explicit PathElements(const String & path) + { + const char * start = path.data(); + const char * end = start + path.size(); + const char * pos = start; + const char * last_dot_pos = pos - 1; + for (pos = start; pos != end; ++pos) + { + if (*pos == '.') + { + elements.emplace_back(last_dot_pos + 1, size_t(pos - last_dot_pos - 1)); + last_dot_pos = pos; + } + } + + elements.emplace_back(last_dot_pos + 1, size_t(pos - last_dot_pos - 1)); + } + + size_t size() const { return elements.size(); } + + std::vector elements; +}; + +/// Struct that represents a prefix of a JSON path. Used during output of the JSON object. +struct Prefix +{ + /// Shrink current prefix to the common prefix of current prefix and specified path. + /// For example, if current prefix is a.b.c.d and path is a.b.e, then shrink the prefix to a.b. + void shrinkToCommonPrefix(const PathElements & path_elements) + { + /// Don't include last element in path_elements in the prefix. + size_t i = 0; + while (i != elements.size() && i != (path_elements.elements.size() - 1) && elements[i].first == path_elements.elements[i]) + ++i; + elements.resize(i); + } + + /// Check is_first flag in current object. + bool isFirstInCurrentObject() const + { + if (elements.empty()) + return root_is_first_flag; + return elements.back().second; + } + + /// Set flag is_first = false in current object. + void setNotFirstInCurrentObject() + { + if (elements.empty()) + root_is_first_flag = false; + else + elements.back().second = false; + } + + size_t size() const { return elements.size(); } + + /// Elements of the prefix: (path element, is_first flag in this prefix). + /// is_first flag indicates if we already serialized some key in the object with such prefix. + std::vector> elements; + bool root_is_first_flag = true; +}; + +} + +template +void SerializationJSON::serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, bool pretty, size_t indent) const +{ + const auto & column_object = assert_cast(column); + const auto & typed_paths = column_object.getTypedPaths(); + const auto & dynamic_paths = column_object.getDynamicPaths(); + const auto & shared_data_offsets = column_object.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + size_t shared_data_offset = shared_data_offsets[ssize_t(row_num) - 1]; + size_t shared_data_end = shared_data_offsets[ssize_t(row_num)]; + + /// We need to convert the set of paths in this row to a JSON object. + /// To do it, we first collect all the paths from current row, then we sort them + /// and construct the resulting JSON object by iterating over sorter list of paths. + /// For example: + /// b.c, a.b, a.a, b.e, g, h.u.t -> a.a, a.b, b.c, b.e, g, h.u.t -> {"a" : {"a" : ..., "b" : ...}, "b" : {"c" : ..., "e" : ...}, "g" : ..., "h" : {"u" : {"t" : ...}}}. + std::vector sorted_paths; + sorted_paths.reserve(typed_paths.size() + dynamic_paths.size() + (shared_data_end - shared_data_offset)); + for (const auto & [path, _] : typed_paths) + sorted_paths.emplace_back(path); + for (const auto & [path, dynamic_column] : dynamic_paths) + { + /// We consider null value and absense of the path in a row as equivalent cases, because we cannot actually distinguish them. + /// So, we don't output null values at all. + if (!dynamic_column->isNullAt(row_num)) + sorted_paths.emplace_back(path); + } + for (size_t i = shared_data_offset; i != shared_data_end; ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + sorted_paths.emplace_back(path); + } + + std::sort(sorted_paths.begin(), sorted_paths.end()); + + if (pretty) + writeCString("{\n", ostr); + else + writeChar('{', ostr); + size_t index_in_shared_data_values = shared_data_offset; + /// current_prefix represents the path of the object we are currently serializing keys in. + Prefix current_prefix; + for (const auto & path : sorted_paths) + { + PathElements path_elements(path); + /// Change prefix to common prefix between current prefix and current path. + /// If prefix changed (it can only decrease), close all finished objects. + /// For example: + /// Current prefix: a.b.c.d + /// Current path: a.b.e.f + /// It means now we have : {..., "a" : {"b" : {"c" : {"d" : ... + /// Common prefix will be a.b, so it means we should close objects a.b.c.d and a.b.c: {..., "a" : {"b" : {"c" : {"d" : ...}} + /// and continue serializing keys in object a.b + size_t prev_prefix_size = current_prefix.size(); + current_prefix.shrinkToCommonPrefix(path_elements); + size_t prefix_size = current_prefix.size(); + if (prefix_size != prev_prefix_size) + { + size_t objects_to_close = prev_prefix_size - prefix_size; + if (pretty) + { + writeChar('\n', ostr); + for (size_t i = 0; i != objects_to_close; ++i) + { + writeChar(' ', (indent + prefix_size + objects_to_close - i) * 4, ostr); + if (i != objects_to_close - 1) + writeCString("}\n", ostr); + else + writeChar('}', ostr); + } + } + else + { + for (size_t i = 0; i != objects_to_close; ++i) + writeChar('}', ostr); + } + } + + /// Now we are inside object that has common prefix with current path. + /// We should go inside all objects in current path. + /// From the example above we should open object a.b.e: + /// {..., "a" : {"b" : {"c" : {"d" : ...}}, "e" : { + if (prefix_size + 1 < path_elements.size()) + { + for (size_t i = prefix_size; i != path_elements.size() - 1; ++i) + { + /// Write comma before the key if it's not the first key in this prefix. + if (!current_prefix.isFirstInCurrentObject()) + { + if (pretty) + writeCString(",\n", ostr); + else + writeChar(',', ostr); + } + else + { + current_prefix.setNotFirstInCurrentObject(); + } + + if (pretty) + { + writeChar(' ', (indent + i + 1) * 4, ostr); + writeJSONString(path_elements.elements[i], ostr, settings); + writeCString(" : {\n", ostr); + } + else + { + writeJSONString(path_elements.elements[i], ostr, settings); + writeCString(":{", ostr); + } + + /// Update current prefix. + current_prefix.elements.emplace_back(path_elements.elements[i], true); + } + } + + /// Write comma before the key if it's not the first key in this prefix. + if (!current_prefix.isFirstInCurrentObject()) + { + if (pretty) + writeCString(",\n", ostr); + else + writeChar(',', ostr); + } + else + { + current_prefix.setNotFirstInCurrentObject(); + } + + if (pretty) + { + writeChar(' ', (indent + current_prefix.size() + 1) * 4, ostr); + writeJSONString(path_elements.elements.back(), ostr, settings); + writeCString(" : ", ostr); + } + else + { + writeJSONString(path_elements.elements.back(), ostr, settings); + writeCString(":", ostr); + } + + /// Serialize value of current path. + if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) + { + if (pretty) + typed_path_serializations.at(path)->serializeTextJSONPretty(*typed_it->second, row_num, ostr, settings, indent + current_prefix.size() + 1); + else + typed_path_serializations.at(path)->serializeTextJSON(*typed_it->second, row_num, ostr, settings); + } + else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) + { + if (pretty) + dynamic_serialization->serializeTextJSONPretty(*dynamic_it->second, row_num, ostr, settings, indent + current_prefix.size() + 1); + else + dynamic_serialization->serializeTextJSON(*dynamic_it->second, row_num, ostr, settings); + } + else + { + /// To serialize value stored in shared data we should first deserialize it from binary format. + auto tmp_dynamic_column = ColumnDynamic::create(); + tmp_dynamic_column->reserve(1); + column_object.deserializeValueFromSharedData(shared_data_values, index_in_shared_data_values++, *tmp_dynamic_column); + + if (pretty) + dynamic_serialization->serializeTextJSONPretty(*tmp_dynamic_column, 0, ostr, settings, indent + current_prefix.size() + 1); + else + dynamic_serialization->serializeTextJSON(*tmp_dynamic_column, 0, ostr, settings); + } + } + + /// Close all remaining open objects. + if (pretty) + { + writeChar('\n', ostr); + for (size_t i = 0; i != current_prefix.elements.size(); ++i) + { + writeChar(' ', (indent + current_prefix.size() - i) * 4, ostr); + writeCString("}\n", ostr); + } + writeChar(' ', indent * 4, ostr); + writeChar('}', ostr); + } + else + { + for (size_t i = 0; i != current_prefix.elements.size(); ++i) + writeChar('}', ostr); + writeChar('}', ostr); + } +} + +template +void SerializationJSON::deserializeTextImpl(IColumn & column, std::string_view object, const FormatSettings & settings) const +{ + typename Parser::Element document; + auto parser = parsers_pool.get([] { return new Parser; }); + if (!parser->parse(object, document)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object here: {}", object); + + String error; + if (!json_extract_tree->insertResultToColumn(column, document, insert_settings, settings, error)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot insert data into JSON column: {}", error); +} + +template +void SerializationJSON::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeTextImpl(column, row_num, ostr, settings); +} + +template +void SerializationJSON::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String object; + readStringUntilEOF(object, istr); + deserializeTextImpl(column, object, settings); +} + +template +void SerializationJSON::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString buf; + serializeTextImpl(column, row_num, buf, settings); + writeEscapedString(buf.str(), ostr); +} + +template +void SerializationJSON::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String object; + readEscapedString(object, istr); + deserializeTextImpl(column, object, settings); +} + +template +void SerializationJSON::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString buf; + serializeTextImpl(column, row_num, buf, settings); + writeQuotedString(buf.str(), ostr); +} + +template +void SerializationJSON::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String object; + readQuotedString(object, istr); + deserializeTextImpl(column, object, settings); +} + +template +void SerializationJSON::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString buf; + serializeTextImpl(column, row_num, buf, settings); + writeCSVString(buf.str(), ostr); +} + +template +void SerializationJSON::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String object; + readCSVString(object, istr, settings.csv); + deserializeTextImpl(column, object, settings); +} + +template +void SerializationJSON::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString buf; + serializeTextImpl(column, row_num, buf, settings); + writeXMLStringForTextElement(buf.str(), ostr); +} + +template +void SerializationJSON::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeTextImpl(column, row_num, ostr, settings); +} + +template +void SerializationJSON::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const +{ + serializeTextImpl(column, row_num, ostr, settings, true, indent); +} + +template +void SerializationJSON::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String object_buffer; + auto object_view = readJSONObjectAsViewPossiblyInvalid(istr, object_buffer); + deserializeTextImpl(column, object_view, settings); +} + +#if USE_SIMDJSON +template class SerializationJSON; +#endif +#if USE_RAPIDJSON +template class SerializationJSON; +#else +template class SerializationJSON; +#endif + + + + +} diff --git a/src/DataTypes/Serializations/SerializationJSON.h b/src/DataTypes/Serializations/SerializationJSON.h new file mode 100644 index 00000000000..0c6d2bd164b --- /dev/null +++ b/src/DataTypes/Serializations/SerializationJSON.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Class for text serialization/deserialization of the JSON data type. +template +class SerializationJSON : public SerializationObject +{ +public: + SerializationJSON( + std::unordered_map typed_paths_serializations_, + const std::unordered_set & paths_to_skip_, + const std::vector & path_prefixes_to_skip_, + const std::vector & path_regexps_to_skip_, + std::unique_ptr> json_extract_tree_); + + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + virtual void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + +private: + void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, bool pretty = false, size_t indent = 0) const; + void deserializeTextImpl(IColumn & column, std::string_view object, const FormatSettings & settings) const; + + std::unique_ptr> json_extract_tree; + JSONExtractInsertSettings insert_settings; + /// Pool of parser objects to make SerializationJSON thread safe. + mutable SimpleObjectPool parsers_pool; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationJSONElement.cpp b/src/DataTypes/Serializations/SerializationJSONElement.cpp new file mode 100644 index 00000000000..35119cd0764 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationJSONElement.cpp @@ -0,0 +1,3 @@ +// +// Created by Павел Круглов on 05/07/2024. +// diff --git a/src/DataTypes/Serializations/SerializationJSONElement.h b/src/DataTypes/Serializations/SerializationJSONElement.h new file mode 100644 index 00000000000..791ef2eecc5 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationJSONElement.h @@ -0,0 +1,8 @@ +// +// Created by Павел Круглов on 05/07/2024. +// + +#ifndef CLICKHOUSE_SERIALIZATIONJSONELEMENT_H +#define CLICKHOUSE_SERIALIZATIONJSONELEMENT_H + +#endif //CLICKHOUSE_SERIALIZATIONJSONELEMENT_H diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 40071c4607a..3195a04d348 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -268,9 +268,16 @@ void SerializationLowCardinality::serializeBinaryBulkStateSuffix( void SerializationLowCardinality::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, - SubstreamsDeserializeStatesCache * /*cache*/) const + SubstreamsDeserializeStatesCache * cache) const { settings.path.push_back(Substream::DictionaryKeys); + + if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) + { + state = std::move(cached_state); + return; + } + auto * stream = settings.getter(settings.path); settings.path.pop_back(); diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index c6c87b5aa7b..d4483703c6b 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -1,586 +1,673 @@ #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int INCORRECT_DATA; - extern const int CANNOT_READ_ALL_DATA; - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int CANNOT_PARSE_TEXT; - extern const int EXPERIMENTAL_FEATURE_ERROR; + extern const int LOGICAL_ERROR; } -template -template -void SerializationObject::deserializeTextImpl(IColumn & column, Reader && reader) const +SerializationObject::SerializationObject( + std::unordered_map typed_path_serializations_, + const std::unordered_set & paths_to_skip_, + const std::vector & path_prefixes_to_skip_, + const std::vector & path_regexps_to_skip_) + : typed_path_serializations(std::move(typed_path_serializations_)) + , paths_to_skip(paths_to_skip_) + , path_prefixes_to_skip(path_prefixes_to_skip_) + , dynamic_serialization(std::make_shared()) + , shared_data_serialization(getTypeOfSharedData()->getDefaultSerialization()) { - auto & column_object = assert_cast(column); + /// We will need sorted order of typed paths to serialize them in order for consistency. + sorted_typed_paths.reserve(typed_path_serializations.size()); + for (const auto & [path, _] : typed_path_serializations) + sorted_typed_paths.emplace_back(path); + std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); + for (const auto & regexp_str : path_regexps_to_skip_) + path_regexps_to_skip.emplace_back(regexp_str); +} - String buf; - reader(buf); - std::optional result; +const DataTypePtr & SerializationObject::getTypeOfSharedData() +{ + /// Array(Tuple(String, String)) + static const DataTypePtr type = std::make_shared(std::make_shared(DataTypes{std::make_shared(), std::make_shared()}, Names{"paths", "values"})); + return type; +} - /// Treat empty string as an empty object - /// for better CAST from String to Object. - if (!buf.empty()) +bool SerializationObject::shouldSkipPath(const String & path) const +{ + if (paths_to_skip.contains(path)) + return true; + + for (const auto & prefix : path_prefixes_to_skip) { - auto parser = parsers_pool.get([] { return new Parser; }); - result = parser->parse(buf.data(), buf.size()); - } - else - { - result = ParseResult{}; + if (path.starts_with(prefix)) + return true; } - if (!result) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse object"); - - auto & [paths, values] = *result; - assert(paths.size() == values.size()); - - size_t old_column_size = column_object.size(); - for (size_t i = 0; i < paths.size(); ++i) + for (const auto & regexp : path_regexps_to_skip) { - auto field_info = getFieldInfo(values[i]); - if (field_info.need_fold_dimension) - values[i] = applyVisitor(FieldVisitorFoldDimension(field_info.num_dimensions), std::move(values[i])); - if (isNothing(field_info.scalar_type)) - continue; + if (re2::RE2::FullMatch(path, regexp)) + return true; + } - if (!column_object.hasSubcolumn(paths[i])) + return false; +} + +SerializationObject::ObjectSerializationVersion::ObjectSerializationVersion(UInt64 version) : value(static_cast(version)) +{ + checkVersion(version); +} + +void SerializationObject::ObjectSerializationVersion::checkVersion(UInt64 version) +{ + if (version != BASIC) + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid version for Object structure serialization."); +} + +struct SerializeBinaryBulkStateObject: public ISerialization::SerializeBinaryBulkState +{ + SerializationObject::ObjectSerializationVersion serialization_version; + std::vector sorted_dynamic_paths; + std::unordered_map typed_path_states; + std::unordered_map dynamic_path_states; + ISerialization::SerializeBinaryBulkStatePtr shared_data_state; + /// Paths statistics. Map (dynamic path) -> (number of non-null values in this path). + ColumnObject::Statistics statistics = { .source = ColumnObject::Statistics::Source::READ, .data = {} }; + + explicit SerializeBinaryBulkStateObject(UInt64 serialization_version_) : serialization_version(serialization_version_) {} +}; + +struct DeserializeBinaryBulkStateObject : public ISerialization::DeserializeBinaryBulkState +{ + std::unordered_map typed_path_states; + std::unordered_map dynamic_path_states; + ISerialization::DeserializeBinaryBulkStatePtr shared_data_state; + ISerialization::DeserializeBinaryBulkStatePtr structure_state; +}; + +void SerializationObject::enumerateStreams(EnumerateStreamsSettings & settings, const StreamCallback & callback, const SubstreamData & data) const +{ + settings.path.push_back(Substream::ObjectStructure); + callback(settings.path); + settings.path.pop_back(); + + const auto * column_object = data.column ? &assert_cast(*data.column) : nullptr; + const auto * type_object = data.type ? &assert_cast(*data.type) : nullptr; + const auto * deserialize_state = data.deserialize_state ? checkAndGetState(data.deserialize_state) : nullptr; + const auto * structure_state = deserialize_state ? checkAndGetState(deserialize_state->structure_state) : nullptr; + settings.path.push_back(Substream::ObjectData); + + /// First, iterate over typed paths in sorted order, we will always serialize them. + for (const auto & path : sorted_typed_paths) + { + settings.path.back().creator = std::make_shared(path); + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + const auto & serialization = typed_path_serializations.at(path); + auto path_data = SubstreamData(serialization) + .withType(type_object ? type_object->getTypedPaths().at(path) : nullptr) + .withColumn(column_object ? column_object->getTypedPaths().at(path) : nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state ? deserialize_state->typed_path_states.at(path) : nullptr); + settings.path.back().data = path_data; + serialization->enumerateStreams(settings, callback, path_data); + settings.path.pop_back(); + settings.path.back().creator.reset(); + } + + /// If column or deserialization state was provided, iterate over dynamic paths, + if (column_object || structure_state) + { + /// Enumerate dynamic paths in sorted order for consistency. + const auto * dynamic_paths = column_object ? &column_object->getDynamicPaths() : nullptr; + std::vector sorted_dynamic_paths; + /// If we have deserialize_state we can take sorted dynamic paths list from it. + if (structure_state) { - if (paths[i].hasNested()) - column_object.addNestedSubcolumn(paths[i], field_info, old_column_size); - else - column_object.addSubcolumn(paths[i], old_column_size); + sorted_dynamic_paths = structure_state->sorted_dynamic_paths; + } + else + { + sorted_dynamic_paths.reserve(dynamic_paths->size()); + for (const auto & [path, _] : *dynamic_paths) + sorted_dynamic_paths.push_back(path); + std::sort(sorted_dynamic_paths.begin(), sorted_dynamic_paths.end()); } - auto & subcolumn = column_object.getSubcolumn(paths[i]); - assert(subcolumn.size() == old_column_size); - - subcolumn.insert(std::move(values[i]), std::move(field_info)); - } - - /// Insert default values to missed subcolumns. - const auto & subcolumns = column_object.getSubcolumns(); - for (const auto & entry : subcolumns) - { - if (entry->data.size() == old_column_size) + DataTypePtr dynamic_type = std::make_shared(); + for (const auto & path : sorted_dynamic_paths) { - bool inserted = column_object.tryInsertDefaultFromNested(entry); - if (!inserted) - entry->data.insertDefault(); + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + auto path_data = SubstreamData(dynamic_serialization) + .withType(dynamic_type) + .withColumn(dynamic_paths ? dynamic_paths->at(path) : nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state ? deserialize_state->dynamic_path_states.at(path) : nullptr); + settings.path.back().data = path_data; + dynamic_serialization->enumerateStreams(settings, callback, path_data); + settings.path.pop_back(); } } - column_object.incrementNumRows(); + settings.path.push_back(Substream::ObjectSharedData); + auto shared_data_substream_data = SubstreamData(shared_data_serialization) + .withType(getTypeOfSharedData()) + .withColumn(column_object ? column_object->getSharedDataPtr() : nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state ? deserialize_state->shared_data_state : nullptr); + shared_data_serialization->enumerateStreams(settings, callback, shared_data_substream_data); + settings.path.pop_back(); + settings.path.pop_back(); } -template -void SerializationObject::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -{ - deserializeTextImpl(column, [&](String & s) { readStringInto(s, istr); }); -} - -template -void SerializationObject::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -{ - deserializeTextImpl(column, [&](String & s) { settings.tsv.crlf_end_of_line_input ? readEscapedStringCRLF(s, istr) : readEscapedString(s, istr); }); -} - -template -void SerializationObject::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -{ - deserializeTextImpl(column, [&](String & s) { readQuotedStringInto(s, istr); }); -} - -template -void SerializationObject::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -{ - deserializeTextImpl(column, [&](String & s) { Parser::readJSON(s, istr); }); -} - -template -void SerializationObject::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -{ - deserializeTextImpl(column, [&](String & s) { readCSVStringInto(s, istr, settings.csv); }); -} - -template -template -void SerializationObject::checkSerializationIsSupported(const TSettings & settings) const -{ - if (settings.position_independent_encoding) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DataTypeObject doesn't support serialization with position independent encoding"); -} - -template -struct SerializationObject::SerializeStateObject : public ISerialization::SerializeBinaryBulkState -{ - DataTypePtr nested_type; - SerializationPtr nested_serialization; - SerializeBinaryBulkStatePtr nested_state; -}; - -template -struct SerializationObject::DeserializeStateObject : public ISerialization::DeserializeBinaryBulkState -{ - BinarySerializationKind kind; - DataTypePtr nested_type; - SerializationPtr nested_serialization; - DeserializeBinaryBulkStatePtr nested_state; -}; - -template -void SerializationObject::serializeBinaryBulkStatePrefix( +void SerializationObject::serializeBinaryBulkStatePrefix( const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - checkSerializationIsSupported(settings); - if (state) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DataTypeObject doesn't support serialization with non-trivial state"); - const auto & column_object = assert_cast(column); - if (!column_object.isFinalized()) - { - auto finalized = column_object.cloneFinalized(); - serializeBinaryBulkStatePrefix(*finalized, settings, state); - return; - } + const auto & typed_paths = column_object.getTypedPaths(); + const auto & dynamic_paths = column_object.getDynamicPaths(); + const auto & shared_data = column_object.getSharedDataPtr(); settings.path.push_back(Substream::ObjectStructure); auto * stream = settings.getter(settings.path); + settings.path.pop_back(); if (!stream) - throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Missing stream for kind of binary serialization"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for Object column structure during serialization of binary bulk state prefix"); - auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); + /// Write serialization version. + UInt64 serialization_version = ObjectSerializationVersion::Value::BASIC; + writeBinaryLittleEndian(serialization_version, *stream); - writeIntBinary(static_cast(BinarySerializationKind::TUPLE), *stream); - writeStringBinary(tuple_type->getName(), *stream); + /// Write all dynamic paths in sorted order. + auto object_state = std::make_shared(serialization_version); + object_state->sorted_dynamic_paths.reserve(dynamic_paths.size()); + for (const auto & [path, _] : dynamic_paths) + object_state->sorted_dynamic_paths.push_back(path); + std::sort(object_state->sorted_dynamic_paths.begin(), object_state->sorted_dynamic_paths.end()); + writeVarUInt(object_state->sorted_dynamic_paths.size(), *stream); + for (const auto & path : object_state->sorted_dynamic_paths) + writeStringBinary(path, *stream); - auto state_object = std::make_shared(); - state_object->nested_type = tuple_type; - state_object->nested_serialization = tuple_type->getDefaultSerialization(); - - settings.path.back() = Substream::ObjectData; - state_object->nested_serialization->serializeBinaryBulkStatePrefix(*tuple_column, settings, state_object->nested_state); - - state = std::move(state_object); - settings.path.pop_back(); -} - -template -void SerializationObject::serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - checkSerializationIsSupported(settings); - auto * state_object = checkAndGetState(state); + /// Write statistics in prefix if needed. + if (settings.object_and_dynamic_write_statistics == SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::PREFIX) + { + const auto & statistics = column_object.getStatistics(); + for (const auto & path : object_state->sorted_dynamic_paths) + { + size_t number_of_non_null_values = 0; + /// Check if we can use statistics stored in the column. There are 2 possible sources + /// of this statistics: + /// - statistics calculated during merge of some data parts (Statistics::Source::MERGE) + /// - statistics read from the data part during deserialization of Object column (Statistics::Source::READ). + /// We can rely only on statistics calculated during the merge, because column with statistics that was read + /// during deserialization from some data part could be filtered/limited/transformed/etc and so the statistics can be outdated. + if (!statistics.data.empty() && statistics.source == ColumnObject::Statistics::Source::MERGE) + number_of_non_null_values = statistics.data.at(path); + /// Otherwise we can use only path column from current object column. + else + number_of_non_null_values = (dynamic_paths.at(path)->size() - dynamic_paths.at(path)->getNumberOfDefaultRows()); + writeVarUInt(number_of_non_null_values, *stream); + } + } settings.path.push_back(Substream::ObjectData); - state_object->nested_serialization->serializeBinaryBulkStateSuffix(settings, state_object->nested_state); + + for (const auto & path : sorted_typed_paths) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + typed_path_serializations.at(path)->serializeBinaryBulkStatePrefix(*typed_paths.at(path), settings, object_state->typed_path_states[path]); + settings.path.pop_back(); + } + + for (const auto & path : object_state->sorted_dynamic_paths) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + dynamic_serialization->serializeBinaryBulkStatePrefix(*dynamic_paths.at(path), settings, object_state->dynamic_path_states[path]); + settings.path.pop_back(); + } + + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->serializeBinaryBulkStatePrefix(*shared_data, settings, object_state->shared_data_state); settings.path.pop_back(); + settings.path.pop_back(); + + state = std::move(object_state); } -template -void SerializationObject::deserializeBinaryBulkStatePrefix( +void SerializationObject::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const { - checkSerializationIsSupported(settings); - if (state) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DataTypeObject doesn't support serialization with non-trivial state"); + auto structure_state = deserializeObjectStructureStatePrefix(settings, cache); + if (!structure_state) + return; - settings.path.push_back(Substream::ObjectStructure); - auto * stream = settings.getter(settings.path); - settings.path.pop_back(); - - if (!stream) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, - "Cannot read kind of binary serialization of DataTypeObject, because its stream is missing"); - - UInt8 kind_raw; - readIntBinary(kind_raw, *stream); - auto kind = magic_enum::enum_cast(kind_raw); - if (!kind) - throw Exception(ErrorCodes::INCORRECT_DATA, - "Unknown binary serialization kind of Object: {}", std::to_string(kind_raw)); - - auto state_object = std::make_shared(); - state_object->kind = *kind; - - if (state_object->kind == BinarySerializationKind::TUPLE) - { - String data_type_name; - readStringBinary(data_type_name, *stream); - state_object->nested_type = DataTypeFactory::instance().get(data_type_name); - state_object->nested_serialization = state_object->nested_type->getDefaultSerialization(); - - if (!isTuple(state_object->nested_type)) - throw Exception(ErrorCodes::INCORRECT_DATA, - "Data of type Object should be written as Tuple, got: {}", data_type_name); - } - else if (state_object->kind == BinarySerializationKind::STRING) - { - state_object->nested_type = std::make_shared(); - state_object->nested_serialization = std::make_shared(); - } - else - { - throw Exception(ErrorCodes::INCORRECT_DATA, - "Unknown binary serialization kind of Object: {}", std::to_string(kind_raw)); - } + auto object_state = std::make_shared(); + object_state->structure_state = std::move(structure_state); settings.path.push_back(Substream::ObjectData); - state_object->nested_serialization->deserializeBinaryBulkStatePrefix(settings, state_object->nested_state, cache); - settings.path.pop_back(); - state = std::move(state_object); + for (const auto & path : sorted_typed_paths) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + typed_path_serializations.at(path)->deserializeBinaryBulkStatePrefix(settings, object_state->typed_path_states[path], cache); + settings.path.pop_back(); + } + + const auto & sorted_dynamic_paths = checkAndGetState(object_state->structure_state)->sorted_dynamic_paths; + for (const auto & path : sorted_dynamic_paths) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + dynamic_serialization->deserializeBinaryBulkStatePrefix(settings, object_state->dynamic_path_states[path], cache); + settings.path.pop_back(); + } + + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->deserializeBinaryBulkStatePrefix(settings, object_state->shared_data_state, cache); + settings.path.pop_back(); + settings.path.pop_back(); + + state = std::move(object_state); } -template -void SerializationObject::serializeBinaryBulkWithMultipleStreams( +ISerialization::DeserializeBinaryBulkStatePtr SerializationObject::deserializeObjectStructureStatePrefix( + DeserializeBinaryBulkSettings & settings, SubstreamsDeserializeStatesCache * cache) +{ + settings.path.push_back(Substream::ObjectStructure); + + DeserializeBinaryBulkStatePtr state = nullptr; + /// Check if we already deserialized this state. It can happen when we read both object column and its subcolumns. + if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) + { + state = cached_state; + } + else if (auto * structure_stream = settings.getter(settings.path)) + { + /// Read structure serialization version. + UInt64 serialization_version; + readBinaryLittleEndian(serialization_version, *structure_stream); + auto structure_state = std::make_shared(serialization_version); + /// Read the sorted list of dynamic paths. + size_t dynamic_paths_size; + readVarUInt(dynamic_paths_size, *structure_stream); + structure_state->sorted_dynamic_paths.reserve(dynamic_paths_size); + structure_state->dynamic_paths.reserve(dynamic_paths_size); + for (size_t i = 0; i != dynamic_paths_size; ++i) + { + structure_state->sorted_dynamic_paths.emplace_back(); + readStringBinary(structure_state->sorted_dynamic_paths.back(), *structure_stream); + structure_state->dynamic_paths.insert(structure_state->sorted_dynamic_paths.back()); + } + + /// Read statistics if needed. + if (settings.object_and_dynamic_read_statistics) + { + for (const auto & path : structure_state->sorted_dynamic_paths) + readVarUInt(structure_state->statistics.data[path], *structure_stream); + } + + state = std::move(structure_state); + addToSubstreamsDeserializeStatesCache(cache, settings.path, state); + } + + settings.path.pop_back(); + return state; +} + +void SerializationObject::serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - checkSerializationIsSupported(settings); const auto & column_object = assert_cast(column); - auto * state_object = checkAndGetState(state); - - if (!column_object.isFinalized()) - { - auto finalized = column_object.cloneFinalized(); - serializeBinaryBulkWithMultipleStreams(*finalized, offset, limit, settings, state); - return; - } - - auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); - - if (!state_object->nested_type->equals(*tuple_type)) - { - throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, - "Types of internal column of Object mismatched. Expected: {}, Got: {}", - state_object->nested_type->getName(), tuple_type->getName()); - } + const auto & typed_paths = column_object.getTypedPaths(); + const auto & dynamic_paths = column_object.getDynamicPaths(); + const auto & shared_data = column_object.getSharedDataPtr(); + auto * object_state = checkAndGetState(state); settings.path.push_back(Substream::ObjectData); - if (auto * stream = settings.getter(settings.path)) + + for (const auto & path : sorted_typed_paths) { - state_object->nested_serialization->serializeBinaryBulkWithMultipleStreams( - *tuple_column, offset, limit, settings, state_object->nested_state); + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + typed_path_serializations.at(path)->serializeBinaryBulkWithMultipleStreams(*typed_paths.at(path), offset, limit, settings, object_state->typed_path_states[path]); + settings.path.pop_back(); } + const auto * dynamic_serialization_typed = assert_cast(dynamic_serialization.get()); + for (const auto & path : object_state->sorted_dynamic_paths) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + auto it = dynamic_paths.find(path); + if (it == dynamic_paths.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Dynamic structure mismatch for Object column: dynamic path '{}' is not found in the column", path); + size_t number_of_non_null_values = 0; + dynamic_serialization_typed->serializeBinaryBulkWithMultipleStreamsAndCountTotalSizeOfVariants(*it->second, offset, limit, settings, object_state->dynamic_path_states[path], number_of_non_null_values); + object_state->statistics.data[path] += number_of_non_null_values; + settings.path.pop_back(); + } + + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->serializeBinaryBulkWithMultipleStreams(*shared_data, offset, limit, settings, object_state->shared_data_state); + settings.path.pop_back(); settings.path.pop_back(); } -template -void SerializationObject::deserializeBinaryBulkWithMultipleStreams( +void SerializationObject::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + auto * object_state = checkAndGetState(state); + settings.path.push_back(Substream::ObjectStructure); + auto * stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!stream) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for Object column structure during serialization of binary bulk state suffix"); + + /// Write statistics in suffix if needed. + if (settings.object_and_dynamic_write_statistics == SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::SUFFIX) + { + for (const auto & path : object_state->sorted_dynamic_paths) + writeVarUInt(object_state->statistics.data[path], *stream); + } + + settings.path.push_back(Substream::ObjectData); + + for (const auto & path : sorted_typed_paths) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + typed_path_serializations.at(path)->serializeBinaryBulkStateSuffix(settings, object_state->typed_path_states[path]); + settings.path.pop_back(); + } + + for (const auto & path : object_state->sorted_dynamic_paths) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + dynamic_serialization->serializeBinaryBulkStateSuffix(settings, object_state->dynamic_path_states[path]); + settings.path.pop_back(); + } + + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->serializeBinaryBulkStateSuffix(settings, object_state->shared_data_state); + settings.path.pop_back(); + settings.path.pop_back(); +} + +void SerializationObject::deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, size_t limit, DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - checkSerializationIsSupported(settings); - if (!column->empty()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DataTypeObject cannot be deserialized to non-empty column"); - + auto * object_state = checkAndGetState(state); + auto * structure_state = checkAndGetState(object_state->structure_state); auto mutable_column = column->assumeMutable(); auto & column_object = assert_cast(*mutable_column); - auto * state_object = checkAndGetState(state); + /// If it's a new object column, set dynamic paths and statistics. + if (column_object.empty()) + { + column_object.setDynamicPaths(structure_state->sorted_dynamic_paths); + column_object.setStatistics(structure_state->statistics); + } + + auto & typed_paths = column_object.getTypedPaths(); + auto & dynamic_paths = column_object.getDynamicPaths(); + auto & shared_data = column_object.getSharedDataPtr(); settings.path.push_back(Substream::ObjectData); - if (state_object->kind == BinarySerializationKind::STRING) - deserializeBinaryBulkFromString(column_object, limit, settings, *state_object, cache); - else - deserializeBinaryBulkFromTuple(column_object, limit, settings, *state_object, cache); + for (const auto & path : sorted_typed_paths) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + typed_path_serializations.at(path)->deserializeBinaryBulkWithMultipleStreams(typed_paths[path], limit, settings, object_state->typed_path_states[path], cache); + settings.path.pop_back(); + } + for (const auto & path : structure_state->sorted_dynamic_paths) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + dynamic_serialization->deserializeBinaryBulkWithMultipleStreams(dynamic_paths[path], limit, settings, object_state->dynamic_path_states[path], cache); + settings.path.pop_back(); + } + + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->deserializeBinaryBulkWithMultipleStreams(shared_data, limit, settings, object_state->shared_data_state, cache); + settings.path.pop_back(); settings.path.pop_back(); - column_object.checkConsistency(); - column_object.finalize(); - column = std::move(mutable_column); } -template -void SerializationObject::deserializeBinaryBulkFromString( - ColumnObject & column_object, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeStateObject & state, - SubstreamsCache * cache) const +void SerializationObject::serializeBinary(const Field & field, WriteBuffer & ostr, const DB::FormatSettings & settings) const { - ColumnPtr column_string = state.nested_type->createColumn(); - state.nested_serialization->deserializeBinaryBulkWithMultipleStreams( - column_string, limit, settings, state.nested_state, cache); - - size_t input_rows_count = column_string->size(); - column_object.reserve(input_rows_count); - - FormatSettings format_settings; - for (size_t i = 0; i < input_rows_count; ++i) + auto & object = field.get(); + /// Serialize number of paths and then pairs (path, value). + writeVarUInt(object.size(), ostr); + for (const auto & [path, value] : object) { - const auto & val = column_string->getDataAt(i); - ReadBufferFromMemory read_buffer(val.data, val.size); - deserializeWholeText(column_object, read_buffer, format_settings); - - if (!read_buffer.eof()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, - "Cannot parse string to column Object. Expected eof"); - } -} - -template -void SerializationObject::deserializeBinaryBulkFromTuple( - ColumnObject & column_object, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeStateObject & state, - SubstreamsCache * cache) const -{ - ColumnPtr column_tuple = state.nested_type->createColumn(); - state.nested_serialization->deserializeBinaryBulkWithMultipleStreams( - column_tuple, limit, settings, state.nested_state, cache); - - auto [tuple_paths, tuple_types] = flattenTuple(state.nested_type); - auto flattened_tuple = flattenTuple(column_tuple); - const auto & tuple_columns = assert_cast(*flattened_tuple).getColumns(); - - assert(tuple_paths.size() == tuple_types.size()); - size_t num_subcolumns = tuple_paths.size(); - - if (tuple_columns.size() != num_subcolumns) - throw Exception(ErrorCodes::INCORRECT_DATA, - "Inconsistent type ({}) and column ({}) while reading column of type Object", - state.nested_type->getName(), column_tuple->getName()); - - for (size_t i = 0; i < num_subcolumns; ++i) - column_object.addSubcolumn(tuple_paths[i], tuple_columns[i]->assumeMutable()); -} - -template -void SerializationObject::serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); -} - -template -void SerializationObject::deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); -} - -template -void SerializationObject::serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); -} - -template -void SerializationObject::deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); -} - -/// TODO: use format different of JSON in serializations. - -template -void SerializationObject::serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - const auto & column_object = assert_cast(column); - const auto & subcolumns = column_object.getSubcolumns(); - - writeChar('{', ostr); - for (auto it = subcolumns.begin(); it != subcolumns.end(); ++it) - { - const auto & entry = *it; - if (it != subcolumns.begin()) - writeCString(",", ostr); - - writeDoubleQuoted(entry->path.getPath(), ostr); - writeChar(':', ostr); - serializeTextFromSubcolumn(entry->data, row_num, ostr, settings); - } - writeChar('}', ostr); -} - -template -template -void SerializationObject::serializeTextFromSubcolumn( - const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const -{ - const auto & least_common_type = subcolumn.getLeastCommonType(); - - if (subcolumn.isFinalized()) - { - const auto & finalized_column = subcolumn.getFinalizedColumn(); - auto info = least_common_type->getSerializationInfo(finalized_column); - auto serialization = least_common_type->getSerialization(*info); - if constexpr (pretty_json) - serialization->serializeTextJSONPretty(finalized_column, row_num, ostr, settings, indent); + writeStringBinary(path, ostr); + if (auto it = typed_path_serializations.find(path); it != typed_path_serializations.end()) + it->second->serializeBinary(value, ostr, settings); else - serialization->serializeTextJSON(finalized_column, row_num, ostr, settings); - return; + dynamic_serialization->serializeBinary(value, ostr, settings); + } +} + +void SerializationObject::serializeBinary(const IColumn & col, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_object = assert_cast(col); + const auto & typed_paths = column_object.getTypedPaths(); + const auto & dynamic_paths = column_object.getDynamicPaths(); + const auto & shared_data_offsets = column_object.getSharedDataOffsets(); + size_t offset = shared_data_offsets[ssize_t(row_num) - 1]; + size_t end = shared_data_offsets[ssize_t(row_num)]; + + /// Serialize number of paths and then pairs (path, value). + writeVarUInt(typed_paths.size() + dynamic_paths.size() + (end - offset), ostr); + + for (const auto & [path, column] : typed_paths) + { + writeStringBinary(path, ostr); + typed_path_serializations.at(path)->serializeBinary(*column, row_num, ostr, settings); } - size_t ind = row_num; - if (ind < subcolumn.getNumberOfDefaultsInPrefix()) + for (const auto & [path, column] : dynamic_paths) { - /// Suboptimal, but it should happen rarely. - auto tmp_column = subcolumn.getLeastCommonType()->createColumn(); - tmp_column->insertDefault(); - - auto info = least_common_type->getSerializationInfo(*tmp_column); - auto serialization = least_common_type->getSerialization(*info); - if constexpr (pretty_json) - serialization->serializeTextJSONPretty(*tmp_column, 0, ostr, settings, indent); - else - serialization->serializeTextJSON(*tmp_column, 0, ostr, settings); - return; + writeStringBinary(path, ostr); + dynamic_serialization->serializeBinary(*column, row_num, ostr, settings); } - ind -= subcolumn.getNumberOfDefaultsInPrefix(); - for (const auto & part : subcolumn.getData()) + const auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + for (size_t i = offset; i != end; ++i) { - if (ind < part->size()) + writeStringBinary(shared_data_paths->getDataAt(i), ostr); + auto value = shared_data_values->getDataAt(i); + ostr.write(value.data, value.size); + } +} + +void SerializationObject::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const +{ + Object object; + size_t number_of_paths; + readVarUInt(number_of_paths, istr); + /// Read pairs (path, value). + for (size_t i = 0; i != number_of_paths; ++i) + { + String path; + readStringBinary(path, istr); + if (!shouldSkipPath(path)) { - auto part_type = getDataTypeByColumn(*part); - auto info = part_type->getSerializationInfo(*part); - auto serialization = part_type->getSerialization(*info); - if constexpr (pretty_json) - serialization->serializeTextJSONPretty(*part, ind, ostr, settings, indent); + if (auto it = typed_path_serializations.find(path); it != typed_path_serializations.end()) + it->second->deserializeBinary(object[path], istr, settings); else - serialization->serializeTextJSON(*part, ind, ostr, settings); - return; + dynamic_serialization->deserializeBinary(object[path], istr, settings); + } + else + { + /// Skip value of this path. + Field tmp; + dynamic_serialization->deserializeBinary(tmp, istr, settings); + } + } + + field = std::move(object); +} + +/// Restore column object to the state with previous size. +/// We can use it in case of an exception during deserialization. +void SerializationObject::restoreColumnObject(ColumnObject & column_object, size_t prev_size) +{ + auto & typed_paths = column_object.getTypedPaths(); + auto & dynamic_paths = column_object.getDynamicPaths(); + auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + auto & shared_data_offsets = column_object.getSharedDataOffsets(); + + for (auto & [_, column] : typed_paths) + { + if (column->size() > prev_size) + column->popBack(column->size() - prev_size); + } + + for (auto & [_, column] : dynamic_paths) + { + if (column->size() > prev_size) + column->popBack(column->size() - prev_size); + } + + if (shared_data_offsets.size() > prev_size) + shared_data_offsets.resize(prev_size); + size_t prev_shared_data_offset = shared_data_offsets.back(); + if (shared_data_paths->size() > prev_shared_data_offset) + shared_data_paths->popBack(shared_data_paths->size() - prev_shared_data_offset); + if (shared_data_values->size() > prev_shared_data_offset) + shared_data_values->popBack(shared_data_values->size() - prev_shared_data_offset); +} + +void SerializationObject::deserializeBinary(IColumn & col, ReadBuffer & istr, const FormatSettings & settings) const +{ + auto & column_object = assert_cast(col); + auto & typed_paths = column_object.getTypedPaths(); + auto & dynamic_paths = column_object.getDynamicPaths(); + auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + auto & shared_data_offsets = column_object.getSharedDataOffsets(); + + size_t number_of_paths; + readVarUInt(number_of_paths, istr); + std::vector> paths_and_values_for_shared_data; + size_t prev_size = column_object.size(); + try + { + /// Read pairs (path, value). + for (size_t i = 0; i != number_of_paths; ++i) + { + String path; + readStringBinary(path, istr); + if (!shouldSkipPath(path)) + { + /// Check if we have this path in typed paths. + if (auto typed_it = typed_path_serializations.find(path); typed_it != typed_path_serializations.end()) + { + auto & typed_column = typed_paths[path]; + /// Check if we already had this path. + if (typed_column->size() > prev_size) + { + if (!settings.json.type_json_skip_duplicated_paths) + throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of Object type: {}", path); + } + else + { + typed_it->second->deserializeBinary(*typed_column, istr, settings); + } + } + /// Check if we have this path in dynamc paths. + else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) + { + /// Check if we already had this path. + if (dynamic_it->second->size() > prev_size) + { + if (!settings.json.type_json_skip_duplicated_paths) + throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of Object type: {}", path); + } + + dynamic_serialization->deserializeBinary(*dynamic_it->second, istr, settings); + } + /// Try to add a new dynamic paths. + else if (auto * dynamic_column = column_object.tryToAddNewDynamicPath(path)) + { + dynamic_serialization->deserializeBinary(*dynamic_column, istr, settings); + } + /// Otherwise this path should go to shared data. + else + { + auto tmp_dynamic_column = ColumnDynamic::create(); + tmp_dynamic_column->reserve(1); + String value; + readParsedValueIntoString(value, istr, [&](ReadBuffer & buf){ dynamic_serialization->deserializeBinary(*tmp_dynamic_column, buf, settings); }); + paths_and_values_for_shared_data.emplace_back(std::move(path), std::move(value)); + } + } + else + { + /// Skip value of this path. + Field tmp; + dynamic_serialization->deserializeBinary(tmp, istr, settings); + } } - ind -= part->size(); + std::sort(paths_and_values_for_shared_data.begin(), paths_and_values_for_shared_data.end()); + for (size_t i = 0; i != paths_and_values_for_shared_data.size(); ++i) + { + const auto & [path, value] = paths_and_values_for_shared_data[i]; + if (i != 0 && path == paths_and_values_for_shared_data[i - 1].first) + { + if (!settings.json.type_json_skip_duplicated_paths) + throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of Object type: {}", path); + } + else + { + shared_data_paths->insertData(path.data(), path.size()); + shared_data_values->insertData(value.data(), value.size()); + } + } + shared_data_offsets.push_back(shared_data_paths->size()); } - - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for text serialization is out of range", row_num); -} - -template -void SerializationObject::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeTextImpl(column, row_num, ostr, settings); -} - -template -void SerializationObject::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - WriteBufferFromOwnString ostr_str; - serializeTextImpl(column, row_num, ostr_str, settings); - writeEscapedString(ostr_str.str(), ostr); -} - -template -void SerializationObject::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - WriteBufferFromOwnString ostr_str; - serializeTextImpl(column, row_num, ostr_str, settings); - writeQuotedString(ostr_str.str(), ostr); -} - -template -void SerializationObject::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeTextImpl(column, row_num, ostr, settings); -} - -template -void SerializationObject::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - WriteBufferFromOwnString ostr_str; - serializeTextImpl(column, row_num, ostr_str, settings); - writeCSVString(ostr_str.str(), ostr); -} - -template -void SerializationObject::serializeTextMarkdown( - const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - if (settings.markdown.escape_special_characters) + catch (...) { - WriteBufferFromOwnString ostr_str; - serializeTextImpl(column, row_num, ostr_str, settings); - writeMarkdownEscapedString(ostr_str.str(), ostr); - } - else - { - serializeTextEscaped(column, row_num, ostr, settings); + restoreColumnObject(column_object, prev_size); + throw; } } -template -void SerializationObject::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const +SerializationPtr SerializationObject::TypedPathSubcolumnCreator::create(const DB::SerializationPtr & prev) const { - const auto & column_object = assert_cast(column); - const auto & subcolumns = column_object.getSubcolumns(); - - writeCString("{\n", ostr); - for (auto it = subcolumns.begin(); it != subcolumns.end(); ++it) - { - const auto & entry = *it; - if (it != subcolumns.begin()) - writeCString(",\n", ostr); - - writeChar(' ', (indent + 1) * 4, ostr); - writeDoubleQuoted(entry->path.getPath(), ostr); - writeCString(": ", ostr); - serializeTextFromSubcolumn(entry->data, row_num, ostr, settings, indent + 1); - } - writeChar('\n', ostr); - writeChar(' ', indent * 4, ostr); - writeChar('}', ostr); -} - - -SerializationPtr getObjectSerialization(const String & schema_format) -{ - if (schema_format == "json") - { -#if USE_SIMDJSON - return std::make_shared>>(); -#elif USE_RAPIDJSON - return std::make_shared>>(); -#else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "To use data type Object with JSON format ClickHouse should be built with Simdjson or Rapidjson"); -#endif - } - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown schema format '{}'", schema_format); + return std::make_shared(prev, path); } } + diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 4cb7d0ab6a8..8702b0c7f64 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -1,34 +1,44 @@ #pragma once #include -#include -#include +#include +#include namespace DB { -/** Serialization for data type Object. - * Supported only text serialization/deserialization. - * and binary bulk serialization/deserialization without position independent - * encoding, i.e. serialization/deserialization into Native format. - */ -template +class SerializationObjectDynamicPath; +class SerializationSubObject; + +/// Class for binary serialization/deserialization of an Object type (currently only JSON). class SerializationObject : public ISerialization { public: - /** In Native format ColumnObject can be serialized - * in two formats: as Tuple or as String. - * The format is the following: - * - * 1 byte -- 0 if Tuple, 1 if String. - * [type_name] -- Only for tuple serialization. - * ... data of internal column ... - * - * ClickHouse client serializazes objects as tuples. - * String serialization exists for clients, which cannot - * do parsing by themselves and they can send raw data as - * string. It will be parsed on the server side. - */ + /// Serialization can change in future. Let's introduce serialization version. + struct ObjectSerializationVersion + { + enum Value + { + BASIC = 0, + }; + + Value value; + + static void checkVersion(UInt64 version); + + explicit ObjectSerializationVersion(UInt64 version); + }; + + SerializationObject( + std::unordered_map typed_path_serializations_, + const std::unordered_set & paths_to_skip_, + const std::vector & path_prefixes_to_skip_, + const std::vector & path_regexps_to_skip_); + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( const IColumn & column, @@ -63,59 +73,54 @@ public: void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; - void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - void serializeTextMarkdown(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - - void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + static void restoreColumnObject(ColumnObject & column_object, size_t prev_size); private: - enum class BinarySerializationKind : UInt8 + friend SerializationObjectDynamicPath; + friend SerializationSubObject; + + /// State of an Object structure. Can be also used during deserializing of Object subcolumns. + struct DeserializeBinaryBulkStateObjectStructure : public ISerialization::DeserializeBinaryBulkState { - TUPLE = 0, - STRING = 1, + ObjectSerializationVersion structure_version; + std::vector sorted_dynamic_paths; + std::unordered_set dynamic_paths; + /// Paths statistics. Map (dynamic path) -> (number of non-null values in this path). + ColumnObject::Statistics statistics = {.source = ColumnObject::Statistics::Source::READ, .data = {}}; + + explicit DeserializeBinaryBulkStateObjectStructure(UInt64 structure_version_) : structure_version(structure_version_) {} }; - struct SerializeStateObject; - struct DeserializeStateObject; - - void deserializeBinaryBulkFromString( - ColumnObject & column_object, - size_t limit, + static DeserializeBinaryBulkStatePtr deserializeObjectStructureStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeStateObject & state, - SubstreamsCache * cache) const; + SubstreamsDeserializeStatesCache * cache); - void deserializeBinaryBulkFromTuple( - ColumnObject & column_object, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeStateObject & state, - SubstreamsCache * cache) const; + /// Shared data has type Array(Tuple(String, String)). + static const DataTypePtr & getTypeOfSharedData(); - template - void checkSerializationIsSupported(const TSettings & settings) const; + struct TypedPathSubcolumnCreator : public ISubcolumnCreator + { + String path; - template - void deserializeTextImpl(IColumn & column, Reader && reader) const; + TypedPathSubcolumnCreator(const String & path_) : path(path_) {} - void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + DataTypePtr create(const DataTypePtr & prev) const override { return prev; } + ColumnPtr create(const ColumnPtr & prev) const override { return prev; } + SerializationPtr create(const SerializationPtr & prev) const override; + }; - template - void serializeTextFromSubcolumn(const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent = 0) const; +protected: + bool shouldSkipPath(const String & path) const; - /// Pool of parser objects to make SerializationObject thread safe. - mutable SimpleObjectPool parsers_pool; + std::unordered_map typed_path_serializations; + std::unordered_set paths_to_skip; + std::vector path_prefixes_to_skip; + std::list path_regexps_to_skip; + SerializationPtr dynamic_serialization; + +private: + std::vector sorted_typed_paths; + SerializationPtr shared_data_serialization; }; -SerializationPtr getObjectSerialization(const String & schema_format); - } diff --git a/src/DataTypes/Serializations/SerializationObjectDeprecated.cpp b/src/DataTypes/Serializations/SerializationObjectDeprecated.cpp new file mode 100644 index 00000000000..4e9ebf6c03d --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectDeprecated.cpp @@ -0,0 +1,586 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int INCORRECT_DATA; + extern const int CANNOT_READ_ALL_DATA; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int CANNOT_PARSE_TEXT; + extern const int EXPERIMENTAL_FEATURE_ERROR; +} + +template +template +void SerializationObjectDeprecated::deserializeTextImpl(IColumn & column, Reader && reader) const +{ + auto & column_object = assert_cast(column); + + String buf; + reader(buf); + std::optional result; + + /// Treat empty string as an empty object + /// for better CAST from String to Object. + if (!buf.empty()) + { + auto parser = parsers_pool.get([] { return new Parser; }); + result = parser->parse(buf.data(), buf.size()); + } + else + { + result = ParseResult{}; + } + + if (!result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse object"); + + auto & [paths, values] = *result; + assert(paths.size() == values.size()); + + size_t old_column_size = column_object.size(); + for (size_t i = 0; i < paths.size(); ++i) + { + auto field_info = getFieldInfo(values[i]); + if (field_info.need_fold_dimension) + values[i] = applyVisitor(FieldVisitorFoldDimension(field_info.num_dimensions), std::move(values[i])); + if (isNothing(field_info.scalar_type)) + continue; + + if (!column_object.hasSubcolumn(paths[i])) + { + if (paths[i].hasNested()) + column_object.addNestedSubcolumn(paths[i], field_info, old_column_size); + else + column_object.addSubcolumn(paths[i], old_column_size); + } + + auto & subcolumn = column_object.getSubcolumn(paths[i]); + assert(subcolumn.size() == old_column_size); + + subcolumn.insert(std::move(values[i]), std::move(field_info)); + } + + /// Insert default values to missed subcolumns. + const auto & subcolumns = column_object.getSubcolumns(); + for (const auto & entry : subcolumns) + { + if (entry->data.size() == old_column_size) + { + bool inserted = column_object.tryInsertDefaultFromNested(entry); + if (!inserted) + entry->data.insertDefault(); + } + } + + column_object.incrementNumRows(); +} + +template +void SerializationObjectDeprecated::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + deserializeTextImpl(column, [&](String & s) { readStringInto(s, istr); }); +} + +template +void SerializationObjectDeprecated::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextImpl(column, [&](String & s) { settings.tsv.crlf_end_of_line_input ? readEscapedStringCRLF(s, istr) : readEscapedString(s, istr); }); +} + +template +void SerializationObjectDeprecated::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + deserializeTextImpl(column, [&](String & s) { readQuotedStringInto(s, istr); }); +} + +template +void SerializationObjectDeprecated::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + deserializeTextImpl(column, [&](String & s) { Parser::readJSON(s, istr); }); +} + +template +void SerializationObjectDeprecated::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextImpl(column, [&](String & s) { readCSVStringInto(s, istr, settings.csv); }); +} + +template +template +void SerializationObjectDeprecated::checkSerializationIsSupported(const TSettings & settings) const +{ + if (settings.position_independent_encoding) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DataTypeObject doesn't support serialization with position independent encoding"); +} + +template +struct SerializationObjectDeprecated::SerializeStateObject : public ISerialization::SerializeBinaryBulkState +{ + DataTypePtr nested_type; + SerializationPtr nested_serialization; + SerializeBinaryBulkStatePtr nested_state; +}; + +template +struct SerializationObjectDeprecated::DeserializeStateObject : public ISerialization::DeserializeBinaryBulkState +{ + BinarySerializationKind kind; + DataTypePtr nested_type; + SerializationPtr nested_serialization; + DeserializeBinaryBulkStatePtr nested_state; +}; + +template +void SerializationObjectDeprecated::serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + checkSerializationIsSupported(settings); + if (state) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DataTypeObject doesn't support serialization with non-trivial state"); + + const auto & column_object = assert_cast(column); + if (!column_object.isFinalized()) + { + auto finalized = column_object.cloneFinalized(); + serializeBinaryBulkStatePrefix(*finalized, settings, state); + return; + } + + settings.path.push_back(Substream::DeprecatedObjectStructure); + auto * stream = settings.getter(settings.path); + + if (!stream) + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Missing stream for kind of binary serialization"); + + auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); + + writeIntBinary(static_cast(BinarySerializationKind::TUPLE), *stream); + writeStringBinary(tuple_type->getName(), *stream); + + auto state_object = std::make_shared(); + state_object->nested_type = tuple_type; + state_object->nested_serialization = tuple_type->getDefaultSerialization(); + + settings.path.back() = Substream::DeprecatedObjectData; + state_object->nested_serialization->serializeBinaryBulkStatePrefix(*tuple_column, settings, state_object->nested_state); + + state = std::move(state_object); + settings.path.pop_back(); +} + +template +void SerializationObjectDeprecated::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + checkSerializationIsSupported(settings); + auto * state_object = checkAndGetState(state); + + settings.path.push_back(Substream::DeprecatedObjectData); + state_object->nested_serialization->serializeBinaryBulkStateSuffix(settings, state_object->nested_state); + settings.path.pop_back(); +} + +template +void SerializationObjectDeprecated::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const +{ + checkSerializationIsSupported(settings); + if (state) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DataTypeObject doesn't support serialization with non-trivial state"); + + settings.path.push_back(Substream::DeprecatedObjectStructure); + auto * stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!stream) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read kind of binary serialization of DataTypeObject, because its stream is missing"); + + UInt8 kind_raw; + readIntBinary(kind_raw, *stream); + auto kind = magic_enum::enum_cast(kind_raw); + if (!kind) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Unknown binary serialization kind of Object: {}", std::to_string(kind_raw)); + + auto state_object = std::make_shared(); + state_object->kind = *kind; + + if (state_object->kind == BinarySerializationKind::TUPLE) + { + String data_type_name; + readStringBinary(data_type_name, *stream); + state_object->nested_type = DataTypeFactory::instance().get(data_type_name); + state_object->nested_serialization = state_object->nested_type->getDefaultSerialization(); + + if (!isTuple(state_object->nested_type)) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Data of type Object should be written as Tuple, got: {}", data_type_name); + } + else if (state_object->kind == BinarySerializationKind::STRING) + { + state_object->nested_type = std::make_shared(); + state_object->nested_serialization = std::make_shared(); + } + else + { + throw Exception(ErrorCodes::INCORRECT_DATA, + "Unknown binary serialization kind of Object: {}", std::to_string(kind_raw)); + } + + settings.path.push_back(Substream::DeprecatedObjectData); + state_object->nested_serialization->deserializeBinaryBulkStatePrefix(settings, state_object->nested_state, cache); + settings.path.pop_back(); + + state = std::move(state_object); +} + +template +void SerializationObjectDeprecated::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + checkSerializationIsSupported(settings); + const auto & column_object = assert_cast(column); + auto * state_object = checkAndGetState(state); + + if (!column_object.isFinalized()) + { + auto finalized = column_object.cloneFinalized(); + serializeBinaryBulkWithMultipleStreams(*finalized, offset, limit, settings, state); + return; + } + + auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); + + if (!state_object->nested_type->equals(*tuple_type)) + { + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, + "Types of internal column of Object mismatched. Expected: {}, Got: {}", + state_object->nested_type->getName(), tuple_type->getName()); + } + + settings.path.push_back(Substream::DeprecatedObjectData); + if (auto * stream = settings.getter(settings.path)) + { + state_object->nested_serialization->serializeBinaryBulkWithMultipleStreams( + *tuple_column, offset, limit, settings, state_object->nested_state); + } + + settings.path.pop_back(); +} + +template +void SerializationObjectDeprecated::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + checkSerializationIsSupported(settings); + if (!column->empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DataTypeObject cannot be deserialized to non-empty column"); + + auto mutable_column = column->assumeMutable(); + auto & column_object = assert_cast(*mutable_column); + auto * state_object = checkAndGetState(state); + + settings.path.push_back(Substream::DeprecatedObjectData); + if (state_object->kind == BinarySerializationKind::STRING) + deserializeBinaryBulkFromString(column_object, limit, settings, *state_object, cache); + else + deserializeBinaryBulkFromTuple(column_object, limit, settings, *state_object, cache); + + settings.path.pop_back(); + column_object.checkConsistency(); + column_object.finalize(); + column = std::move(mutable_column); +} + +template +void SerializationObjectDeprecated::deserializeBinaryBulkFromString( + ColumnObjectDeprecated & column_object, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeStateObject & state, + SubstreamsCache * cache) const +{ + ColumnPtr column_string = state.nested_type->createColumn(); + state.nested_serialization->deserializeBinaryBulkWithMultipleStreams( + column_string, limit, settings, state.nested_state, cache); + + size_t input_rows_count = column_string->size(); + column_object.reserve(input_rows_count); + + FormatSettings format_settings; + for (size_t i = 0; i < input_rows_count; ++i) + { + const auto & val = column_string->getDataAt(i); + ReadBufferFromMemory read_buffer(val.data, val.size); + deserializeWholeText(column_object, read_buffer, format_settings); + + if (!read_buffer.eof()) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, + "Cannot parse string to column Object. Expected eof"); + } +} + +template +void SerializationObjectDeprecated::deserializeBinaryBulkFromTuple( + ColumnObjectDeprecated & column_object, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeStateObject & state, + SubstreamsCache * cache) const +{ + ColumnPtr column_tuple = state.nested_type->createColumn(); + state.nested_serialization->deserializeBinaryBulkWithMultipleStreams( + column_tuple, limit, settings, state.nested_state, cache); + + auto [tuple_paths, tuple_types] = flattenTuple(state.nested_type); + auto flattened_tuple = flattenTuple(column_tuple); + const auto & tuple_columns = assert_cast(*flattened_tuple).getColumns(); + + assert(tuple_paths.size() == tuple_types.size()); + size_t num_subcolumns = tuple_paths.size(); + + if (tuple_columns.size() != num_subcolumns) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Inconsistent type ({}) and column ({}) while reading column of type Object", + state.nested_type->getName(), column_tuple->getName()); + + for (size_t i = 0; i < num_subcolumns; ++i) + column_object.addSubcolumn(tuple_paths[i], tuple_columns[i]->assumeMutable()); +} + +template +void SerializationObjectDeprecated::serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObjectDeprecated"); +} + +template +void SerializationObjectDeprecated::deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObjectDeprecated"); +} + +template +void SerializationObjectDeprecated::serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObjectDeprecated"); +} + +template +void SerializationObjectDeprecated::deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObjectDeprecated"); +} + +/// TODO: use format different of JSON in serializations. + +template +void SerializationObjectDeprecated::serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_object = assert_cast(column); + const auto & subcolumns = column_object.getSubcolumns(); + + writeChar('{', ostr); + for (auto it = subcolumns.begin(); it != subcolumns.end(); ++it) + { + const auto & entry = *it; + if (it != subcolumns.begin()) + writeCString(",", ostr); + + writeDoubleQuoted(entry->path.getPath(), ostr); + writeChar(':', ostr); + serializeTextFromSubcolumn(entry->data, row_num, ostr, settings); + } + writeChar('}', ostr); +} + +template +template +void SerializationObjectDeprecated::serializeTextFromSubcolumn( + const ColumnObjectDeprecated::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const +{ + const auto & least_common_type = subcolumn.getLeastCommonType(); + + if (subcolumn.isFinalized()) + { + const auto & finalized_column = subcolumn.getFinalizedColumn(); + auto info = least_common_type->getSerializationInfo(finalized_column); + auto serialization = least_common_type->getSerialization(*info); + if constexpr (pretty_json) + serialization->serializeTextJSONPretty(finalized_column, row_num, ostr, settings, indent); + else + serialization->serializeTextJSON(finalized_column, row_num, ostr, settings); + return; + } + + size_t ind = row_num; + if (ind < subcolumn.getNumberOfDefaultsInPrefix()) + { + /// Suboptimal, but it should happen rarely. + auto tmp_column = subcolumn.getLeastCommonType()->createColumn(); + tmp_column->insertDefault(); + + auto info = least_common_type->getSerializationInfo(*tmp_column); + auto serialization = least_common_type->getSerialization(*info); + if constexpr (pretty_json) + serialization->serializeTextJSONPretty(*tmp_column, 0, ostr, settings, indent); + else + serialization->serializeTextJSON(*tmp_column, 0, ostr, settings); + return; + } + + ind -= subcolumn.getNumberOfDefaultsInPrefix(); + for (const auto & part : subcolumn.getData()) + { + if (ind < part->size()) + { + auto part_type = getDataTypeByColumn(*part); + auto info = part_type->getSerializationInfo(*part); + auto serialization = part_type->getSerialization(*info); + if constexpr (pretty_json) + serialization->serializeTextJSONPretty(*part, ind, ostr, settings, indent); + else + serialization->serializeTextJSON(*part, ind, ostr, settings); + return; + } + + ind -= part->size(); + } + + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for text serialization is out of range", row_num); +} + +template +void SerializationObjectDeprecated::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeTextImpl(column, row_num, ostr, settings); +} + +template +void SerializationObjectDeprecated::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString ostr_str; + serializeTextImpl(column, row_num, ostr_str, settings); + writeEscapedString(ostr_str.str(), ostr); +} + +template +void SerializationObjectDeprecated::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString ostr_str; + serializeTextImpl(column, row_num, ostr_str, settings); + writeQuotedString(ostr_str.str(), ostr); +} + +template +void SerializationObjectDeprecated::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeTextImpl(column, row_num, ostr, settings); +} + +template +void SerializationObjectDeprecated::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + WriteBufferFromOwnString ostr_str; + serializeTextImpl(column, row_num, ostr_str, settings); + writeCSVString(ostr_str.str(), ostr); +} + +template +void SerializationObjectDeprecated::serializeTextMarkdown( + const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + if (settings.markdown.escape_special_characters) + { + WriteBufferFromOwnString ostr_str; + serializeTextImpl(column, row_num, ostr_str, settings); + writeMarkdownEscapedString(ostr_str.str(), ostr); + } + else + { + serializeTextEscaped(column, row_num, ostr, settings); + } +} + +template +void SerializationObjectDeprecated::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const +{ + const auto & column_object = assert_cast(column); + const auto & subcolumns = column_object.getSubcolumns(); + + writeCString("{\n", ostr); + for (auto it = subcolumns.begin(); it != subcolumns.end(); ++it) + { + const auto & entry = *it; + if (it != subcolumns.begin()) + writeCString(",\n", ostr); + + writeChar(' ', (indent + 1) * 4, ostr); + writeDoubleQuoted(entry->path.getPath(), ostr); + writeCString(": ", ostr); + serializeTextFromSubcolumn(entry->data, row_num, ostr, settings, indent + 1); + } + writeChar('\n', ostr); + writeChar(' ', indent * 4, ostr); + writeChar('}', ostr); +} + + +SerializationPtr getObjectSerialization(const String & schema_format) +{ + if (schema_format == "json") + { +#if USE_SIMDJSON + return std::make_shared>>(); +#elif USE_RAPIDJSON + return std::make_shared>>(); +#else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "To use data type Object with JSON format ClickHouse should be built with Simdjson or Rapidjson"); +#endif + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown schema format '{}'", schema_format); +} + +} diff --git a/src/DataTypes/Serializations/SerializationObjectDeprecated.h b/src/DataTypes/Serializations/SerializationObjectDeprecated.h new file mode 100644 index 00000000000..c209f946850 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectDeprecated.h @@ -0,0 +1,121 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/** Serialization for data type Object (deprecated). + * Supported only text serialization/deserialization. + * and binary bulk serialization/deserialization without position independent + * encoding, i.e. serialization/deserialization into Native format. + */ +template +class SerializationObjectDeprecated : public ISerialization +{ +public: + /** In Native format ColumnObjectDeprecated can be serialized + * in two formats: as Tuple or as String. + * The format is the following: + * + * 1 byte -- 0 if Tuple, 1 if String. + * [type_name] -- Only for tuple serialization. + * ... data of internal column ... + * + * ClickHouse client serializazes objects as tuples. + * String serialization exists for clients, which cannot + * do parsing by themselves and they can send raw data as + * string. It will be parsed on the server side. + */ + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextMarkdown(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + +private: + enum class BinarySerializationKind : UInt8 + { + TUPLE = 0, + STRING = 1, + }; + + struct SerializeStateObject; + struct DeserializeStateObject; + + void deserializeBinaryBulkFromString( + ColumnObjectDeprecated & column_object, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeStateObject & state, + SubstreamsCache * cache) const; + + void deserializeBinaryBulkFromTuple( + ColumnObjectDeprecated & column_object, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeStateObject & state, + SubstreamsCache * cache) const; + + template + void checkSerializationIsSupported(const TSettings & settings) const; + + template + void deserializeTextImpl(IColumn & column, Reader && reader) const; + + void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + + template + void serializeTextFromSubcolumn(const ColumnObjectDeprecated::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent = 0) const; + + /// Pool of parser objects to make SerializationObjectDeprecated thread safe. + mutable SimpleObjectPool parsers_pool; +}; + +SerializationPtr getObjectSerialization(const String & schema_format); + +} diff --git a/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp b/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp new file mode 100644 index 00000000000..3a635671978 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp @@ -0,0 +1,180 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +SerializationObjectDynamicPath::SerializationObjectDynamicPath( + const DB::SerializationPtr & nested_, const String & path_, const String & path_subcolumn_, size_t max_dynamic_types_) + : SerializationWrapper(nested_) + , path(path_) + , path_subcolumn(path_subcolumn_) + , dynamic_serialization(std::make_shared()) + , shared_data_serialization(SerializationObject::getTypeOfSharedData()->getDefaultSerialization()) + , max_dynamic_types(max_dynamic_types_) +{ +} + +struct DeserializeBinaryBulkStateObjectDynamicPath : public ISerialization::DeserializeBinaryBulkState +{ + ISerialization::DeserializeBinaryBulkStatePtr structure_state; + ISerialization::DeserializeBinaryBulkStatePtr nested_state; + bool read_from_shared_data; + ColumnPtr shared_data; +}; + +void SerializationObjectDynamicPath::enumerateStreams( + DB::ISerialization::EnumerateStreamsSettings & settings, + const DB::ISerialization::StreamCallback & callback, + const DB::ISerialization::SubstreamData & data) const +{ + settings.path.push_back(Substream::ObjectStructure); + callback(settings.path); + settings.path.pop_back(); + + const auto * deserialize_state = data.deserialize_state ? checkAndGetState(data.deserialize_state) : nullptr; + + /// We cannot enumerate anything if we don't have deserialization state, as we don't know the dynamic structure. + if (!deserialize_state) + return; + + settings.path.push_back(Substream::ObjectData); + const auto * structure_state = checkAndGetState(deserialize_state->structure_state); + /// Check if we have our path in dynamic paths. + if (structure_state->dynamic_paths.contains(path)) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + auto path_data = SubstreamData(nested_serialization) + .withType(data.type) + .withColumn(data.column) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state->nested_state); + settings.path.back().data = path_data; + nested_serialization->enumerateStreams(settings, callback, path_data); + settings.path.pop_back(); + } + /// Otherwise we will have to read all shared data and try to find our path there. + else + { + settings.path.push_back(Substream::ObjectSharedData); + auto shared_data_substream_data = SubstreamData(shared_data_serialization) + .withType(data.type ? SerializationObject::getTypeOfSharedData() : nullptr) + .withColumn(data.column ? SerializationObject::getTypeOfSharedData()->createColumn() : nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state->nested_state); + settings.path.back().data = shared_data_substream_data; + shared_data_serialization->enumerateStreams(settings, callback, shared_data_substream_data); + settings.path.pop_back(); + } + + settings.path.pop_back(); +} + +void SerializationObjectDynamicPath::serializeBinaryBulkStatePrefix(const IColumn &, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStatePrefix is not implemented for SerializationObjectDynamicPath"); +} + +void SerializationObjectDynamicPath::serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationObjectDynamicPath"); +} + +void SerializationObjectDynamicPath::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const +{ + auto structure_state = SerializationObject::deserializeObjectStructureStatePrefix(settings, cache); + if (!structure_state) + return; + + auto dynamic_path_state = std::make_shared(); + dynamic_path_state->structure_state = std::move(structure_state); + /// Remember if we need to read from shared data or we have this path in dynamic paths. + dynamic_path_state->read_from_shared_data = !checkAndGetState(dynamic_path_state->structure_state)->dynamic_paths.contains(path); + settings.path.push_back(Substream::ObjectData); + if (dynamic_path_state->read_from_shared_data) + { + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_path_state->nested_state, cache); + settings.path.pop_back(); + } + else + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + nested_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_path_state->nested_state, cache); + settings.path.pop_back(); + } + + settings.path.pop_back(); + state = std::move(dynamic_path_state); +} + +void SerializationObjectDynamicPath::serializeBinaryBulkWithMultipleStreams(const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkWithMultipleStreams is not implemented for SerializationObjectDynamicPath"); +} + +void SerializationObjectDynamicPath::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & result_column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + if (!state) + return; + + auto * dynamic_path_state = checkAndGetState(state); + settings.path.push_back(Substream::ObjectData); + /// Check if we don't need to read shared data. In this case just read data from dynamic path. + if (!dynamic_path_state->read_from_shared_data) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + nested_serialization->deserializeBinaryBulkWithMultipleStreams(result_column, limit, settings, dynamic_path_state->nested_state, cache); + settings.path.pop_back(); + } + /// Otherwise, read the whole shared data column and extract requested path from it. + else + { + settings.path.push_back(Substream::ObjectSharedData); + /// Initialize shared_data column if needed. + if (result_column->empty()) + dynamic_path_state->shared_data = SerializationObject::getTypeOfSharedData()->createColumn(); + size_t prev_size = result_column->size(); + shared_data_serialization->deserializeBinaryBulkWithMultipleStreams(dynamic_path_state->shared_data, limit, settings, dynamic_path_state->nested_state, cache); + /// If we need to read a subcolumn from Dynamic column, create an empty Dynamic column, fill it and extract subcolumn. + MutableColumnPtr dynamic_column = path_subcolumn.empty() ? result_column->assumeMutable() : ColumnDynamic::create(max_dynamic_types)->getPtr(); + /// Check if we don't have any paths in shared data in current range. + const auto & offsets = assert_cast(*dynamic_path_state->shared_data).getOffsets(); + if (offsets.back() == offsets[ssize_t(prev_size) - 1]) + dynamic_column->insertManyDefaults(limit); + else + ColumnObject::fillPathColumnFromSharedData(*dynamic_column, path, dynamic_path_state->shared_data, prev_size, dynamic_path_state->shared_data->size()); + + /// Extract subcolumn from Dynamic column if needed. + if (!path_subcolumn.empty()) + { + auto subcolumn = std::make_shared(max_dynamic_types)->getSubcolumn(path_subcolumn, dynamic_column->getPtr()); + result_column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size()); + } + + settings.path.pop_back(); + } + + settings.path.pop_back(); +} + +} diff --git a/src/DataTypes/Serializations/SerializationObjectDynamicPath.h b/src/DataTypes/Serializations/SerializationObjectDynamicPath.h new file mode 100644 index 00000000000..e11d0cded73 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectDynamicPath.h @@ -0,0 +1,58 @@ +#pragma once + +#include + +namespace DB +{ + +/// Serialization of dynamic Object paths. +/// For example, if we have type JSON(a.b UInt32, b.c String) and data {"a" : {"b" : 42}, "b" : {"c" : "Hello}, "c" : {"d" : [1, 2, 3]}, "d" : 42} +/// this class will be responsible for reading dynamic paths 'c.d' and 'd' as subcolumns. +/// Typed paths 'a.b' and 'b.c' are serialized in SerializationObjectTypedPath. +class SerializationObjectDynamicPath final : public SerializationWrapper +{ +public: + SerializationObjectDynamicPath(const SerializationPtr & nested_, const String & path_, const String & path_subcolumn_, size_t max_dynamic_types_); + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + +private: + String path; + String path_subcolumn; + SerializationPtr dynamic_serialization; + SerializationPtr shared_data_serialization; + size_t max_dynamic_types; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationObjectElement.cpp b/src/DataTypes/Serializations/SerializationObjectElement.cpp new file mode 100644 index 00000000000..35119cd0764 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectElement.cpp @@ -0,0 +1,3 @@ +// +// Created by Павел Круглов on 05/07/2024. +// diff --git a/src/DataTypes/Serializations/SerializationObjectElement.h b/src/DataTypes/Serializations/SerializationObjectElement.h new file mode 100644 index 00000000000..791ef2eecc5 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectElement.h @@ -0,0 +1,8 @@ +// +// Created by Павел Круглов on 05/07/2024. +// + +#ifndef CLICKHOUSE_SERIALIZATIONJSONELEMENT_H +#define CLICKHOUSE_SERIALIZATIONJSONELEMENT_H + +#endif //CLICKHOUSE_SERIALIZATIONJSONELEMENT_H diff --git a/src/DataTypes/Serializations/SerializationObjectTypedPath.cpp b/src/DataTypes/Serializations/SerializationObjectTypedPath.cpp new file mode 100644 index 00000000000..ef086d486f7 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectTypedPath.cpp @@ -0,0 +1,78 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + + +void SerializationObjectTypedPath::enumerateStreams( + DB::ISerialization::EnumerateStreamsSettings & settings, + const DB::ISerialization::StreamCallback & callback, + const DB::ISerialization::SubstreamData & data) const +{ + settings.path.push_back(Substream::ObjectData); + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + auto path_data = SubstreamData(nested_serialization) + .withType(data.type) + .withColumn(data.column) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(data.deserialize_state); + nested_serialization->enumerateStreams(settings, callback, path_data); + settings.path.pop_back(); + settings.path.pop_back(); +} + +void SerializationObjectTypedPath::serializeBinaryBulkStatePrefix(const IColumn &, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStatePrefix is not implemented for SerializationObjectTypedPath"); +} + +void SerializationObjectTypedPath::serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationObjectTypedPath"); +} + +void SerializationObjectTypedPath::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const +{ + settings.path.push_back(Substream::ObjectData); + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + nested_serialization->deserializeBinaryBulkStatePrefix(settings, state, cache); + settings.path.pop_back(); + settings.path.pop_back(); +} + +void SerializationObjectTypedPath::serializeBinaryBulkWithMultipleStreams(const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkWithMultipleStreams is not implemented for SerializationObjectTypedPath"); +} + +void SerializationObjectTypedPath::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & result_column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + settings.path.push_back(Substream::ObjectData); + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + nested_serialization->deserializeBinaryBulkWithMultipleStreams(result_column, limit, settings, state, cache); + settings.path.pop_back(); + settings.path.pop_back(); +} + +} diff --git a/src/DataTypes/Serializations/SerializationObjectTypedPath.h b/src/DataTypes/Serializations/SerializationObjectTypedPath.h new file mode 100644 index 00000000000..a893e07f798 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationObjectTypedPath.h @@ -0,0 +1,57 @@ +#pragma once + +#include + +namespace DB +{ + +/// Serialization of typed Object paths. +/// For example, for type JSON(a.b UInt32, b.c String) this serialization +/// will be used to read paths 'a.b' and 'b.c' as subcolumns. +class SerializationObjectTypedPath final : public SerializationWrapper +{ +public: + SerializationObjectTypedPath(const SerializationPtr & nested_, const String & path_) + : SerializationWrapper(nested_) + , path(path_) + { + } + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + +private: + String path; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationSubObject.cpp b/src/DataTypes/Serializations/SerializationSubObject.cpp new file mode 100644 index 00000000000..62ebd4669e1 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationSubObject.cpp @@ -0,0 +1,259 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +SerializationSubObject::SerializationSubObject( + const String & path_prefix_, const std::unordered_map & typed_paths_serializations_) + : path_prefix(path_prefix_) + , typed_paths_serializations(typed_paths_serializations_) + , dynamic_serialization(std::make_shared()) + , shared_data_serialization(SerializationObject::getTypeOfSharedData()->getDefaultSerialization()) +{ +} + +struct DeserializeBinaryBulkStateSubObject : public ISerialization::DeserializeBinaryBulkState +{ + std::unordered_map typed_path_states; + std::unordered_map dynamic_path_states; + std::vector dynamic_paths; + std::vector dynamic_sub_paths; + ISerialization::DeserializeBinaryBulkStatePtr shared_data_state; + ColumnPtr shared_data; +}; + +void SerializationSubObject::enumerateStreams( + DB::ISerialization::EnumerateStreamsSettings & settings, + const DB::ISerialization::StreamCallback & callback, + const DB::ISerialization::SubstreamData & data) const +{ + settings.path.push_back(Substream::ObjectStructure); + callback(settings.path); + settings.path.pop_back(); + + const auto * column_object = data.column ? &assert_cast(*data.column) : nullptr; + const auto * type_object = data.type ? &assert_cast(*data.type) : nullptr; + const auto * deserialize_state = data.deserialize_state ? checkAndGetState(data.deserialize_state) : nullptr; + + settings.path.push_back(Substream::ObjectData); + + /// typed_paths_serializations contains only typed paths with requested prefix from original Object column. + for (const auto & [path, serialization] : typed_paths_serializations) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + auto path_data = SubstreamData(serialization) + .withType(type_object ? type_object->getTypedPaths().at(path.substr(path_prefix.size() + 1)) : nullptr) + .withColumn(column_object ? column_object->getTypedPaths().at(path.substr(path_prefix.size() + 1)) : nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state ? deserialize_state->typed_path_states.at(path) : nullptr); + settings.path.back().data = path_data; + serialization->enumerateStreams(settings, callback, path_data); + settings.path.pop_back(); + } + + /// We will need to read shared data to find all paths with requested prefix. + settings.path.push_back(Substream::ObjectSharedData); + auto shared_data_substream_data = SubstreamData(shared_data_serialization) + .withType(data.type ? SerializationObject::getTypeOfSharedData() : nullptr) + .withColumn(data.column ? SerializationObject::getTypeOfSharedData()->createColumn() : nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(deserialize_state ? deserialize_state->shared_data_state : nullptr); + settings.path.back().data = shared_data_substream_data; + shared_data_serialization->enumerateStreams(settings, callback, shared_data_substream_data); + settings.path.pop_back(); + + /// If deserialize state is provided, enumerate streams for dynamic paths. + if (deserialize_state) + { + DataTypePtr type = std::make_shared(); + for (const auto & [path, state] : deserialize_state->dynamic_path_states) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + auto path_data = SubstreamData(dynamic_serialization) + .withType(type_object ? type : nullptr) + .withColumn(nullptr) + .withSerializationInfo(data.serialization_info) + .withDeserializeState(state); + settings.path.back().data = path_data; + dynamic_serialization->enumerateStreams(settings, callback, path_data); + settings.path.pop_back(); + } + } + + settings.path.pop_back(); +} + +void SerializationSubObject::serializeBinaryBulkStatePrefix(const IColumn &, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStatePrefix is not implemented for SerializationSubObject"); +} + +void SerializationSubObject::serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationSubObject"); +} + +namespace +{ + +/// Return sub-path by specified prefix. +/// For example, for prefix a.b: +/// a.b.c.d -> c.d, a.b.c -> c +String getSubPath(const String & path, const String & prefix) +{ + return path.substr(prefix.size() + 1); +} + +std::string_view getSubPath(const std::string_view & path, const String & prefix) +{ + return path.substr(prefix.size() + 1); +} + +} + +void SerializationSubObject::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const +{ + auto structure_state = SerializationObject::deserializeObjectStructureStatePrefix(settings, cache); + if (!structure_state) + return; + + auto sub_object_state = std::make_shared(); + settings.path.push_back(Substream::ObjectData); + for (const auto & [path, serialization] : typed_paths_serializations) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + serialization->deserializeBinaryBulkStatePrefix(settings, sub_object_state->typed_path_states[path], cache); + settings.path.pop_back(); + } + + for (const auto & dynamic_path : checkAndGetState(structure_state)->sorted_dynamic_paths) + { + /// Save only dynamic paths with requested prefix. + if (dynamic_path.starts_with(path_prefix) && dynamic_path.size() != path_prefix.size()) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = dynamic_path; + dynamic_serialization->deserializeBinaryBulkStatePrefix(settings, sub_object_state->dynamic_path_states[dynamic_path], cache); + settings.path.pop_back(); + sub_object_state->dynamic_paths.push_back(dynamic_path); + sub_object_state->dynamic_sub_paths.push_back(getSubPath(dynamic_path, path_prefix)); + } + } + + settings.path.push_back(Substream::ObjectSharedData); + shared_data_serialization->deserializeBinaryBulkStatePrefix(settings, sub_object_state->shared_data_state, cache); + settings.path.pop_back(); + + settings.path.pop_back(); + state = std::move(sub_object_state); +} + +void SerializationSubObject::serializeBinaryBulkWithMultipleStreams(const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkWithMultipleStreams is not implemented for SerializationSubObject"); +} + +void SerializationSubObject::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & result_column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + if (!state) + return; + + auto * sub_object_state = checkAndGetState(state); + auto mutable_column = result_column->assumeMutable(); + auto & column_object = assert_cast(*mutable_column); + /// If it's a new object column, set dynamic paths and statistics. + if (column_object.empty()) + column_object.setDynamicPaths(sub_object_state->dynamic_sub_paths); + + auto & typed_paths = column_object.getTypedPaths(); + auto & dynamic_paths = column_object.getDynamicPaths(); + + settings.path.push_back(Substream::ObjectData); + for (const auto & [path, serialization] : typed_paths_serializations) + { + settings.path.push_back(Substream::ObjectTypedPath); + settings.path.back().object_path_name = path; + serialization->deserializeBinaryBulkWithMultipleStreams(typed_paths[getSubPath(path, path_prefix)], limit, settings, sub_object_state->typed_path_states[path], cache); + settings.path.pop_back(); + } + + for (const auto & path : sub_object_state->dynamic_paths) + { + settings.path.push_back(Substream::ObjectDynamicPath); + settings.path.back().object_path_name = path; + dynamic_serialization->deserializeBinaryBulkWithMultipleStreams(dynamic_paths[getSubPath(path, path_prefix)], limit, settings, sub_object_state->dynamic_path_states[path], cache); + settings.path.pop_back(); + } + + settings.path.push_back(Substream::ObjectSharedData); + /// If it's a new object column, reinitialize column for shared data. + if (result_column->empty()) + sub_object_state->shared_data = SerializationObject::getTypeOfSharedData()->createColumn(); + size_t prev_size = column_object.size(); + shared_data_serialization->deserializeBinaryBulkWithMultipleStreams(sub_object_state->shared_data, limit, settings, sub_object_state->shared_data_state, cache); + settings.path.pop_back(); + + auto & sub_object_shared_data = column_object.getSharedDataColumn(); + const auto & offsets = assert_cast(*sub_object_state->shared_data).getOffsets(); + /// Check if there is no data in shared data in current range. + if (offsets.back() == offsets[ssize_t(prev_size) - 1]) + { + sub_object_shared_data.insertManyDefaults(limit); + } + else + { + const auto & shared_data_array = assert_cast(*sub_object_state->shared_data); + const auto & shared_data_offsets = shared_data_array.getOffsets(); + const auto & shared_data_tuple = assert_cast(shared_data_array.getData()); + const auto & shared_data_paths = assert_cast(shared_data_tuple.getColumn(0)); + const auto & shared_data_values = assert_cast(shared_data_tuple.getColumn(1)); + + auto & sub_object_data_offsets = column_object.getSharedDataOffsets(); + auto [sub_object_shared_data_paths, sub_object_shared_data_values] = column_object.getSharedDataPathsAndValues(); + StringRef prefix_ref(path_prefix); + for (size_t i = prev_size; i != shared_data_offsets.size(); ++i) + { + size_t start = shared_data_offsets[ssize_t(i) - 1]; + size_t end = shared_data_offsets[ssize_t(i)]; + size_t lower_bound_index = ColumnObject::findPathLowerBoundInSharedData(prefix_ref, shared_data_paths, start, end); + for (; lower_bound_index != end; ++lower_bound_index) + { + auto path = shared_data_paths.getDataAt(lower_bound_index).toView(); + if (!path.starts_with(path_prefix)) + break; + + /// Don't include path that is equal to the prefix. + if (path.size() != path_prefix.size()) + { + auto sub_path = getSubPath(path, path_prefix); + sub_object_shared_data_paths->insertData(sub_path.data(), sub_path.size()); + sub_object_shared_data_values->insertFrom(shared_data_values, lower_bound_index); + } + } + sub_object_data_offsets.push_back(sub_object_shared_data_paths->size()); + } + } + settings.path.pop_back(); +} + +} diff --git a/src/DataTypes/Serializations/SerializationSubObject.h b/src/DataTypes/Serializations/SerializationSubObject.h new file mode 100644 index 00000000000..8d037720f1f --- /dev/null +++ b/src/DataTypes/Serializations/SerializationSubObject.h @@ -0,0 +1,72 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Serialization of a sub-object Object subcolumns. +/// For example, if we have type JSON and data {"a" : {"b" : {"c" : 42, "d" : "Hello"}}, "c" : [1, 2, 3], "d" : 42} +/// this class will be responsible for reading sub-object a.b and will read JSON column with data {"c" : 43, "d" : "Hello"}. +class SerializationSubObject final : public SimpleTextSerialization +{ +public: + SerializationSubObject(const String & path_prefix_, const std::unordered_map & typed_paths_serializations_); + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + + void serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } + bool tryDeserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } + +private: + [[noreturn]] static void throwNoSerialization() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Text/binary serialization is not implemented for object sub-object subcolumn"); + } + +private: + String path_prefix; + std::unordered_map typed_paths_serializations; + SerializationPtr dynamic_serialization; + SerializationPtr shared_data_serialization; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index e4d71e84cc7..0f6a17ef167 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -218,7 +218,8 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreamsAndUpdateVarian size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state, - std::unordered_map & variants_statistics) const + std::unordered_map & variants_statistics, + size_t & total_size_of_variants) const { const ColumnVariant & col = assert_cast(column); if (const size_t size = col.size(); limit == 0 || offset + limit > size) @@ -265,6 +266,7 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreamsAndUpdateVarian /// We can use the same offset/limit as for whole Variant column variants[non_empty_global_discr]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(non_empty_global_discr), offset, limit, settings, variant_state->variant_states[non_empty_global_discr]); variants_statistics[variant_names[non_empty_global_discr]] += limit; + total_size_of_variants += limit; settings.path.pop_back(); settings.path.pop_back(); return; @@ -315,7 +317,9 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreamsAndUpdateVarian { addVariantElementToPath(settings.path, i); variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), 0, 0, settings, variant_state->variant_states[i]); - variants_statistics[variant_names[i]] += col.getVariantByGlobalDiscriminator(i).size(); + size_t variant_size = col.getVariantByGlobalDiscriminator(i).size(); + variants_statistics[variant_names[i]] += variant_size; + total_size_of_variants += variant_size; settings.path.pop_back(); } settings.path.pop_back(); @@ -386,6 +390,7 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreamsAndUpdateVarian settings, variant_state->variant_states[i]); variants_statistics[variant_names[i]] += variant_offsets_and_limits[i].second; + total_size_of_variants += variant_offsets_and_limits[i].second; settings.path.pop_back(); } } @@ -400,7 +405,8 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( DB::ISerialization::SerializeBinaryBulkStatePtr & state) const { std::unordered_map tmp_statistics; - serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(column, offset, limit, settings, state, tmp_statistics); + size_t tmp_size; + serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(column, offset, limit, settings, state, tmp_statistics, tmp_size); } void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( @@ -1068,6 +1074,16 @@ void SerializationVariant::serializeTextJSON(const IColumn & column, size_t row_ variants[global_discr]->serializeTextJSON(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings); } +void SerializationVariant::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const +{ + const ColumnVariant & col = assert_cast(column); + auto global_discr = col.globalDiscriminatorAt(row_num); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + SerializationNullable::serializeNullJSON(ostr); + else + variants[global_discr]->serializeTextJSONPretty(col.getVariantByGlobalDiscriminator(global_discr), col.offsetAt(row_num), ostr, settings, indent); +} + bool SerializationVariant::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String field; diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index af89632cf81..a76a211e897 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -113,7 +113,8 @@ public: size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state, - std::unordered_map & variants_statistics) const; + std::unordered_map & variants_statistics, + size_t & total_size_of_variants) const; void deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, @@ -145,6 +146,7 @@ public: bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/tests/gtest_deprecated_object_serialization.cpp b/src/DataTypes/Serializations/tests/gtest_deprecated_object_serialization.cpp new file mode 100644 index 00000000000..ec53df18297 --- /dev/null +++ b/src/DataTypes/Serializations/tests/gtest_deprecated_object_serialization.cpp @@ -0,0 +1,80 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#if USE_SIMDJSON + +using namespace DB; + +TEST(SerializationObjectDeprecated, FromString) +{ + WriteBufferFromOwnString out; + + auto column_string = ColumnString::create(); + column_string->insert(R"({"k1" : 1, "k2" : [{"k3" : "aa", "k4" : 2}, {"k3": "bb", "k4": 3}]})"); + column_string->insert(R"({"k1" : 2, "k2" : [{"k3" : "cc", "k5" : 4}, {"k4": 5}, {"k4": 6}]})"); + + { + auto serialization = std::make_shared(); + + ISerialization::SerializeBinaryBulkSettings settings; + ISerialization::SerializeBinaryBulkStatePtr state; + settings.position_independent_encoding = false; + settings.getter = [&out](const auto &) { return &out; }; + + writeIntBinary(static_cast(1), out); + serialization->serializeBinaryBulkStatePrefix(*column_string, settings, state); + serialization->serializeBinaryBulkWithMultipleStreams(*column_string, 0, column_string->size(), settings, state); + serialization->serializeBinaryBulkStateSuffix(settings, state); + } + + auto type_object = std::make_shared("json", false); + ColumnPtr result_column = type_object->createColumn(); + + ReadBufferFromOwnString in(out.str()); + + { + auto serialization = type_object->getDefaultSerialization(); + + ISerialization::DeserializeBinaryBulkSettings settings; + ISerialization::DeserializeBinaryBulkStatePtr state; + settings.position_independent_encoding = false; + settings.getter = [&in](const auto &) { return ∈ }; + + serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); + serialization->deserializeBinaryBulkWithMultipleStreams(result_column, column_string->size(), settings, state, nullptr); + } + + auto & column_object = assert_cast(*result_column->assumeMutable()); + column_object.finalize(); + + ASSERT_TRUE(column_object.size() == 2); + ASSERT_TRUE(column_object.getSubcolumns().size() == 4); + + auto check_subcolumn = [&](const auto & name, const auto & type_name, const std::vector & expected) + { + const auto & subcolumn = column_object.getSubcolumn(PathInData{name}); + ASSERT_EQ(subcolumn.getLeastCommonType()->getName(), type_name); + + const auto & data = subcolumn.getFinalizedColumn(); + for (size_t i = 0; i < expected.size(); ++i) + ASSERT_EQ( + applyVisitor(FieldVisitorToString(), data[i]), + applyVisitor(FieldVisitorToString(), expected[i])); + }; + + check_subcolumn("k1", "Int8", {1, 2}); + check_subcolumn("k2.k3", "Array(String)", {Array{"aa", "bb"}, Array{"cc", "", ""}}); + check_subcolumn("k2.k4", "Array(Int8)", {Array{2, 3}, Array{0, 5, 6}}); + check_subcolumn("k2.k5", "Array(Int8)", {Array{0, 0}, Array{4, 0, 0}}); +} + +#endif diff --git a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp index c6337a31fce..8a0c712fa86 100644 --- a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp +++ b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp @@ -1,80 +1,98 @@ -#include -#include -#include -#include -#include -#include #include -#include -#include +#include +#include +#include #include -#if USE_SIMDJSON - using namespace DB; -TEST(SerializationObject, FromString) +TEST(ObjectSerialization, FieldBinarySerialization) { - WriteBufferFromOwnString out; - - auto column_string = ColumnString::create(); - column_string->insert(R"({"k1" : 1, "k2" : [{"k3" : "aa", "k4" : 2}, {"k3": "bb", "k4": 3}]})"); - column_string->insert(R"({"k1" : 2, "k2" : [{"k3" : "cc", "k5" : 4}, {"k4": 5}, {"k4": 6}]})"); - - { - auto serialization = std::make_shared(); - - ISerialization::SerializeBinaryBulkSettings settings; - ISerialization::SerializeBinaryBulkStatePtr state; - settings.position_independent_encoding = false; - settings.getter = [&out](const auto &) { return &out; }; - - writeIntBinary(static_cast(1), out); - serialization->serializeBinaryBulkStatePrefix(*column_string, settings, state); - serialization->serializeBinaryBulkWithMultipleStreams(*column_string, 0, column_string->size(), settings, state); - serialization->serializeBinaryBulkStateSuffix(settings, state); - } - - auto type_object = std::make_shared("json", false); - ColumnPtr result_column = type_object->createColumn(); - - ReadBufferFromOwnString in(out.str()); - - { - auto serialization = type_object->getDefaultSerialization(); - - ISerialization::DeserializeBinaryBulkSettings settings; - ISerialization::DeserializeBinaryBulkStatePtr state; - settings.position_independent_encoding = false; - settings.getter = [&in](const auto &) { return ∈ }; - - serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); - serialization->deserializeBinaryBulkWithMultipleStreams(result_column, column_string->size(), settings, state, nullptr); - } - - auto & column_object = assert_cast(*result_column->assumeMutable()); - column_object.finalize(); - - ASSERT_TRUE(column_object.size() == 2); - ASSERT_TRUE(column_object.getSubcolumns().size() == 4); - - auto check_subcolumn = [&](const auto & name, const auto & type_name, const std::vector & expected) - { - const auto & subcolumn = column_object.getSubcolumn(PathInData{name}); - ASSERT_EQ(subcolumn.getLeastCommonType()->getName(), type_name); - - const auto & data = subcolumn.getFinalizedColumn(); - for (size_t i = 0; i < expected.size(); ++i) - ASSERT_EQ( - applyVisitor(FieldVisitorToString(), data[i]), - applyVisitor(FieldVisitorToString(), expected[i])); - }; - - check_subcolumn("k1", "Int8", {1, 2}); - check_subcolumn("k2.k3", "Array(String)", {Array{"aa", "bb"}, Array{"cc", "", ""}}); - check_subcolumn("k2.k4", "Array(Int8)", {Array{2, 3}, Array{0, 5, 6}}); - check_subcolumn("k2.k5", "Array(Int8)", {Array{0, 0}, Array{4, 0, 0}}); + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, a.b UInt32, a.c Array(String))"); + auto serialization = type->getDefaultSerialization(); + Object object1 = Object{{"a.c", Array{"Str1", "Str2"}}, {"a.d", Field(42)}, {"a.e", Tuple{Field(43), "Str3"}}}; + WriteBufferFromOwnString ostr; + serialization->serializeBinary(object1, ostr, FormatSettings()); + ReadBufferFromString istr(ostr.str()); + Field object2; + serialization->deserializeBinary(object2, istr, FormatSettings()); + ASSERT_EQ(object1, object2.get()); } -#endif + +TEST(ObjectSerialization, ColumnBinarySerialization) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, a.b UInt32, a.c Array(String))"); + auto serialization = type->getDefaultSerialization(); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + col_object.insert(Object{{"a.c", Array{"Str1", "Str2"}}, {"a.d", Field(42)}, {"a.e", Tuple{Field(43), "Str3"}}}); + WriteBufferFromOwnString ostr1; + serialization->serializeBinary(col_object, 0, ostr1, FormatSettings()); + ReadBufferFromString istr1(ostr1.str()); + serialization->deserializeBinary(col_object, istr1, FormatSettings()); + ASSERT_EQ(col_object[0], col_object[1]); + col_object.insert(Object{{"a.c", Array{"Str1", "Str2"}}, {"a.e", Field(42)}, {"b.d", Field(42)}, {"b.e", Tuple{Field(43), "Str3"}}, {"b.g", Field("Str4")}}); + WriteBufferFromOwnString ostr2; + serialization->serializeBinary(col_object, 2, ostr2, FormatSettings()); + ReadBufferFromString istr2(ostr2.str()); + serialization->deserializeBinary(col_object, istr2, FormatSettings()); + ASSERT_EQ(col_object[2], col_object[3]); +} + +TEST(ObjectSerialization, JSONSerialization) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, a.b UInt32, a.c Array(String))"); + auto serialization = type->getDefaultSerialization(); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + col_object.insert(Object{{"a.c", Array{"Str1", "Str2"}}, {"a.d", Field(42)}, {"a.e", Tuple{Field(43), "Str3"}}}); + col_object.insert(Object{{"a.c", Array{"Str1", "Str2"}}, {"a", Tuple{Field(43), "Str3"}}, {"a.b.c", Field(42)}, {"a.b.e", Field(43)}, {"b.c.d.e", Field(42)}, {"b.c.d.g", Field(43)}, {"b.c.h.r", Field(44)}, {"c.g.h.t", Array{Field("Str"), Field("Str2")}}, {"h", Field("Str")}, {"j", Field("Str")}}); + WriteBufferFromOwnString buf1; + serialization->serializeTextJSON(col_object, 1, buf1, FormatSettings()); + ASSERT_EQ(buf1.str(), R"({"a":[43,"Str3"],"a":{"b":0,"b":{"c":42,"e":43},"c":["Str1","Str2"]},"b":{"c":{"d":{"e":42,"g":43},"h":{"r":44}}},"c":{"g":{"h":{"t":["Str","Str2"]}}},"h":"Str","j":"Str"})"); + WriteBufferFromOwnString buf2; + serialization->serializeTextJSONPretty(col_object, 1, buf2, FormatSettings(), 0); + ASSERT_EQ(buf2.str(), R"({ + "a" : [ + 43, + "Str3" + ], + "a" : { + "b" : 0, + "b" : { + "c" : 42, + "e" : 43 + }, + "c" : [ + "Str1", + "Str2" + ] + }, + "b" : { + "c" : { + "d" : { + "e" : 42, + "g" : 43 + }, + "h" : { + "r" : 44 + } + } + }, + "c" : { + "g" : { + "h" : { + "t" : [ + "Str", + "Str2" + ] + } + } + }, + "h" : "Str", + "j" : "Str" +})"); + +} diff --git a/src/DataTypes/Utils.cpp b/src/DataTypes/Utils.cpp index e7e69e379af..a6e9452d7ef 100644 --- a/src/DataTypes/Utils.cpp +++ b/src/DataTypes/Utils.cpp @@ -216,6 +216,7 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ return false; } case TypeIndex::String: + case TypeIndex::ObjectDeprecated: case TypeIndex::Object: case TypeIndex::Set: case TypeIndex::Interval: diff --git a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp index 4d0bfc67183..2e646d1e62a 100644 --- a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp +++ b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp @@ -32,7 +32,7 @@ extern const int UNSUPPORTED_METHOD; void check(const DataTypePtr & type) { -// std::cerr << "Check " << type->getName() << "\n"; + std::cerr << "Check " << type->getName() << "\n"; WriteBufferFromOwnString ostr; encodeDataType(type, ostr); ReadBufferFromString istr(ostr.str()); @@ -126,4 +126,7 @@ GTEST_TEST(DataTypesBinaryEncoding, EncodeAndDecode) check(DataTypeFactory::instance().get("Polygon")); check(DataTypeFactory::instance().get("MultiPolygon")); check(DataTypeFactory::instance().get("Tuple(Map(LowCardinality(String), Array(AggregateFunction(2, quantiles(0.1, 0.2), Float32))), Array(Array(Tuple(UInt32, Tuple(a Map(String, String), b Nullable(Date), c Variant(Tuple(g String, d Array(UInt32)), Date, Map(String, String)))))))")); + check(DataTypeFactory::instance().get("JSON")); + check(DataTypeFactory::instance().get("JSON(max_dynamic_paths=10)")); + check(DataTypeFactory::instance().get("JSON(max_dynamic_paths=10, max_dynamic_types=10, a.b.c UInt32, SKIP a.c, b.g String, SKIP l.d.f)")); } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 4ca9afc49eb..133834180c0 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -996,6 +996,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("allow_experimental_object_type", 1); query_context->setSetting("allow_experimental_variant_type", 1); query_context->setSetting("allow_experimental_dynamic_type", 1); + query_context->setSetting("allow_experimental_json_type", 1); query_context->setSetting("allow_experimental_annoy_index", 1); query_context->setSetting("allow_experimental_usearch_index", 1); query_context->setSetting("allow_experimental_bigint_types", 1); diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 58407a810c5..5ccc09c7e8f 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -463,7 +463,7 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo settings.json.read_arrays_as_strings, settings.json.try_infer_objects_as_tuples, settings.json.infer_incomplete_types_as_strings, - settings.json.allow_object_type, + settings.json.allow_deprecated_object_type, settings.json.use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects); break; default: diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 3d92023318e..325e3bbb25a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -144,12 +144,14 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata; format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; - format_settings.json.allow_object_type = context->getSettingsRef().allow_experimental_object_type; + format_settings.json.allow_deprecated_object_type = context->getSettingsRef().allow_experimental_object_type; + format_settings.json.allow_json_type = context->getSettingsRef().allow_experimental_json_type; format_settings.json.compact_allow_variable_number_of_columns = settings.input_format_json_compact_allow_variable_number_of_columns; format_settings.json.try_infer_objects_as_tuples = settings.input_format_json_try_infer_named_tuples_from_objects; format_settings.json.throw_on_bad_escape_sequence = settings.input_format_json_throw_on_bad_escape_sequence; format_settings.json.ignore_unnecessary_fields = settings.input_format_json_ignore_unnecessary_fields; format_settings.json.ignore_key_case = settings.input_format_json_ignore_key_case; + format_settings.json.type_json_skip_duplicated_paths = settings.type_json_skip_duplicated_paths; format_settings.null_as_default = settings.input_format_null_as_default; format_settings.force_null_for_omitted_fields = settings.input_format_force_null_for_omitted_fields; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8b38ffba524..c2bc1c113eb 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -227,7 +227,8 @@ struct FormatSettings bool try_infer_numbers_from_strings = false; bool validate_types_from_metadata = true; bool validate_utf8 = false; - bool allow_object_type = false; + bool allow_deprecated_object_type = false; + bool allow_json_type = false; bool valid_output_on_exception = false; bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; @@ -235,6 +236,7 @@ struct FormatSettings bool throw_on_bad_escape_sequence = true; bool ignore_unnecessary_fields = true; bool ignore_key_case = false; + bool type_json_skip_duplicated_paths = false; } json{}; struct diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 242d2dc9f80..db984d67819 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -8,7 +8,6 @@ #if USE_RAPIDJSON #include #endif - #include #include @@ -22,6 +21,7 @@ #include #include #include +#include #include #include @@ -38,8 +38,10 @@ #include #include #include +#include #include #include +#include #include @@ -53,6 +55,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int INCORRECT_DATA; } template @@ -172,7 +175,7 @@ bool tryGetNumericValueFromJSONElement( { if (!tryReadFloatText(value, rb) || !rb.eof()) { - error = fmt::format("cannot parse {} value here: {}", TypeName, element.getString()); + error = fmt::format("cannot parse {} value here: \"{}\"", TypeName, element.getString()); return false; } } @@ -186,13 +189,13 @@ bool tryGetNumericValueFromJSONElement( rb.position() = rb.buffer().begin(); if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) { - error = fmt::format("cannot parse {} value here: {}", TypeName, element.getString()); + error = fmt::format("cannot parse {} value here: \"{}\"", TypeName, element.getString()); return false; } if (!accurate::convertNumeric(tmp_float, value)) { - error = fmt::format("cannot parse {} value here: {}", TypeName, element.getString()); + error = fmt::format("cannot parse {} value here: \"{}\"", TypeName, element.getString()); return false; } } @@ -1104,7 +1107,7 @@ public: } } - if (!were_valid_elements) + if (data.size() != old_size && !were_valid_elements) { data.popBack(data.size() - old_size); return false; @@ -1174,7 +1177,7 @@ public: else { set_size(old_size); - error += fmt::format("(during reading tuple {} element)", index); + error += fmt::format(" (during reading tuple {} element)", index); return false; } } @@ -1202,7 +1205,7 @@ public: else { set_size(old_size); - error += fmt::format("(during reading tuple {} element)", index); + error += fmt::format(" (during reading tuple {} element)", index); return false; } } @@ -1221,7 +1224,7 @@ public: else if (!insert_settings.insert_default_on_invalid_elements_in_complex_types) { set_size(old_size); - error += fmt::format("(during reading tuple element \"{}\")", key); + error += fmt::format(" (during reading tuple element \"{}\")", key); return false; } } @@ -1288,7 +1291,7 @@ public: { key_col.popBack(key_col.size() - offsets.back()); value_col.popBack(value_col.size() - offsets.back()); - error += fmt::format("(during reading value of key \"{}\")", pair.first); + error += fmt::format(" (during reading value of key \"{}\")", pair.first); return false; } } @@ -1346,6 +1349,13 @@ template class DynamicNode : public JSONExtractTreeNode { public: + DynamicNode( + size_t max_dynamic_paths_for_object_ = DataTypeObject::DEFAULT_MAX_SEPARATELY_STORED_PATHS, + size_t max_dynamic_types_for_object_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES) + : max_dynamic_paths_for_object(max_dynamic_paths_for_object_), max_dynamic_types_for_object(max_dynamic_types_for_object_) + { + } + bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, @@ -1362,13 +1372,22 @@ public: } auto & variant_column = column_dynamic.getVariantColumn(); - auto variant_info = column_dynamic.getVariantInfo(); - /// Second, infer ClickHouse type for this element and add it as a new variant. - auto element_type = elementToDataType(element, format_settings); + auto & variant_info = column_dynamic.getVariantInfo(); + /// First, infer ClickHouse type for this element and add it as a new variant. + auto element_type = removeNullable(elementToDataType(element, format_settings)); + if (!checkIfTypeIsComplete(element_type)) + { + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Cannot infer the type of JSON element {}, because it contains only nulls. To use String type for elements with incomplete " + "type, enable setting input_format_json_infer_incomplete_types_as_strings", + jsonElementToString(element, format_settings)); + } + if (column_dynamic.addNewVariant(element_type)) { auto node = buildJSONExtractTree(element_type, "Dynamic inference"); - auto global_discriminator = variant_info.variant_name_to_discriminator[element_type->getName()]; + auto global_discriminator = variant_info.variant_name_to_discriminator.at(element_type->getName()); auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discriminator); if (!node->insertResultToColumn(variant, element, insert_settings, format_settings, error)) return false; @@ -1384,7 +1403,7 @@ public: /// We couldn't insert element into any existing variant, add String variant and read value as String. column_dynamic.addStringVariant(); - auto string_global_discriminator = variant_info.variant_name_to_discriminator["String"]; + auto string_global_discriminator = variant_info.variant_name_to_discriminator.at("String"); auto & string_column = variant_column.getVariantByGlobalDiscriminator(string_global_discriminator); if (!getStringNode()->insertResultToColumn(string_column, element, insert_settings, format_settings, error)) return false; @@ -1400,16 +1419,18 @@ public: return string_node; } - static DataTypePtr elementToDataType(const typename JSONParser::Element & element, const FormatSettings & format_settings) + DataTypePtr elementToDataType(const typename JSONParser::Element & element, const FormatSettings & format_settings) const { JSONInferenceInfo json_inference_info; auto type = elementToDataTypeImpl(element, format_settings, json_inference_info); transformFinalInferredJSONTypeIfNeeded(type, format_settings, &json_inference_info); + if (format_settings.schema_inference_make_columns_nullable) + type = makeNullableRecursively(type); return type; } private: - static DataTypePtr elementToDataTypeImpl(const typename JSONParser::Element & element, const FormatSettings & format_settings, JSONInferenceInfo & json_inference_info) + DataTypePtr elementToDataTypeImpl(const typename JSONParser::Element & element, const FormatSettings & format_settings, JSONInferenceInfo & json_inference_info) const { switch (element.type()) { @@ -1452,10 +1473,10 @@ private: DataTypes types; types.reserve(array.size()); for (auto value : array) - types.push_back(makeNullableSafe(elementToDataTypeImpl(value, format_settings, json_inference_info))); + types.push_back(elementToDataTypeImpl(value, format_settings, json_inference_info)); if (types.empty()) - return std::make_shared(makeNullable(std::make_shared())); + return std::make_shared(std::make_shared()); if (checkIfTypesAreEqual(types)) return std::make_shared(types.back()); @@ -1482,12 +1503,231 @@ private: return std::make_shared(types); } - case ElementType::OBJECT: { - /// TODO: Use new JSON type here when it's ready. - return std::make_shared(std::make_shared(), makeNullable(std::make_shared())); + case ElementType::OBJECT: + { + return std::make_shared(DataTypeObject::SchemaFormat::JSON, max_dynamic_paths_for_object, max_dynamic_types_for_object); } } } + + size_t max_dynamic_paths_for_object; + size_t max_dynamic_types_for_object; +}; + +template +class ObjectJSONNode : public JSONExtractTreeNode +{ +public: + ObjectJSONNode( + std::unordered_map>> typed_path_nodes_, + const std::unordered_set & paths_to_skip_, + const std::vector & path_prefixes_to_skip_, + const std::vector & path_regexps_to_skip_, + size_t max_dynamic_paths_, + size_t max_dynamic_types) + : typed_path_nodes(std::move(typed_path_nodes_)) + , paths_to_skip(paths_to_skip_) + , path_prefixes_to_skip(path_prefixes_to_skip_) + , dynamic_node(std::make_unique>( + max_dynamic_paths_ / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, + std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))) + , dynamic_serialization(std::make_shared()) + { + for (const auto & regexp : path_regexps_to_skip_) + path_regexps_to_skip.emplace_back(regexp); + } + + bool insertResultToColumn(IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + if (!element.isObject()) + { + error = fmt::format("Cannot read JSON object from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto & column_object = assert_cast(column); + size_t prev_size = column_object.size(); + + /// Paths in shared data should be sorted, so we cannot insert paths there during traverse. + /// Instead we collect all paths and values that should go to shared data, sort them and insert later. + /// It's not optimal, but it's a price we pay for faster reading of subcolumns. + std::vector> paths_and_values_for_shared_data; + if (!traverseAndInsert(column_object, element, "", insert_settings, format_settings, paths_and_values_for_shared_data, prev_size, error)) + { + /// If there was an error, restore previous state. + SerializationObject::restoreColumnObject(column_object, prev_size); + return false; + } + + /// Fill shared data. + auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + std::sort(paths_and_values_for_shared_data.begin(), paths_and_values_for_shared_data.end()); + for (size_t i = 0; i != paths_and_values_for_shared_data.size(); ++i) + { + const auto & [path, value] = paths_and_values_for_shared_data[i]; + /// Check if we duplicated paths. + if (i != 0 && path == paths_and_values_for_shared_data[i - 1].first) + { + if (!format_settings.json.type_json_skip_duplicated_paths) + { + error = fmt::format("Duplicate path found during parsing JSON object: {}", path); + SerializationObject::restoreColumnObject(column_object, prev_size); + return false; + } + } + else + { + shared_data_paths->insertData(path.data(), path.size()); + shared_data_values->insertData(value.data(), value.size()); + } + } + column_object.getSharedDataOffsets().push_back(shared_data_paths->size()); + + /// Fill remaining typed and dynamic paths. + for (auto & [_, typed_column] : column_object.getTypedPaths()) + { + if (typed_column->size() == prev_size) + typed_column->insertDefault(); + } + + for (auto & [_, dynamic_column] : column_object.getDynamicPaths()) + { + if (dynamic_column->size() == prev_size) + dynamic_column->insertDefault(); + } + + return true; + } + +private: + bool traverseAndInsert( + ColumnObject & column_object, + const typename JSONParser::Element & element, + const String & current_path, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + std::vector> & paths_and_values_for_shared_data, + size_t current_size, + String & error) const + { + if (element.isObject() && !typed_path_nodes.contains(current_path)) + { + for (auto [key, value] : element.getObject()) + { + String path = current_path; + if (!path.empty()) + path.append("."); + path += key; + if (!traverseAndInsert(column_object, value, path, insert_settings, format_settings, paths_and_values_for_shared_data, current_size, error)) + return false; + } + + return true; + } + + if (shouldSkipPath(current_path)) + return true; + + auto & typed_paths = column_object.getTypedPaths(); + auto & dynamic_paths = column_object.getDynamicPaths(); + /// Check if we have this path in typed pahts. + if (auto typed_it = typed_paths.find(current_path); typed_it != typed_paths.end()) + { + /// Check if we already had this path. + if (typed_it->second->size() > current_size) + { + if (!format_settings.json.type_json_skip_duplicated_paths) + { + error = fmt::format("Duplicate path found during parsing JSON object: {}", current_path); + return false; + } + } + else if (!typed_path_nodes.at(current_path)->insertResultToColumn(*typed_it->second, element, insert_settings, format_settings, error)) + { + error += fmt::format(" (while reading path {})", current_path); + return false; + } + } + /// Check if we have this path in dynamic paths. + else if (auto dynamic_it = dynamic_paths.find(current_path); dynamic_it != dynamic_paths.end()) + { + /// Check if we already had this path. + if (dynamic_it->second->size() > current_size) + { + if (!format_settings.json.type_json_skip_duplicated_paths) + { + error = fmt::format("Duplicate path found during parsing JSON object: {}", current_path); + return false; + } + } + else if (!dynamic_node->insertResultToColumn(*dynamic_it->second, element, insert_settings, format_settings, error)) + { + error += fmt::format(" (while reading path {})", current_path); + return false; + } + } + /// Try to add a new dynamic path. + else if (auto dynamic_column = column_object.tryToAddNewDynamicPath(current_path)) + { + if (!dynamic_node->insertResultToColumn(*dynamic_column, element, insert_settings, format_settings, error)) + { + error += fmt::format(" (while reading path {})", current_path); + return false; + } + } + /// Otherwise this path should go to the shared data. + /// Don't insert null values into shared data. + /// We consider null equivalent to the absense of this path. + else if (!element.isNull()) + { + auto tmp_dynamic_column = ColumnDynamic::create(); + tmp_dynamic_column->reserve(1); + if (!dynamic_node->insertResultToColumn(*tmp_dynamic_column, element, insert_settings, format_settings, error)) + { + error += fmt::format(" (while reading path {})", current_path); + return false; + } + + paths_and_values_for_shared_data.emplace_back(current_path, ""); + WriteBufferFromString buf(paths_and_values_for_shared_data.back().second); + dynamic_serialization->serializeBinary(*tmp_dynamic_column, 0, buf, format_settings); + } + + return true; + } + + bool shouldSkipPath(const String & path) const + { + if (paths_to_skip.contains(path)) + return true; + + for (const auto & prefix : path_prefixes_to_skip) + { + if (path.starts_with(prefix)) + return true; + } + + for (const auto & regexp : path_regexps_to_skip) + { + if (re2::RE2::FullMatch(path, regexp)) + return true; + } + + return false; + } + + std::unordered_map>> typed_path_nodes; + std::unordered_set paths_to_skip; + std::vector path_prefixes_to_skip; + std::list path_regexps_to_skip; + std::unique_ptr> dynamic_node; + std::shared_ptr dynamic_serialization; }; } @@ -1634,6 +1874,27 @@ std::unique_ptr> buildJSONExtractTree(const Data } case TypeIndex::Dynamic: return std::make_unique>(); + case TypeIndex::Object: + { + const auto & object_type = assert_cast(*type); + const auto & typed_paths = object_type.getTypedPaths(); + std::unordered_map>> typed_path_nodes; + typed_path_nodes.reserve(typed_paths.size()); + for (const auto & [path, path_type] : typed_paths) + typed_path_nodes[path] = buildJSONExtractTree(path_type, source_for_exception_message); + + switch (object_type.getSchemaFormat()) + { + case DataTypeObject::SchemaFormat::JSON: + return std::make_unique>( + std::move(typed_path_nodes), + object_type.getPathsToSkip(), + object_type.getPathPrefixesToSkip(), + object_type.getPathRegexpsToSkip(), + object_type.getMaxDynamicPaths(), + object_type.getMaxDynamicTypes()); + } + } default: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index f0985f4a6b7..f859ad73ea6 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -1,13 +1,13 @@ -#include +#include +#include +#include +#include +#include #include #include -#include #include +#include #include -#include -#include -#include -#include #include diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 3c374ada9e6..0f8dc83bd52 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1,22 +1,22 @@ -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include -#include #include #include @@ -1176,8 +1176,8 @@ namespace { if constexpr (is_json) { - if (settings.json.allow_object_type) - return std::make_shared("json", true); + if (settings.json.allow_deprecated_object_type) + return std::make_shared("json", true); } /// Empty Map is Map(Nothing, Nothing) @@ -1186,8 +1186,8 @@ namespace if constexpr (is_json) { - if (settings.json.allow_object_type) - return std::make_shared("json", true); + if (settings.json.allow_deprecated_object_type) + return std::make_shared("json", true); if (settings.json.read_objects_as_strings) return std::make_shared(); @@ -1242,7 +1242,7 @@ namespace { if constexpr (is_json) { - if (!settings.json.allow_object_type && settings.json.try_infer_objects_as_tuples) + if (!settings.json.allow_deprecated_object_type && settings.json.try_infer_objects_as_tuples) return tryInferJSONPaths(buf, settings, json_info, depth); } @@ -1525,15 +1525,15 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) return nested_type ? std::make_shared(nested_type) : nullptr; } - if (which.isObject()) + if (which.isObjectDeprecated()) { - const auto * object_type = assert_cast(type.get()); + const auto * object_type = assert_cast(type.get()); if (object_type->hasNullableSubcolumns()) return type; - return std::make_shared(object_type->getSchemaFormat(), true); + return std::make_shared(object_type->getSchemaFormat(), true); } - return makeNullable(type); + return makeNullableSafe(type); } NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index f3e54d2fbd9..0b2cfcab165 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4,17 +4,18 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include #include #include -#include #include #include #include @@ -24,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -34,18 +36,18 @@ #include #include #include +#include #include #include #include #include #include -#include #include #include #include #include -#include #include +#include #include #include #include @@ -3878,7 +3880,7 @@ private: "Expected tuple with {} subcolumn, but got {} subcolumns", tuple_size, column_tuple.getColumns().size()); - auto res = ColumnObject::create(has_nullable_subcolumns); + auto res = ColumnObjectDeprecated::create(has_nullable_subcolumns); for (size_t i = 0; i < tuple_size; ++i) { ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }}; @@ -3955,7 +3957,7 @@ private: subcolumn->insertDefault(); } - auto column_object = ColumnObject::create(has_nullable_subcolumns); + auto column_object = ColumnObjectDeprecated::create(has_nullable_subcolumns); for (auto && [key, subcolumn] : subcolumns) { PathInData path(key.toView()); @@ -3966,7 +3968,7 @@ private: }; } - WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_type) const + WrapperType createObjectDeprecatedWrapper(const DataTypePtr & from_type, const DataTypeObjectDeprecated * to_type) const { if (const auto * from_tuple = checkAndGetDataType(from_type.get())) { @@ -3985,12 +3987,12 @@ private: return res; }; } - else if (checkAndGetDataType(from_type.get())) + else if (checkAndGetDataType(from_type.get())) { return [is_nullable = to_type->hasNullableSubcolumns()] (ColumnsWithTypeAndName & arguments, const DataTypePtr & , const ColumnNullable * , size_t) -> ColumnPtr { - const auto & column_object = assert_cast(*arguments.front().column); - auto res = ColumnObject::create(is_nullable); + const auto & column_object = assert_cast(*arguments.front().column); + auto res = ColumnObjectDeprecated::create(is_nullable); for (size_t i = 0; i < column_object.size(); i++) res->insert(column_object[i]); @@ -4002,6 +4004,24 @@ private: throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName()); } + + WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_object) const + { + if (checkAndGetDataType(from_type.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) + { + auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context)->assumeMutable(); + res->finalize(); + return res; + }; + } + + /// TODO: support CAST between JSON types with different parameters + /// support CAST from Map to JSON + /// support CAST from Tuple to JSON + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to {} can be performed only from String. Got: {}", magic_enum::enum_name(to_object->getSchemaFormat()), from_type->getName()); + } WrapperType createVariantToVariantWrapper(const DataTypeVariant & from_variant, const DataTypeVariant & to_variant) const { @@ -5136,6 +5156,8 @@ private: return createTupleWrapper(from_type, checkAndGetDataType(to_type.get())); case TypeIndex::Map: return createMapWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::ObjectDeprecated: + return createObjectDeprecatedWrapper(from_type, checkAndGetDataType(to_type.get())); case TypeIndex::Object: return createObjectWrapper(from_type, checkAndGetDataType(to_type.get())); case TypeIndex::AggregateFunction: diff --git a/src/Functions/JSONEmpty.cpp b/src/Functions/JSONEmpty.cpp new file mode 100644 index 00000000000..e932074ccb8 --- /dev/null +++ b/src/Functions/JSONEmpty.cpp @@ -0,0 +1,126 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + + +namespace +{ + +/// Implements the function JSONEmpty which returns true if provided JSON object is empty and false otherwise. +class FunctionJSONEmpty : public IFunction +{ +public: + static constexpr auto name = "JSONEmpty"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override + { + if (data_types.size() != 1 || data_types[0]->getTypeId() != TypeIndex::Object) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single argument with type JSON", getName()); + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const ColumnWithTypeAndName & elem = arguments[0]; + const auto * object_column = typeid_cast(elem.column.get()); + if (!object_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected column type in function {}. Expected Object column, got {}", getName(), elem.column->getName()); + + auto res = DataTypeUInt8().createColumn(); + auto & data = typeid_cast(*res).getData(); + const auto & typed_paths = object_column->getTypedPaths(); + size_t size = object_column->size(); + /// If object column has at least 1 typed path, it will never be empty, because these paths always have values. + if (!typed_paths.empty()) + { + data.resize_fill(size, 0); + return res; + } + + const auto & dynamic_paths = object_column->getDynamicPaths(); + const auto & shared_data = object_column->getSharedDataPtr(); + data.reserve(size); + for (size_t i = 0; i != size; ++i) + { + bool empty = true; + /// Check if there is no paths in shared data. + if (!shared_data->isDefaultAt(i)) + { + empty = false; + } + /// Check that all dynamic paths have NULL value in this row. + else + { + for (const auto & [path, column] : dynamic_paths) + { + if (!column->isNullAt(i)) + { + empty = false; + break; + } + } + } + + data.push_back(empty); + } + + return res; + } +}; + +} + +REGISTER_FUNCTION(JSONEmpty) +{ + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Checks whether thee input JSON object is empty. +)", + .syntax = {"JSONEmpty(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {}}, {"json" : {"b" : "Hello"}}, {"json" : {}} +SELECT json, JSONEmpty(json) FROM test; +)", + R"( +┌─json──────────┬─JSONEmpty(json)─┠+│ {"a":"42"} │ 0 │ +│ {} │ 1 │ +│ {"b":"Hello"} │ 0 │ +│ {} │ 1 │ +└───────────────┴─────────────────┘ + +)"}}}, + .categories{"JSON"}, + }); +} + +} diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp new file mode 100644 index 00000000000..bbdf1385cf2 --- /dev/null +++ b/src/Functions/JSONPaths.cpp @@ -0,0 +1,339 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +enum class PathsMode +{ + ALL_PATHS, + DYNAMIC_PATHS, + SHARED_DATA_PATHS, +}; + +struct JSONAllPathsImpl +{ + static constexpr auto name = "JSONAllPaths"; + static constexpr auto paths_mode = PathsMode::ALL_PATHS; + static constexpr auto with_types = false; +}; + +struct JSONAllPathsWithTypesImpl +{ + static constexpr auto name = "JSONAllPathsWithTypes"; + static constexpr auto paths_mode = PathsMode::ALL_PATHS; + static constexpr auto with_types = true; +}; + +struct JSONDynamicPathsImpl +{ + static constexpr auto name = "JSONDynamicPaths"; + static constexpr auto paths_mode = PathsMode::DYNAMIC_PATHS; + static constexpr auto with_types = false; +}; + +struct JSONDynamicPathsWithTypesImpl +{ + static constexpr auto name = "JSONDynamicPathsWithTypes"; + static constexpr auto paths_mode = PathsMode::DYNAMIC_PATHS; + static constexpr auto with_types = true; +}; + +struct JSONSharedDataPathsImpl +{ + static constexpr auto name = "JSONSharedDataPaths"; + static constexpr auto paths_mode = PathsMode::SHARED_DATA_PATHS; + static constexpr auto with_types = false; +}; + +struct JSONSharedDataPathsWithTypesImpl +{ + static constexpr auto name = "JSONSharedDataPathsWithTypes"; + static constexpr auto paths_mode = PathsMode::SHARED_DATA_PATHS; + static constexpr auto with_types = true; +}; + +/// Implements functions that extracts paths and types from JSON object column. +template +class FunctionJSONPaths : public IFunction +{ +public: + static constexpr auto name = Impl::name; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + std::string getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override + { + if (data_types.size() != 1 || data_types[0]->getTypeId() != TypeIndex::Object) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single argument with type JSON", getName()); + + if constexpr (Impl::with_types) + return std::make_shared(std::make_shared(), std::make_shared()); + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const ColumnWithTypeAndName & elem = arguments[0]; + const auto * column_object = typeid_cast(elem.column.get()); + if (!column_object) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected column type in function {}. Expected Object column, got {}", getName(), elem.column->getName()); + + const auto & type_object = assert_cast(*elem.type); + if constexpr (Impl::with_types) + return executeWithTypes(*column_object, type_object); + return executeWithoutTypes(*column_object); + } + +private: + ColumnPtr executeWithoutTypes(const ColumnObject & column_object) const + { + if constexpr (Impl::paths_mode == PathsMode::SHARED_DATA_PATHS) + { + /// No need to do anything, we already have a column with all sorted paths in shared data. + const auto & shared_data_array = column_object.getSharedDataNestedColumn(); + const auto & shared_data_paths = assert_cast(shared_data_array.getData()).getColumnPtr(0); + return ColumnArray::create(shared_data_paths, shared_data_array.getOffsetsPtr()); + } + + auto res = ColumnArray::create(ColumnString::create()); + auto & offsets = res->getOffsets(); + ColumnString & data = assert_cast(res->getData()); + + if constexpr (Impl::paths_mode == PathsMode::DYNAMIC_PATHS) + { + /// Collect all dynamic paths. + const auto & dynamic_path_columns = column_object.getDynamicPaths(); + std::vector dynamic_paths; + dynamic_paths.reserve(dynamic_path_columns.size()); + for (const auto & [path, _] : dynamic_path_columns) + dynamic_paths.push_back(path); + /// We want the resulting arrays of paths to be sorted for consistency. + std::sort(dynamic_paths.begin(), dynamic_paths.end()); + + for (const auto & path : dynamic_paths) + data.insertData(path.data(), path.size()); + offsets.push_back(data.size()); + return res->replicate(IColumn::Offsets(1, column_object.size())); + } + + /// Collect all paths: typed, dynamic and paths from shared data. + std::vector sorted_dynamic_and_typed_paths; + const auto & typed_path_columns = column_object.getTypedPaths(); + const auto & dynamic_path_columns = column_object.getDynamicPaths(); + for (const auto & [path, _] : typed_path_columns) + sorted_dynamic_and_typed_paths.push_back(path); + for (const auto & [path, _] : dynamic_path_columns) + sorted_dynamic_and_typed_paths.push_back(path); + + /// We want the resulting arrays of paths to be sorted for consistency. + std::sort(sorted_dynamic_and_typed_paths.begin(), sorted_dynamic_and_typed_paths.end()); + + const auto & shared_data_offsets = column_object.getSharedDataOffsets(); + const auto [shared_data_paths, _] = column_object.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_offsets.size(); ++i) + { + size_t start = shared_data_offsets[ssize_t(i) - 1]; + size_t end = shared_data_offsets[ssize_t(i)]; + /// Merge sorted list of paths from shared data and sorted_dynamic_and_typed_paths + size_t sorted_paths_index = 0; + for (size_t j = start; j != end; ++j) + { + auto shared_data_path = shared_data_paths->getDataAt(j); + while (sorted_paths_index != sorted_dynamic_and_typed_paths.size() && sorted_dynamic_and_typed_paths[sorted_paths_index] < shared_data_path) + { + auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; + data.insertData(path.data, path.size); + ++sorted_paths_index; + } + + data.insertData(shared_data_path.data, shared_data_path.size); + } + + for (; sorted_paths_index != sorted_dynamic_and_typed_paths.size(); ++sorted_paths_index) + { + auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; + data.insertData(path.data, path.size); + } + + offsets.push_back(data.size()); + } + + return res; + } + + ColumnPtr executeWithTypes(const ColumnObject & column_object, const DataTypeObject & type_object) const + { + auto offsets_column = ColumnArray::ColumnOffsets::create(); + auto & offsets = offsets_column->getData(); + auto paths_column = ColumnString::create(); + auto types_column = ColumnString::create(); + + if constexpr (Impl::paths_mode == PathsMode::DYNAMIC_PATHS) + { + const auto & dynamic_path_columns = column_object.getDynamicPaths(); + std::vector sorted_dynamic_paths; + sorted_dynamic_paths.reserve(dynamic_path_columns.size()); + for (const auto & [path, _] : dynamic_path_columns) + sorted_dynamic_paths.push_back(path); + /// We want the resulting arrays of paths and values to be sorted for consistency. + std::sort(sorted_dynamic_paths.begin(), sorted_dynamic_paths.end()); + + /// Iterate over all rows and extract types from dynamic columns. + for (size_t i = 0; i != column_object.size(); ++i) + { + for (auto & path : sorted_dynamic_paths) + { + auto type = getDynamicValueType(dynamic_path_columns.at(path), i); + paths_column->insertData(path.data(), path.size()); + types_column->insertData(type.data(), type.size()); + } + + offsets.push_back(types_column->size()); + } + + return ColumnMap::create(ColumnPtr(std::move(paths_column)), ColumnPtr(std::move(types_column)), ColumnPtr(std::move(offsets_column))); + } + + if constexpr (Impl::paths_mode == PathsMode::SHARED_DATA_PATHS) + { + const auto & shared_data_offsets = column_object.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + /// Iterate over all rows and extract types from dynamic values in shared data. + for (size_t i = 0; i != shared_data_offsets.size(); ++i) + { + size_t start = shared_data_offsets[ssize_t(i) - 1]; + size_t end = shared_data_offsets[ssize_t(i)]; + for (size_t j = start; j != end; ++j) + { + paths_column->insertFrom(*shared_data_paths, j); + auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j)); + types_column->insertData(type_name.data(), type_name.size()); + } + + offsets.push_back(paths_column->size()); + } + + return ColumnMap::create(ColumnPtr(std::move(paths_column)), ColumnPtr(std::move(types_column)), ColumnPtr(std::move(offsets_column))); + } + + /// Iterate over all rows and extract types from dynamic columns from dynamic paths and from values in shared data. + std::vector> sorted_typed_and_dynamic_paths_with_types; + const auto & typed_path_types = type_object.getTypedPaths(); + const auto & dynamic_path_columns = column_object.getDynamicPaths(); + sorted_typed_and_dynamic_paths_with_types.reserve(typed_path_types.size() + dynamic_path_columns.size()); + for (const auto & [path, type] : typed_path_types) + sorted_typed_and_dynamic_paths_with_types.emplace_back(path, type->getName()); + for (const auto & [path, _] : dynamic_path_columns) + sorted_typed_and_dynamic_paths_with_types.emplace_back(path, ""); + + /// We want the resulting arrays of paths and values to be sorted for consistency. + std::sort(sorted_typed_and_dynamic_paths_with_types.begin(), sorted_typed_and_dynamic_paths_with_types.end()); + + const auto & shared_data_offsets = column_object.getSharedDataOffsets(); + const auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_offsets.size(); ++i) + { + size_t start = shared_data_offsets[ssize_t(i) - 1]; + size_t end = shared_data_offsets[ssize_t(i)]; + /// Merge sorted list of paths and values from shared data and sorted_typed_and_dynamic_paths_with_types + size_t sorted_paths_index = 0; + for (size_t j = start; j != end; ++j) + { + auto shared_data_path = shared_data_paths->getDataAt(j); + auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j)); + + while (sorted_paths_index != sorted_typed_and_dynamic_paths_with_types.size() && sorted_typed_and_dynamic_paths_with_types[sorted_paths_index].first < shared_data_path) + { + auto & [path, type] = sorted_typed_and_dynamic_paths_with_types[sorted_paths_index]; + paths_column->insertData(path.data(), path.size()); + /// Update type for path from dynamic paths. + if (auto it = dynamic_path_columns.find(path); it != dynamic_path_columns.end()) + type = getDynamicValueType(it->second, i); + types_column->insertData(type.data(), type.size()); + ++sorted_paths_index; + } + + paths_column->insertData(shared_data_path.data, shared_data_path.size); + types_column->insertData(type_name.data(), type_name.size()); + } + + for (; sorted_paths_index != sorted_typed_and_dynamic_paths_with_types.size(); ++sorted_paths_index) + { + auto & [path, type] = sorted_typed_and_dynamic_paths_with_types[sorted_paths_index]; + paths_column->insertData(path.data(), path.size()); + if (auto it = dynamic_path_columns.find(path); it != dynamic_path_columns.end()) + type = getDynamicValueType(it->second, i); + types_column->insertData(type.data(), type.size()); + } + + offsets.push_back(paths_column->size()); + } + + return ColumnMap::create(ColumnPtr(std::move(paths_column)), ColumnPtr(std::move(types_column)), ColumnPtr(std::move(offsets_column))); + } + + String getDynamicValueType(const ColumnPtr & column, size_t i) const + { + const ColumnDynamic * dynamic_column = checkAndGetColumn(column.get()); + const auto & variant_info = dynamic_column->getVariantInfo(); + const auto & variant_column = dynamic_column->getVariantColumn(); + auto global_discr = variant_column.globalDiscriminatorAt(i); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + return "None"; + + return variant_info.variant_names[global_discr]; + } + + String getDynamicValueTypeFromSharedData(StringRef value) const + { + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + return isNothing(type) ? "None" : type->getName(); + } +}; + +} + +REGISTER_FUNCTION(JSONPaths) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..cbaf2b4c1d0 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include @@ -854,6 +855,12 @@ void readBackQuotedString(String & s, ReadBuffer & buf) readBackQuotedStringInto(s, buf); } +bool tryReadBackQuotedString(String & s, ReadBuffer & buf) +{ + s.clear(); + return readAnyQuotedStringInto<'`', false, String, bool>(s, buf); +} + void readBackQuotedStringWithSQLStyle(String & s, ReadBuffer & buf) { s.clear(); @@ -1269,6 +1276,81 @@ ReturnType readJSONArrayInto(Vector & s, ReadBuffer & buf) template void readJSONArrayInto, void>(PaddedPODArray & s, ReadBuffer & buf); template bool readJSONArrayInto, bool>(PaddedPODArray & s, ReadBuffer & buf); +std::string_view readJSONObjectAsViewPossiblyInvalid(ReadBuffer & buf, String & object_buffer) +{ + if (buf.eof() || *buf.position() != '{') + throw Exception(ErrorCodes::INCORRECT_DATA, "JSON object should start with '{{'"); + + char * start = buf.position(); + bool use_object_buffer = false; + object_buffer.clear(); + + ++buf.position(); + Int64 balance = 1; + bool quotes = false; + + while (true) + { + if (!buf.hasPendingData() && !use_object_buffer) + { + use_object_buffer = true; + object_buffer.append(start, buf.position() - start); + } + + if (buf.eof()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading JSON object"); + + char * next_pos = find_first_symbols<'\\', '{', '}', '"'>(buf.position(), buf.buffer().end()); + if (use_object_buffer) + object_buffer.append(buf.position(), next_pos - buf.position()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (use_object_buffer) + object_buffer.push_back(*buf.position()); + + if (*buf.position() == '\\') + { + ++buf.position(); + if (!buf.hasPendingData() && !use_object_buffer) + { + use_object_buffer = true; + object_buffer.append(start, buf.position() - start); + } + + if (buf.eof()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading JSON object"); + + if (use_object_buffer) + object_buffer.push_back(*buf.position()); + ++buf.position(); + + continue; + } + + if (*buf.position() == '"') + quotes = !quotes; + else if (!quotes) // can be only opening_bracket or closing_bracket + balance += *buf.position() == '{' ? 1 : -1; + + ++buf.position(); + + if (balance == 0) + { + if (use_object_buffer) + return object_buffer; + return {start, buf.position()}; + } + + if (balance < 0) + break; + } + + throw Exception(ErrorCodes::INCORRECT_DATA, "JSON object should have equal number of opening and closing brackets"); +} + template ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf) { @@ -1894,6 +1976,11 @@ static ReturnType readParsedValueInto(Vector & s, ReadBuffer & buf, ParseFunc pa return ReturnType(true); } +void readParsedValueIntoString(String & s, ReadBuffer & buf, std::function parse_func) +{ + readParsedValueInto(s, buf, std::move(parse_func)); +} + template static ReturnType readQuotedStringFieldInto(Vector & s, ReadBuffer & buf) { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ffba4fafb5c..236c71b1bd9 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -600,6 +600,7 @@ bool tryReadDoubleQuotedStringWithSQLStyle(String & s, ReadBuffer & buf); void readJSONString(String & s, ReadBuffer & buf, const FormatSettings::JSON & settings); void readBackQuotedString(String & s, ReadBuffer & buf); +bool tryReadBackQuotedString(String & s, ReadBuffer & buf); void readBackQuotedStringWithSQLStyle(String & s, ReadBuffer & buf); void readStringUntilEOF(String & s, ReadBuffer & buf); @@ -687,6 +688,10 @@ ReturnType readJSONObjectPossiblyInvalid(Vector & s, ReadBuffer & buf); template ReturnType readJSONArrayInto(Vector & s, ReadBuffer & buf); +/// Similar to readJSONObjectPossiblyInvalid but avoids copying the data if JSON object fits into current read buffer +/// If copying is unavoidable, it copies data into provided object_buffer and returns string_view to it. +std::string_view readJSONObjectAsViewPossiblyInvalid(ReadBuffer & buf, String & object_buffer); + template void readStringUntilWhitespaceInto(Vector & s, ReadBuffer & buf); @@ -1893,6 +1898,8 @@ struct PcgDeserializer } }; +void readParsedValueIntoString(String & s, ReadBuffer & buf, std::function parse_func); + template ReturnType readQuotedFieldInto(Vector & s, ReadBuffer & buf); diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 6b0de441e94..306635272f5 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -90,6 +90,12 @@ inline void writeUUIDBinary(const UUID & x, WriteBuffer & buf) writePODBinary(uuid.items[1], buf); } +template +void setValue(char * data, const T & value) +{ + memcpy(data, reinterpret_cast(&value), sizeof(T)); +} + template inline void writeIntBinary(const T & x, WriteBuffer & buf) { diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index a3c5a7ed3ed..858d441a92f 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -47,10 +47,10 @@ #include #include -#include -#include -#include #include +#include +#include +#include #include #include @@ -1172,9 +1172,9 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (object_pos != std::string::npos) { String object_name = it->substr(0, object_pos); - if (pair.name == object_name && pair.type->getTypeId() == TypeIndex::Object) + if (pair.name == object_name && pair.type->getTypeId() == TypeIndex::ObjectDeprecated) { - const auto * object_type = typeid_cast(pair.type.get()); + const auto * object_type = typeid_cast(pair.type.get()); if (object_type->getSchemaFormat() == "json" && object_type->hasNullableSubcolumns()) { missed_subcolumns.insert(*it); diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 184c263dbdb..665f1a6ecd2 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -459,7 +459,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return src; } - else if (isObject(type)) + else if (isObjectDeprecated(type)) { if (src.getType() == Field::Types::Object) return src; /// Already in needed type. diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index 3529863a623..ed17a83e45e 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -110,7 +111,7 @@ void validateDataType(const DataTypePtr & type_to_check, const DataTypeValidatio if (!settings.allow_experimental_dynamic_type) { - if (data_type.hasDynamicSubcolumns()) + if (isDynamic(data_type)) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, @@ -119,6 +120,19 @@ void validateDataType(const DataTypePtr & type_to_check, const DataTypeValidatio data_type.getName()); } } + + if (!settings.allow_experimental_json_type) + { + const auto * object_type = typeid_cast(&data_type); + if (object_type && object_type->getSchemaFormat() == DataTypeObject::SchemaFormat::JSON) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot create column with type '{}' because experimental JSON type is not allowed. " + "Set setting allow_experimental_json_type = 1 in order to allow it", + data_type.getName()); + } + } }; validate_callback(*type_to_check); diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index e2d2bc97ff7..ae5d574229f 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -22,6 +22,7 @@ struct DataTypeValidationSettings , allow_suspicious_variant_types(settings.allow_suspicious_variant_types) , validate_nested_types(settings.validate_experimental_and_suspicious_types_inside_nested_types) , allow_experimental_dynamic_type(settings.allow_experimental_dynamic_type) + , allow_experimental_json_type(settings.allow_experimental_json_type) { } @@ -32,6 +33,7 @@ struct DataTypeValidationSettings bool allow_suspicious_variant_types = true; bool validate_nested_types = true; bool allow_experimental_dynamic_type = true; + bool allow_experimental_json_type = true; }; void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings); diff --git a/src/Parsers/ASTObjectTypeArgument.cpp b/src/Parsers/ASTObjectTypeArgument.cpp new file mode 100644 index 00000000000..afe875586ad --- /dev/null +++ b/src/Parsers/ASTObjectTypeArgument.cpp @@ -0,0 +1,75 @@ +#include +#include +#include + + +namespace DB +{ + +ASTPtr ASTObjectTypeArgument::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (path_with_type) + { + res->path_with_type = path_with_type->clone(); + res->children.push_back(res->path_with_type); + } + else if (skip_path) + { + res->skip_path = skip_path->clone(); + res->children.push_back(res->skip_path); + } + else if (skip_path_prefix) + { + res->skip_path_prefix = skip_path_prefix->clone(); + res->children.push_back(res->skip_path_prefix); + } + else if (skip_path_regexp) + { + res->skip_path_regexp = skip_path_regexp->clone(); + res->children.push_back(res->skip_path_regexp); + } + else if (parameter) + { + res->parameter = parameter->clone(); + res->children.push_back(res->parameter); + } + + return res; +} + +void ASTObjectTypeArgument::formatImpl(const FormatSettings & parameters, FormatState & state, FormatStateStacked frame) const +{ + if (path_with_type) + { + path_with_type->formatImpl(parameters, state, frame); + } + else if (parameter) + { + parameter->formatImpl(parameters, state, frame); + } + else if (skip_path) + { + std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); + parameters.ostr << indent_str << "SKIP" << ' '; + skip_path->formatImpl(parameters, state, frame); + } + else if (skip_path_prefix) + { + std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); + parameters.ostr << indent_str << "SKIP PREFIX" << ' '; + skip_path_prefix->formatImpl(parameters, state, frame); + } + else if (skip_path_regexp) + { + std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); + parameters.ostr << indent_str << "SKIP REGEXP" << ' '; + skip_path_regexp->formatImpl(parameters, state, frame); + } +} + +} + + diff --git a/src/Parsers/ASTObjectTypeArgument.h b/src/Parsers/ASTObjectTypeArgument.h new file mode 100644 index 00000000000..3a6b2bcdd98 --- /dev/null +++ b/src/Parsers/ASTObjectTypeArgument.h @@ -0,0 +1,35 @@ +#pragma once + +#include + + +namespace DB +{ + +/** An argument of Object data type declaration (for example for JSON). Can contain one of: + * - pair (path, data type) + * - path that should be skipped + * - path prefix for paths that should be skipped + * - path regexp for paths that should be skipped + * - setting in a form of `setting=N` + */ +class ASTObjectTypeArgument : public IAST +{ +public: + ASTPtr path_with_type; + ASTPtr skip_path; + ASTPtr skip_path_prefix; + ASTPtr skip_path_regexp; + ASTPtr parameter; + + /** Get the text that identifies this element. */ + String getID(char) const override { return "ASTObjectTypeArgument"; } + ASTPtr clone() const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + + +} + diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 0ae9ee4833c..73e633d6e6e 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -367,6 +367,7 @@ namespace DB MR_MACROS(POPULATE, "POPULATE") \ MR_MACROS(PRECEDING, "PRECEDING") \ MR_MACROS(PRECISION, "PRECISION") \ + MR_MACROS(PREFIX, "PREFIX") \ MR_MACROS(PREWHERE, "PREWHERE") \ MR_MACROS(PRIMARY_KEY, "PRIMARY KEY") \ MR_MACROS(PRIMARY, "PRIMARY") \ @@ -445,6 +446,7 @@ namespace DB MR_MACROS(SHOW, "SHOW") \ MR_MACROS(SIGNED, "SIGNED") \ MR_MACROS(SIMPLE, "SIMPLE") \ + MR_MACROS(SKIP, "SKIP") \ MR_MACROS(SOURCE, "SOURCE") \ MR_MACROS(SPATIAL, "SPATIAL") \ MR_MACROS(SQL_SECURITY, "SQL SECURITY") \ @@ -635,6 +637,32 @@ protected: } }; +class ParserTokenSequence : public IParserBase +{ +private: + std::vector token_types; +public: + ParserTokenSequence(const std::vector & token_types_) : token_types(token_types_) {} /// NOLINT + +protected: + const char * getName() const override { return "token sequence"; } + + bool parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected) override + { + for (auto token_type : token_types) + { + if (pos->type != token_type) + { + expected.add(pos, getTokenName(token_type)); + return false; + } + + ++pos; + } + + return true; + } +}; // Parser always returns true and do nothing. class ParserNothing : public IParserBase diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index d4fc9a4bc4d..d3175110db9 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -285,19 +285,40 @@ bool ParserTableAsStringLiteralIdentifier::parseImpl(Pos & pos, ASTPtr & node, E bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - ASTPtr id_list; - if (!ParserList(std::make_unique(allow_query_parameter, highlight_type), std::make_unique(TokenType::Dot), false) - .parse(pos, id_list, expected)) - return false; + auto element_parser = std::make_unique(allow_query_parameter, highlight_type); + std::vector> delimiter_parsers; + delimiter_parsers.emplace_back(std::make_unique(std::vector{TokenType::Dot, TokenType::Colon}), SpecialDelimiter::JSON_PATH_DYNAMIC_TYPE); + delimiter_parsers.emplace_back(std::make_unique(std::vector{TokenType::Dot, TokenType::Caret}), SpecialDelimiter::JSON_PATH_PREFIX); + delimiter_parsers.emplace_back(std::make_unique(TokenType::Dot), SpecialDelimiter::NONE); std::vector parts; + SpecialDelimiter last_special_delimiter = SpecialDelimiter::NONE; ASTs params; - const auto & list = id_list->as(); - for (const auto & child : list.children) + + bool parsed_delimiter = true; + while (parsed_delimiter) { - parts.emplace_back(getIdentifierName(child)); + ASTPtr element; + if (!element_parser->parse(pos, element, expected)) + return false; + if (last_special_delimiter != SpecialDelimiter::NONE) + parts.push_back(static_cast(last_special_delimiter) + backQuote(getIdentifierName(element))); + else + parts.push_back(getIdentifierName(element)); + if (parts.back().empty()) - params.push_back(child->as()->getParam()); + params.push_back(element->as()->getParam()); + + parsed_delimiter = false; + for (const auto & [parser, special_delimiter] : delimiter_parsers) + { + if (parser->check(pos, expected)) + { + parsed_delimiter = true; + last_special_delimiter = special_delimiter; + break; + } + } } ParserKeyword s_uuid(Keyword::UUID); diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 4e3f29bfe0c..39608bc583e 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -52,11 +52,21 @@ protected: /** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime, - * possibly with UUID clause like `db name`.`table name` UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx' + * possibly with UUID clause like `db name`.`table name` UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx'. + * There is also special delimiters `.:` and `.^` for JSON type subcolumns. In case of special delimiter + * the next identifier part after it will include special delimiter and be back quoted always: json.a.b.:UInt32 -> ['json', 'a', 'b', ':`UInt32`']. + * It's needed to distinguish identifiers json.a.b.:UInt32 and json.a.b.`:UInt32`. */ class ParserCompoundIdentifier : public IParserBase { public: + enum class SpecialDelimiter : char + { + NONE = '\0', + JSON_PATH_DYNAMIC_TYPE = ':', + JSON_PATH_PREFIX = '^', + }; + explicit ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false, bool allow_query_parameter_ = false, Highlight highlight_type_ = Highlight::identifier) : table_name_with_optional_uuid(table_name_with_optional_uuid_), allow_query_parameter(allow_query_parameter_), highlight_type(highlight_type_) { diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index b4601389696..43c4ab867d1 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -423,6 +423,8 @@ Token Lexer::nextTokenImpl() } case '?': return Token(TokenType::QuestionMark, token_begin, ++pos); + case '^': + return Token(TokenType::Caret, token_begin, ++pos); case ':': { ++pos; diff --git a/src/Parsers/Lexer.h b/src/Parsers/Lexer.h index 6f31d56292d..9dc0850abfd 100644 --- a/src/Parsers/Lexer.h +++ b/src/Parsers/Lexer.h @@ -45,6 +45,7 @@ namespace DB M(Arrow) /** ->. Should be distinguished from minus operator. */ \ M(QuestionMark) \ M(Colon) \ + M(Caret) \ M(DoubleColon) \ M(Equals) \ M(NotEquals) \ diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 014dc7bd3bf..9774a3a65cd 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -62,7 +62,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr name; ASTPtr columns; - /// For now `name == 'Nested'`, probably alternative nested data structures will appear + /// For now `name == 'Nested'` or `name == 'Tuple'`, probably alternative nested data structures will appear if (!name_p.parse(pos, name, expected)) return false; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index ad33c7e4558..2a492e8eaed 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -4,11 +4,13 @@ #include #include #include +#include #include #include #include #include +#include namespace DB { @@ -46,6 +48,89 @@ private: } }; +/// Parser of Object type argument. For example: JSON(some_parameter=N, some.path SomeType, SKIP skip.path, ...) +class ObjectArgumentParser : public IParserBase +{ +private: + const char * getName() const override { return "JSON data type optional argument"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + auto argument = std::make_shared(); + + /// SKIP arguments + if (ParserKeyword(Keyword::SKIP).ignore(pos)) + { + /// SKIP REGEXP '' + if (ParserKeyword(Keyword::REGEXP).ignore(pos)) + { + ParserStringLiteral literal_parser; + ASTPtr literal; + if (!literal_parser.parse(pos, literal, expected)) + return false; + argument->skip_path_regexp = literal; + argument->children.push_back(argument->skip_path_regexp); + } + /// SKIP PREFIX some.path.prefix or SKIP some.path + else + { + bool is_prefix = ParserKeyword(Keyword::PREFIX).ignore(pos); + ParserCompoundIdentifier compound_identifier_parser; + ASTPtr compound_identifier; + if (!compound_identifier_parser.parse(pos, compound_identifier, expected)) + return false; + + if (is_prefix) + { + argument->skip_path_prefix = compound_identifier; + argument->children.push_back(argument->skip_path_prefix); + } + else + { + argument->skip_path = compound_identifier; + argument->children.push_back(argument->skip_path); + } + } + + node = argument; + return true; + } + + ParserCompoundIdentifier compound_identifier_parser; + ASTPtr identifier; + if (!compound_identifier_parser.parse(pos, identifier, expected)) + return false; + + /// some_parameter=N + if (pos->type == TokenType::Equals) + { + ++pos; + ASTPtr number; + ParserNumber number_parser; + if (!number_parser.parse(pos, number, expected)) + return false; + + argument->parameter = makeASTFunction("equals", identifier, number); + argument->children.push_back(argument->parameter); + node = argument; + return true; + } + + ParserDataType type_parser; + ASTPtr type; + if (!type_parser.parse(pos, type, expected)) + return false; + + auto name_and_type = std::make_shared(); + name_and_type->name = getIdentifierName(identifier); + name_and_type->type = type; + name_and_type->children.push_back(name_and_type->type); + argument->path_with_type = name_and_type; + argument->children.push_back(argument->path_with_type); + node = argument; + return true; + } +}; + /// Wrapper to allow mixed lists of nested and normal types. /// Parameters are either: /// - Nested table elements; @@ -70,17 +155,21 @@ private: return parser.parse(pos, node, expected); } - ParserNestedTable nested_parser; + if (type_name == "JSON") + { + ObjectArgumentParser parser; + return parser.parse(pos, node, expected); + } + + ParserNameTypePair name_and_type_parser; ParserDataType data_type_parser; ParserAllCollectionsOfLiterals literal_parser(false); const char * operators[] = {"=", "equals", nullptr}; ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); - if (pos->type == TokenType::BareWord && std::string_view(pos->begin, pos->size()) == "Nested") - return nested_parser.parse(pos, node, expected); - - return enum_parser.parse(pos, node, expected) + return name_and_type_parser.parse(pos, node, expected) + || enum_parser.parse(pos, node, expected) || literal_parser.parse(pos, node, expected) || data_type_parser.parse(pos, node, expected); } @@ -92,10 +181,6 @@ private: bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - ParserNestedTable nested; - if (nested.parse(pos, node, expected)) - return true; - String type_name; ParserIdentifier name_parser; diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index f57e62adba9..c5e36046c62 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -166,10 +166,11 @@ JSONAsObjectRowInputFormat::JSONAsObjectRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) : JSONAsRowInputFormat(header_, in_, params_, format_settings_) { - if (!isObject(header_.getByPosition(0).type)) + const auto & type = header_.getByPosition(0).type; + if (!isObject(type) && !isObjectDeprecated(type)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Input format JSONAsObject is only suitable for tables with a single column of type Object but the column type is {}", - header_.getByPosition(0).type->getName()); + "Input format JSONAsObject is only suitable for tables with a single column of type Object/JSON but the column type is {}", + type->getName()); } void JSONAsObjectRowInputFormat::readJSONObject(IColumn & column) @@ -184,13 +185,13 @@ Chunk JSONAsObjectRowInputFormat::getChunkForCount(size_t rows) return Chunk({std::move(column)}, rows); } -JSONAsObjectExternalSchemaReader::JSONAsObjectExternalSchemaReader(const FormatSettings & settings) +JSONAsObjectExternalSchemaReader::JSONAsObjectExternalSchemaReader(const FormatSettings & settings_) : settings(settings_) { - if (!settings.json.allow_object_type) + if (!settings.json.allow_deprecated_object_type && !settings.json.allow_json_type) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Cannot infer the data structure in JSONAsObject format because experimental Object type is not allowed. Set setting " - "allow_experimental_object_type = 1 in order to allow it"); + "Cannot infer the data structure in JSONAsObject format because experimental Object/JSON type is not allowed. Set setting " + "allow_experimental_object_type = 1 or allow_experimental_json_type=1 in order to allow it"); } void registerInputFormatJSONAsString(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index 16112325a97..a5a436260ba 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -1,11 +1,12 @@ #pragma once -#include -#include +#include +#include +#include #include #include -#include -#include +#include +#include namespace DB { @@ -69,12 +70,17 @@ public: class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader { public: - explicit JSONAsObjectExternalSchemaReader(const FormatSettings & settings); + explicit JSONAsObjectExternalSchemaReader(const FormatSettings & settings_); NamesAndTypesList readSchema() override { - return {{"json", std::make_shared("json", false)}}; + if (settings.json.allow_json_type) + return {{"json", std::make_shared(DataTypeObject::SchemaFormat::JSON)}}; + return {{"json", std::make_shared("json", false)}}; } + +private: + FormatSettings settings; }; } diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index ed4b1906635..42e545828cc 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -49,7 +49,7 @@ bool isSafePrimaryDataKeyType(const IDataType & data_type) case TypeIndex::Float32: case TypeIndex::Float64: case TypeIndex::Nullable: - case TypeIndex::Object: + case TypeIndex::ObjectDeprecated: return false; case TypeIndex::Array: { diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index c07583cd39d..68c11060d88 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -701,17 +701,16 @@ std::optional ColumnsDescription::tryGetColumn(const GetColumns auto jt = subcolumns.get<0>().find(column_name); if (jt != subcolumns.get<0>().end()) return *jt; - } - /// Check for dynamic subcolumns. - auto [ordinary_column_name, dynamic_subcolumn_name] = Nested::splitName(column_name); - it = columns.get<1>().find(ordinary_column_name); - if (it != columns.get<1>().end() && it->type->hasDynamicSubcolumns()) - { - if (auto dynamic_subcolumn_type = it->type->tryGetSubcolumnType(dynamic_subcolumn_name)) - return NameAndTypePair(ordinary_column_name, dynamic_subcolumn_name, it->type, dynamic_subcolumn_type); + /// Check for dynamic subcolumns. + auto [ordinary_column_name, dynamic_subcolumn_name] = Nested::splitName(column_name); + it = columns.get<1>().find(ordinary_column_name); + if (it != columns.get<1>().end() && it->type->hasDynamicSubcolumns()) + { + if (auto dynamic_subcolumn_type = it->type->tryGetSubcolumnType(dynamic_subcolumn_name)) + return NameAndTypePair(ordinary_column_name, dynamic_subcolumn_name, it->type, dynamic_subcolumn_type); + } } - return {}; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 52d12c9db7d..f4be7619fc8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -164,7 +164,7 @@ void writeColumnSingleGranule( serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; serialize_settings.use_compact_variant_discriminators_serialization = settings.use_compact_variant_discriminators_serialization; - serialize_settings.dynamic_write_statistics = ISerialization::SerializeBinaryBulkSettings::DynamicStatisticsMode::PREFIX; + serialize_settings.object_and_dynamic_write_statistics = ISerialization::SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::PREFIX; serialization->serializeBinaryBulkStatePrefix(*column.column, serialize_settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 74ea89a8864..b3413437a31 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -131,6 +131,9 @@ void MergeTreeDataPartWriterWide::addStreams( { assert(!substream_path.empty()); + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto full_stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); String stream_name; @@ -204,6 +207,9 @@ ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGett { return [&, this] (const ISerialization::SubstreamPath & substream_path) -> WriteBuffer * { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return nullptr; + bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; auto stream_name = getStreamName(column, substream_path); @@ -366,6 +372,9 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( min_compress_block_size = settings.min_compress_block_size; getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; auto stream_name = getStreamName(name_and_type, substream_path); @@ -404,6 +413,9 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; auto stream_name = getStreamName(name_and_type, substream_path); @@ -655,7 +667,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(MergeTreeDataPartChecksums & if (!serialization_states.empty()) { serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); - serialize_settings.dynamic_write_statistics = ISerialization::SerializeBinaryBulkSettings::DynamicStatisticsMode::SUFFIX; + serialize_settings.object_and_dynamic_write_statistics = ISerialization::SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::SUFFIX; getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index a2b8f0ad96f..192866260de 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -206,7 +206,7 @@ void MergeTreeReaderCompact::readPrefix( serialization = getSerializationInPart(name_and_type); deserialize_settings.getter = buffer_getter; - deserialize_settings.dynamic_read_statistics = true; + deserialize_settings.object_and_dynamic_read_statistics = true; serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name_and_type.name], nullptr); } catch (Exception & e) diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index b6882fdced9..c017d7db786 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -326,7 +326,7 @@ void MergeTreeReaderWide::deserializePrefix( if (!deserialize_binary_bulk_state_map.contains(name)) { ISerialization::DeserializeBinaryBulkSettings deserialize_settings; - deserialize_settings.dynamic_read_statistics = true; + deserialize_settings.object_and_dynamic_read_statistics = true; deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { return getStream(/* seek_to_start = */true, substream_path, data_part_info_for_read->getChecksums(), name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, cache); From fb4317f12c355f3de7c633b5545952c77b48d583 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 14:46:33 +0000 Subject: [PATCH 0263/1722] Small clean up --- .../Passes/FunctionToSubcolumnsPass.cpp | 17 ----- src/Analyzer/Resolve/QueryAnalyzer.cpp | 12 ++-- src/DataTypes/DataTypeDynamic.cpp | 1 - src/DataTypes/ObjectUtils.cpp | 36 +++++----- src/DataTypes/Serializations/ISerialization.h | 65 ------------------- .../SerializationJSONElement.cpp | 3 - .../Serializations/SerializationJSONElement.h | 8 --- .../SerializationObjectElement.cpp | 3 - .../SerializationObjectElement.h | 8 --- .../gtest_data_types_binary_encoding.cpp | 2 +- src/Formats/JSONUtils.cpp | 12 ++-- src/Formats/SchemaInferenceUtils.cpp | 30 ++++----- src/Functions/FunctionsConversion.cpp | 7 +- src/Parsers/ParserDataType.cpp | 1 - 14 files changed, 50 insertions(+), 155 deletions(-) delete mode 100644 src/DataTypes/Serializations/SerializationJSONElement.cpp delete mode 100644 src/DataTypes/Serializations/SerializationJSONElement.h delete mode 100644 src/DataTypes/Serializations/SerializationObjectElement.cpp delete mode 100644 src/DataTypes/Serializations/SerializationObjectElement.h diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 395ebbfa408..90051779a26 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -52,8 +52,6 @@ void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, Col /// Replace `notEmpty(argument)` with `notEquals(argument.size0, 0)` if not positive /// `argument` may be Array or Map. - std::cerr << "optimizeFunctionEmpty " << ctx.column.name << "\n"; - NameAndTypePair column{ctx.column.name + ".size0", std::make_shared()}; auto & function_arguments_nodes = function_node.getArguments().getNodes(); @@ -236,18 +234,9 @@ std::tuple getTypedNodesForOptimizati return {}; auto column_in_table = storage_snapshot->tryGetColumn(GetColumnsOptions::All, column.name); - std::cerr << "getTypedNodesForOptimization " << column.name << "\n"; if (!column_in_table || !column_in_table->type->equals(*column.type)) - { - std::cerr << "getTypedNodesForOptimization FAIL\n"; - if (column_in_table) - std::cerr << column_in_table->type->getName() << "/" << column.type->getName() << "\n"; - else - std::cerr << "null\n"; return {}; - } - std::cerr << "getTypedNodesForOptimization OK\n"; return std::make_tuple(function_node, first_argument_column_node, table_node); } @@ -433,15 +422,9 @@ public: auto table_name = table_node->getStorage()->getStorageID().getFullTableName(); Identifier qualified_name({table_name, column.name}); - std::cerr << "FunctionToSubcolumnsVisitorSecondPass " << column.name << "\n"; - if (!identifiers_to_optimize.contains(qualified_name)) - { - std::cerr << "FunctionToSubcolumnsVisitorSecondPass FAIL\n"; return; - } - std::cerr << "FunctionToSubcolumnsVisitorSecondPass OK\n"; auto transformer_it = node_transformers.find({column.type->getTypeId(), function_node->getFunctionName()}); if (transformer_it != node_transformers.end()) { diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b8d1e44fca7..911b8bc4606 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -1,14 +1,14 @@ #include -#include -#include -#include +#include +#include #include #include -#include -#include #include -#include +#include +#include +#include +#include #include #include diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index bb7768e16ad..88871f28f06 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -130,7 +130,6 @@ std::pair splitSubcolumnName(std::string_vie std::unique_ptr DataTypeDynamic::getDynamicSubcolumnData(std::string_view subcolumn_name, const DB::IDataType::SubstreamData & data, bool throw_if_null) const { auto [subcolumn_type_name, subcolumn_nested_name] = splitSubcolumnName(subcolumn_name); -// std::cerr << "Dynamic subcolumn: " << subcolumn_name << ", " << subcolumn_nested_name << "\n"; /// Check if requested subcolumn is a valid data type. auto subcolumn_type = DataTypeFactory::instance().tryGet(String(subcolumn_type_name)); diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 4df240daa8f..51588ce2ba3 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -3,28 +3,28 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include +#include +#include +#include +#include namespace DB diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 51fcdaec6ab..22bbdb25c43 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -491,69 +491,4 @@ State * ISerialization::checkAndGetState(const StatePtr & state, const ISerializ return state_concrete; } - -WITH snaps AS ( - SELECT - rs.brokerfk brokerfk, - rs.brokerinstancefk brokerinstancefk, - rs.tradedt tradedt, - min(rs.snapdttm) snapdttmmin, - max(rs.snapdttm) snapdttmmax - FROM risksnapshothistory rs - WHERE rs.snapdttm >= now() - interval '10 minutes' - AND rs.snapdttm <= now() - GROUP BY rs.brokerfk, rs.brokerinstancefk, rs.tradedt - ORDER BY rs.brokerfk, rs.brokerinstancefk, rs.tradedt - ), - start_snaps_by_trade_dt AS ( - SELECT - rs.* - FROM risksnapshothistory rs - WHERE (rs.brokerfk, rs.brokerinstancefk, rs.tradedt, rs.snapdttm) IN ( - SELECT - s.brokerfk s_brokerfk, - s.brokerinstancefk s_brokerinstancefk, - s.tradedt s_tradedt, - s.snapdttmmin s_snapdttmmin - FROM snaps s) - ORDER BY rs.brokerfk, rs.brokerinstancefk, rs.snapdttm - ), - end_snaps_by_trade_dt AS ( - SELECT - rs.* - FROM risksnapshothistory rs - WHERE (rs.brokerfk, rs.brokerinstancefk, rs.tradedt, rs.snapdttm) IN ( - SELECT - s.brokerfk s_brokerfk, - s.brokerinstancefk s_brokerinstancefk, - s.tradedt s_tradedt, - s.snapdttmmax s_snapdttmmax - FROM snaps s) - ORDER BY rs.brokerfk, rs.brokerinstancefk, rs.snapdttm - ), - data as ( - SELECT - ssbtd.snapdttm startdttm, - esbtd.snapdttm enddttm, - case when(ssbtd.snapdttm = '1970-01-01 00:00:00.000000') then 0 else 1 end hasstart, - case when(esbtd.snapdttm = '1970-01-01 00:00:00.000000') then 0 else 1 end hasend, - IF(esbtd.brokerfk = 0, ssbtd.brokerfk, esbtd.brokerfk) brokerfk, - IF(esbtd.brokerinstancefk = 0, ssbtd.brokerinstancefk, esbtd.brokerinstancefk) brokerinstancefk - FROM start_snaps_by_trade_dt ssbtd - FULL OUTER JOIN end_snaps_by_trade_dt esbtd ON ssbtd.brokerfk = esbtd.brokerfk AND ssbtd.brokerinstancefk = esbtd.brokerinstancefk AND ssbtd.tradedt = esbtd.tradedt AND ssbtd.traderfk = esbtd.traderfk AND ssbtd.brokersymbolfk = esbtd.brokersymbolfk - ) - SELECT - d.brokerfk, d.brokerinstancefk, - max(startdttm) startdttm, - max(enddttm) enddttm, - countIf(d.hasstart =1 and d.hasend=1) matches, - countIf(d.hasstart =1 and d.hasend=0) beforeonly, - countIf(d.hasstart =0 and d.hasend=1) afteronly, - (select array_agg(toJSONString(map( - 'snapdttmmax', toString(snapdttmmax), 'snapdttmmin', toString(snapdttmmin), - 'tradedt', toString(tradedt), 'brokerfk', toString(brokerfk), 'brokerinstancefk', toString(brokerinstancefk)))) - from snaps) snaps - FROM data d - GROUP BY d.brokerfk, d.brokerinstancefk - } diff --git a/src/DataTypes/Serializations/SerializationJSONElement.cpp b/src/DataTypes/Serializations/SerializationJSONElement.cpp deleted file mode 100644 index 35119cd0764..00000000000 --- a/src/DataTypes/Serializations/SerializationJSONElement.cpp +++ /dev/null @@ -1,3 +0,0 @@ -// -// Created by Павел Круглов on 05/07/2024. -// diff --git a/src/DataTypes/Serializations/SerializationJSONElement.h b/src/DataTypes/Serializations/SerializationJSONElement.h deleted file mode 100644 index 791ef2eecc5..00000000000 --- a/src/DataTypes/Serializations/SerializationJSONElement.h +++ /dev/null @@ -1,8 +0,0 @@ -// -// Created by Павел Круглов on 05/07/2024. -// - -#ifndef CLICKHOUSE_SERIALIZATIONJSONELEMENT_H -#define CLICKHOUSE_SERIALIZATIONJSONELEMENT_H - -#endif //CLICKHOUSE_SERIALIZATIONJSONELEMENT_H diff --git a/src/DataTypes/Serializations/SerializationObjectElement.cpp b/src/DataTypes/Serializations/SerializationObjectElement.cpp deleted file mode 100644 index 35119cd0764..00000000000 --- a/src/DataTypes/Serializations/SerializationObjectElement.cpp +++ /dev/null @@ -1,3 +0,0 @@ -// -// Created by Павел Круглов on 05/07/2024. -// diff --git a/src/DataTypes/Serializations/SerializationObjectElement.h b/src/DataTypes/Serializations/SerializationObjectElement.h deleted file mode 100644 index 791ef2eecc5..00000000000 --- a/src/DataTypes/Serializations/SerializationObjectElement.h +++ /dev/null @@ -1,8 +0,0 @@ -// -// Created by Павел Круглов on 05/07/2024. -// - -#ifndef CLICKHOUSE_SERIALIZATIONJSONELEMENT_H -#define CLICKHOUSE_SERIALIZATIONJSONELEMENT_H - -#endif //CLICKHOUSE_SERIALIZATIONJSONELEMENT_H diff --git a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp index 2e646d1e62a..789aeac566f 100644 --- a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp +++ b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp @@ -32,7 +32,7 @@ extern const int UNSUPPORTED_METHOD; void check(const DataTypePtr & type) { - std::cerr << "Check " << type->getName() << "\n"; +// std::cerr << "Check " << type->getName() << "\n"; WriteBufferFromOwnString ostr; encodeDataType(type, ostr); ReadBufferFromString istr(ostr.str()); diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index f859ad73ea6..f41c3be71f4 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -1,13 +1,13 @@ -#include -#include -#include -#include -#include +#include #include #include +#include #include -#include #include +#include +#include +#include +#include #include diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 0f8dc83bd52..15dc75c0f3a 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1,22 +1,22 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include +#include #include #include diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 0b2cfcab165..3b0179dff2b 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -16,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -42,12 +41,13 @@ #include #include #include +#include #include #include #include #include -#include #include +#include #include #include #include @@ -4020,6 +4020,7 @@ private: /// TODO: support CAST between JSON types with different parameters /// support CAST from Map to JSON /// support CAST from Tuple to JSON + /// support CAST from Object('json') to JSON throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to {} can be performed only from String. Got: {}", magic_enum::enum_name(to_object->getSchemaFormat()), from_type->getName()); } diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 2a492e8eaed..f4793c7fc27 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -10,7 +10,6 @@ #include #include -#include namespace DB { From 2b28323e65a7ba8f9a1be1c3c6a5953f9332709a Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 14:48:38 +0000 Subject: [PATCH 0264/1722] Add documentation --- docs/en/sql-reference/data-types/dynamic.md | 3 +- docs/en/sql-reference/data-types/json.md | 519 ++++++++++++++++-- .../sql-reference/data-types/object-json.md | 83 +++ .../sql-reference/data-types/object_json.md | 86 +++ 4 files changed, 629 insertions(+), 62 deletions(-) create mode 100644 docs/en/sql-reference/data-types/object-json.md create mode 100644 docs/en/sql-reference/data-types/object_json.md diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index 84a724e121e..44421f3b1f8 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -533,5 +533,4 @@ In RowBinary format values of `Dynamic` type are serialized in the following for ```text -``` - +``` \ No newline at end of file diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index c29be2cff58..d35a3f90ef3 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -1,83 +1,482 @@ --- -slug: /en/sql-reference/data-types/object-data-type -sidebar_position: 26 -sidebar_label: Object Data Type -keywords: [object, data type] +slug: /en/sql-reference/data-types/json +sidebar_position: 63 +sidebar_label: JSON --- -# Object Data Type - -:::note -This feature is not production-ready and is now deprecated. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864) -::: +# JSON` Stores JavaScript Object Notation (JSON) documents in a single column. -`JSON` is an alias for `Object('json')`. +:::note +This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. +If you want to use JSON type, set `allow_experimental_json_type = 1`. +::: -## Example +To declare a column of `JSON` type, use the following syntax: -**Example 1** - -Creating a table with a `JSON` column and inserting data into it: - -```sql -CREATE TABLE json -( - o JSON -) -ENGINE = Memory +``` sql + JSON(max_dynamic_paths=N, max_dynamic_types=M, some.path TypeName, SKIP path.to.skip, SKIP PREFIX path.prefix.to.skip, SKIP REGEXP 'paths_regexp') ``` +Where: +- `max_dynamic_paths` is an optional parameter indicating how many paths can be stored separately as subcolumns across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all other paths will be stored together in a single structure. Default value of `max_dynamic_paths` is `1000`. +- `max_dynamic_types` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a single path column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_dynamic_types` is `32`. +- `some.path TypeName` is an optional type hint for particular path in the JSON. Such paths will be always stored as subcolumns with specified type. +- `SKIP path.to.skip` is an optional hint for particular path that should be skipped during JSON parsing. Such paths will never be stored in the JSON column. +- `SKIP PREFIX path.prefix.to.skip` is an optional hint for particular path prefix that should be skipped during JSON parsing. All paths with such prefix will never be stored in the JSON column. +- `SKIP REGEXP 'path_regexp'` is an optional hint with a regular expression that is used to skip paths during JSON parsing. All paths that match this regular expression will never be stored in the JSON column. + +## Creating JSON + +Using `JSON` type in table column definition: ```sql -INSERT INTO json VALUES ('{"a": 1, "b": { "c": 2, "d": [1, 2, 3] }}') -``` - -```sql -SELECT o.a, o.b.c, o.b.d[3] FROM json +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : 42}, "c" : [1, 2, 3]}'), ('{"f" : "Hello, World!"}'), ('{"a" : {"b" : 43, "e" : 10}, "c" : [4, 5, 6]}'); +SELECT json FROM test; ``` ```text -┌─o.a─┬─o.b.c─┬─arrayElement(o.b.d, 3)─┠-│ 1 │ 2 │ 3 │ -└─────┴───────┴────────────────────────┘ -``` - -**Example 2** - -To be able to create an ordered `MergeTree` family table, the sorting key has to be extracted into its column. For example, to insert a file of compressed HTTP access logs in JSON format: - -```sql -CREATE TABLE logs -( - timestamp DateTime, - message JSON -) -ENGINE = MergeTree -ORDER BY timestamp +┌─json────────────────────────────────────────┠+│ {"a":{"b":"42"},"c":["1","2","3"]} │ +│ {"f":"Hello, World!"} │ +│ {"a":{"b":"43","e":"10"},"c":["4","5","6"]} │ +└─────────────────────────────────────────────┘ ``` ```sql -INSERT INTO logs -SELECT parseDateTimeBestEffort(JSONExtractString(json, 'timestamp')), json -FROM file('access.json.gz', JSONAsString) -``` - -## Displaying JSON columns - -When displaying a `JSON` column, ClickHouse only shows the field values by default (because internally, it is represented as a tuple). You can also display the field names by setting `output_format_json_named_tuples_as_objects = 1`: - -```sql -SET output_format_json_named_tuples_as_objects = 1 - -SELECT * FROM json FORMAT JSONEachRow +CREATE TABLE test (json JSON(a.b UInt32, SKIP a.e)) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : 42}, "c" : [1, 2, 3]}'), ('{"f" : "Hello, World!"}'), ('{"a" : {"b" : 43, "e" : 10}, "c" : [4, 5, 6]}'); +SELECT json FROM test; ``` ```text -{"o":{"a":1,"b":{"c":2,"d":[1,2,3]}}} +┌─json──────────────────────────────┠+│ {"a":{"b":42},"c":[1,2,3]} │ +│ {"a":{"b":0},"f":"Hello, World!"} │ +│ {"a":{"b":43},"c":[4,5,6]} │ +└───────────────────────────────────┘ ``` -## Related Content +Using CAST from 'String': -- [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) -- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) +```sql +SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json; +``` + +```text +┌─json───────────────────────────────────────────┠+│ {"a":{"b":42},"c":[1,2,3],"d":"Hello, World!"} │ +└────────────────────────────────────────────────┘ +``` + +CAST from named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. + +## Reading JSON paths as subcolumns + +JSON type supports reading every path as a separate subcolumn. If type of the requested path was not specified in the JSON type declaration, the subcolumn of the path will always have type [Dynamic](/docs/en/sql-reference/data-types/dynamic.md). + +For example: + +```sql +CREATE TABLE test (json JSON(a.b UInt32, SKIP a.e)) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : 42, "g" : 42.42}, "c" : [1, 2, 3], "d" : "2020-01-01"}'), ('{"f" : "Hello, World!", "d" : "2020-01-02"}'), ('{"a" : {"b" : 43, "e" : 10, "g" : 43.43}, "c" : [4, 5, 6]}'); +SELECT json FROM test; +``` + +```text +┌─json──────────────────────────────────────────────────┠+│ {"a":{"b":42,"g":42.42},"c":[1,2,3],"d":"2020-01-01"} │ +│ {"a":{"b":0},"d":"2020-01-02","f":"Hello, World!"} │ +│ {"a":{"b":43,"g":43.43},"c":[4,5,6]} │ +└───────────────────────────────────────────────────────┘ +``` + +```sql +SELECT json.a.b, json.a.g, json.c, json.d FROM test; +``` + +```text +┌─json.a.b─┬─json.a.g─┬─json.c──┬─json.d─────┠+│ 42 │ 42.42 │ [1,2,3] │ 2020-01-01 │ +│ 0 │ á´ºáµá´¸á´¸ │ á´ºáµá´¸á´¸ │ 2020-01-02 │ +│ 43 │ 43.43 │ [4,5,6] │ á´ºáµá´¸á´¸ │ +└──────────┴──────────┴─────────┴────────────┘ +``` + +If the requested path wasn't found in the data, it will be filled with `NULL` values: + +```sql +SELECT json.non.existing.path FROM test; +``` + +```text +┌─json.non.existing.path─┠+│ á´ºáµá´¸á´¸ │ +│ á´ºáµá´¸á´¸ │ +│ á´ºáµá´¸á´¸ │ +└────────────────────────┘ +``` + +Let's check the data types of returned subcolumns: +```sql +SELECT toTypeName(json.a.b), toTypeName(json.a.g), toTypeName(json.c), toTypeName(json.d) FROM test; +``` + +```text +┌─toTypeName(json.a.b)─┬─toTypeName(json.a.g)─┬─toTypeName(json.c)─┬─toTypeName(json.d)─┠+│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ +│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ +│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ +└──────────────────────┴──────────────────────┴────────────────────┴────────────────────┘ +``` + +As we can see, for `a.b` the type is `UInt32` as we specified in the JSON type declaration, and for all other subcolumns the type is `Dynamic`. + +It is also possible to read subcolumns of a `Dynamic` type using special syntax `json.some.path.:TypeName`: + +```sql +select json.a.g.:Float64, dynamicType(json.a.g), json.d.:Date, dynamicType(json.d) FROM test; +``` + +```text +┌─json.a.g.:`Float64`─┬─dynamicType(json.a.g)─┬─json.d.:`Date`─┬─dynamicType(json.d)─┠+│ 42.42 │ Float64 │ 2020-01-01 │ Date │ +│ á´ºáµá´¸á´¸ │ None │ 2020-01-02 │ Date │ +│ 43.43 │ Float64 │ á´ºáµá´¸á´¸ │ None │ +└─────────────────────┴───────────────────────┴────────────────┴─────────────────────┘ +``` + +`Dynamic` subcolumns can be casted to any data type. In this case the exception will be thrown if internal type inside `Dynamic` cannot be casted to the requested type: + +```sql +select json.a.g::UInt64 as uint FROM test; +``` + +```text +┌─uint─┠+│ 42 │ +│ 0 │ +│ 43 │ +└──────┘ +``` + +```sql +select json.a.g::UUID as float FROM test; +``` + +```text +Received exception: +Code: 48. DB::Exception: Conversion between numeric types and UUID is not supported. Probably the passed UUID is unquoted: while executing 'FUNCTION CAST(__table1.json.a.g :: 2, 'UUID'_String :: 1) -> CAST(__table1.json.a.g, 'UUID'_String) UUID : 0'. (NOT_IMPLEMENTED) +``` + +## Reading JSON sub-objects as subcolumns + +JSON type supports reading nested objects as subcolumns with type `JSON` using special syntax `json.^some.path`: + +```sql +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : {"c" : 42, "g" : 42.42}}, "c" : [1, 2, 3], "d" : {"e" : {"f" : {"g" : "Hello, World", "h" : [1, 2, 3]}}}}'), ('{"f" : "Hello, World!", "d" : {"e" : {"f" : {"h" : [4, 5, 6]}}}}'), ('{"a" : {"b" : {"c" : 43, "e" : 10, "g" : 43.43}}, "c" : [4, 5, 6]}'); +SELECT json FROM test; +``` + +```text +┌─json────────────────────────────────────────────────────────────────────────────────────────┠+│ {"a":{"b":{"c":42,"g":42.42}},"c":[1,2,3],"d":{"e":{"f":{"g":"Hello, World","h":[1,2,3]}}}} │ +│ {"d":{"e":{"f":{"h":[4,5,6]}}},"f":"Hello, World!"} │ +│ {"a":{"b":{"c":43,"e":10,"g":43.43}},"c":[4,5,6]} │ +└─────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +```sql +SELECT json.^a.b, json.^d.e.f FROM test; +``` + +```text +┌─json.^`a`.b───────────────┬─json.^`d`.e.f────────────────────┠+│ {"c":42,"g":42.42} │ {"g":"Hello, World","h":[1,2,3]} │ +│ {} │ {"h":[4,5,6]} │ +│ {"c":43,"e":10,"g":43.43} │ {} │ +└───────────────────────────┴──────────────────────────────────┘ +``` + +:::note +Reading sub-objects as subcolumns may be inefficient, as this may require almost full scan of the JSON data. +::: + +## Types inference for paths + +During JSON parsing ClickHouse tries to detect the most appropriate data type for each JSON path. It works similar to [automatic schema inference from input data](/docs/en/interfaces/schema-inference.md) and controlled by the same settings: + +- [input_format_try_infer_integers](/docs/en/interfaces/schema-inference.md#inputformattryinferintegers) +- [input_format_try_infer_dates](/docs/en/interfaces/schema-inference.md#inputformattryinferdates) +- [input_format_try_infer_datetimes](/docs/en/interfaces/schema-inference.md#inputformattryinferdatetimes) +- [schema_inference_make_columns_nullable](/docs/en/interfaces/schema-inference.md#schemainferencemakecolumnsnullable) +- [input_format_json_try_infer_numbers_from_strings](/docs/en/interfaces/schema-inference.md#inputformatjsontryinfernumbersfromstrings) +- [input_format_json_infer_incomplete_types_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsoninferincompletetypesasstrings) +- [input_format_json_read_numbers_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadnumbersasstrings) +- [input_format_json_read_bools_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadboolsasstrings) +- [input_format_json_read_bools_as_numbers](/docs/en/interfaces/schema-inference.md#inputformatjsonreadboolsasnumbers) +- [input_format_json_read_arrays_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadarraysasstrings) + +Let's see some examples: + +```sql +SELECT JSONAllPathsWithTypes('{"a" : "2020-01-01", "b" : "2020-01-01 10:00:00"}'::JSON) AS paths_with_types settings input_format_try_infer_dates=1, input_format_try_infer_datetimes=1; +``` + +```text +┌─paths_with_types─────────────────┠+│ {'a':'Date','b':'DateTime64(9)'} │ +└──────────────────────────────────┘ +``` + +```sql +SELECT JSONAllPathsWithTypes('{"a" : "2020-01-01", "b" : "2020-01-01 10:00:00"}'::JSON) AS paths_with_types settings input_format_try_infer_dates=0, input_format_try_infer_datetimes=0; +``` + +```text +┌─paths_with_types────────────┠+│ {'a':'String','b':'String'} │ +└─────────────────────────────┘ +``` + +```sql +SELECT JSONAllPathsWithTypes('{"a" : [1, 2, 3]}'::JSON) AS paths_with_types settings schema_inference_make_columns_nullable=1; +``` + +```text +┌─paths_with_types───────────────┠+│ {'a':'Array(Nullable(Int64))'} │ +└────────────────────────────────┘ +``` + +```sql +SELECT JSONAllPathsWithTypes('{"a" : [1, 2, 3]}'::JSON) AS paths_with_types settings schema_inference_make_columns_nullable=0; +``` + +```text +┌─paths_with_types─────┠+│ {'a':'Array(Int64)'} │ +└──────────────────────┘ +``` + +## Handling arrays of JSON objects + +JSON paths that contains an array of objects are parsed as type `Array(JSON)` and inserted into `Dynamic` column for this path. To read an array of objects you can extract it from `Dynamic` column as a subcolumn: + +```sql +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test VALUES +('{"a" : {"b" : [{"c" : 42, "d" : "Hello", "f" : {"g" : 42.42}}, {"c" : 43}, {"e" : [1, 2, 3], "d" : "My", "f" : {"g" : 43.43, "h" : "2020-01-01"}}]}}'), +('{"a" : {"b" : [1, 2, 3]}}'), +('{"a" : {"b" : [{"c" : 44, "f" : {"h" : "2020-01-02"}}, {"e" : [4, 5, 6], "d" : "World", "f" : {"g" : 44.44}}]}}'); +SELECT json FROM test; +``` + +```text +┌─json──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┠+│ {"a":{"b":[{"c":"42","d":"Hello","f":{"g":42.42}},{"c":"43"},{"d":"My","e":["1","2","3"],"f":{"g":43.43,"h":"2020-01-01"}}]}} │ +│ {"a":{"b":["1","2","3"]}} │ +│ {"a":{"b":[{"c":"44","f":{"h":"2020-01-02"}},{"d":"World","e":["4","5","6"],"f":{"g":44.44}}]}} │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +```sql +SELECT json.a.b, dynamicType(json.a.b) FROM test; +``` + +```text +┌─json.a.b────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─dynamicType(json.a.b)───────────────────────────────────┠+│ ['{"c":"42","d":"Hello","f":{"g":42.42}}','{"c":"43"}','{"d":"My","e":["1","2","3"],"f":{"g":43.43,"h":"2020-01-01"}}'] │ Array(JSON(max_dynamic_types=8, max_dynamic_paths=125)) │ +│ [1,2,3] │ Array(Nullable(Int64)) │ +│ ['{"c":"44","f":{"h":"2020-01-02"}}','{"d":"World","e":["4","5","6"],"f":{"g":44.44}}'] │ Array(JSON(max_dynamic_types=8, max_dynamic_paths=125)) │ +└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────┘ +``` + +As you can notice, the `max_dynamic_types/max_dynamic_paths` parameters of the nested `JSON` type were reduced compared to the default values. It's needed to avoid number of subcolumns to grow uncontrolled on nested arrays of JSON objects. + +Let's try to read subcolumns of this nested `JSON` column: + +```sql +SELECT json.a.b.:`Array(JSON)`.c, json.a.b.:`Array(JSON)`.f.g, json.a.b.:`Array(JSON)`.f.g FROM test; +``` + +```text +┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f.g─┬─json.a.b.:`Array(JSON)`.f.g─┠+│ [42,43,NULL] │ [42.42,NULL,43.43] │ [42.42,NULL,43.43] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [NULL,44.44] │ [NULL,44.44] │ +└───────────────────────────┴─────────────────────────────┴─────────────────────────────┘ +``` + +We can also read subcolumns of `Dynamic` columns: + +```sql +SELECT json.a.b.:`Array(JSON)`.f.h.:Date FROM test; +``` + +```text +┌─json.a.b.:`Array(JSON)`.f.h.:`Date`─┠+│ [NULL,NULL,'2020-01-01'] │ +│ [] │ +│ ['2020-01-02',NULL] │ +└─────────────────────────────────────┘ +``` + +We can also read sub-object subcolumns from nested `JSON` column: + +```sql +SELECT json.a.b.:`Array(JSON)`.^f FROM test +``` + +```text +┌─json.a.b.:`Array(JSON)`.^`f`────────────────────────┠+│ ['{"g":42.42}','{}','{"g":43.43,"h":"2020-01-01"}'] │ +│ [] │ +│ ['{"h":"2020-01-02"}','{"g":44.44}'] │ +└─────────────────────────────────────────────────────┘ +``` + +## Reading JSON type from the data + +All text formats (JSONEachRow, TSV, CSV, CustomSeparated, Values, etc) supports reading `JSON` type. + +Examples: + +```sql +SELECT json FROM format(JSONEachRow, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP PREFIX d.e, SKIP REGEXP \'b.*\')', ' +{"json" : {"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}}} +{"json" : {"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}}} +{"json" : {"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"}} +{"json" : {"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43}} +{"json" : {"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : {"c" : 11, "j" : [1, 2, 3]}, "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44}, "h" : "2020-02-02 10:00:00"}}} +') +``` + +```text +┌─json──────────────────────────────────────────────────────────┠+│ {"a":{"b":{"c":1}},"c":"42","d":{"i":["1","2","3"]}} │ +│ {"a":{"b":{"c":2}},"d":{"i":["4","5","6"]}} │ +│ {"a":{"b":{"c":3}},"e":"Hello, World!"} │ +│ {"a":{"b":{"c":4}},"c":"43"} │ +│ {"a":{"b":{"c":5}},"d":{"h":"2020-02-02 10:00:00.000000000"}} │ +└───────────────────────────────────────────────────────────────┘ +``` + +For text formats like CSV/TSV/etc `JSON` is parsed from a string containing JSON object + +```sql +SELECT json FROM format(TSV, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP PREFIX d.e, SKIP REGEXP \'b.*\')', +'{"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}} +{"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}} +{"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"} +{"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43} +{"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : {"c" : 11, "j" : [1, 2, 3]}, "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44}, "h" : "2020-02-02 10:00:00"}}') +``` + +```text +┌─json──────────────────────────────────────────────────────────┠+│ {"a":{"b":{"c":1}},"c":"42","d":{"i":["1","2","3"]}} │ +│ {"a":{"b":{"c":2}},"d":{"i":["4","5","6"]}} │ +│ {"a":{"b":{"c":3}},"e":"Hello, World!"} │ +│ {"a":{"b":{"c":4}},"c":"43"} │ +│ {"a":{"b":{"c":5}},"d":{"h":"2020-02-02 10:00:00.000000000"}} │ +└───────────────────────────────────────────────────────────────┘ +``` + +## Reaching the limit of dynamic paths inside JSON + +`JSON` data type can store only limited number of paths as separate subcolumns inside. By default, this limit is 1000, but you can change it in type declaration using parameter `max_dynamic_paths`. +When the limit is reached, all new paths inserted to `JSON` column will be stored in a single shared data structure. It's still possible to read such paths as subcolumns, but it will require reading the whole +shared data structure to extract the values of this path. This limit is needed to avoid the enormous number of different subcolumns that can make the table unusable. + +Let's see what happens when the limit is reached in different scenarios. + +### Reaching the limit during data parsing + +During parsing of `JSON` object from the data, when the limit is reached for current block of data, all new paths will be stored in a shared data structure. We can check it using introspection functions `JSONDynamicPaths, JSONSharedDataPaths`: + +```sql +SELECT json, JSONDynamicPaths(json), JSONSharedDataPaths(json) FROM format(JSONEachRow, 'json JSON(max_dynamic_paths=3)', ' +{"json" : {"a" : {"b" : 42}, "c" : [1, 2, 3]}} +{"json" : {"a" : {"b" : 43}, "d" : "2020-01-01"}} +{"json" : {"a" : {"b" : 44}, "c" : [4, 5, 6]}} +{"json" : {"a" : {"b" : 43}, "d" : "2020-01-02", "e" : "Hello", "f" : {"g" : 42.42}}} +{"json" : {"a" : {"b" : 43}, "c" : [7, 8, 9], "f" : {"g" : 43.43}, "h" : "World"}} +') +``` + +```text +┌─json───────────────────────────────────────────────────────────┬─JSONDynamicPaths(json)─┬─JSONSharedDataPaths(json)─┠+│ {"a":{"b":"42"},"c":["1","2","3"]} │ ['a.b','c','d'] │ [] │ +│ {"a":{"b":"43"},"d":"2020-01-01"} │ ['a.b','c','d'] │ [] │ +│ {"a":{"b":"44"},"c":["4","5","6"]} │ ['a.b','c','d'] │ [] │ +│ {"a":{"b":"43"},"d":"2020-01-02","e":"Hello","f":{"g":42.42}} │ ['a.b','c','d'] │ ['e','f.g'] │ +│ {"a":{"b":"43"},"c":["7","8","9"],"f":{"g":43.43},"h":"World"} │ ['a.b','c','d'] │ ['f.g','h'] │ +└────────────────────────────────────────────────────────────────┴────────────────────────┴───────────────────────────┘ +``` + +As we can see, after inserting paths `e` and `f.g` the limit was reached and we inserted them into shared data structure. + +### During merges of data parts in MergeTree table engines + +During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths won't be able to store all paths from source parts as subcolumns. +In this case ClickHouse chooses what paths will remain as subcolumns after merge and what types will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contains +the largest number of non-null values and move the rarest paths to the shared data structure, but it depends on the implementation. + +Let's see an example of such merge. First, let's create a table with `JSON` column, set the limit of dynamic paths to `3` and insert values with `5` different paths: + +```sql +CREATE TABLE test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree ORDER BY id; +SYSTEM STOP MERGES test; +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as a) FROM numbers(5); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as b) FROM numbers(4); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as c) FROM numbers(3); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as d) FROM numbers(2); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as e) FROM numbers(1); +``` + +Each insert will create a separate data pert with `JSON` column containing single path: +```sql +SELECT count(), JSONDynamicPaths(json) AS dynamic_paths, JSONSharedDataPaths(json) AS shared_data_paths, _part FROM test GROUP BY _part, dynamic_paths, shared_data_paths ORDER BY _part ASC +``` + +```text +┌─count()─┬─dynamic_paths─┬─shared_data_paths─┬─_part─────┠+│ 5 │ ['a'] │ [] │ all_1_1_0 │ +│ 4 │ ['b'] │ [] │ all_2_2_0 │ +│ 3 │ ['c'] │ [] │ all_3_3_0 │ +│ 2 │ ['d'] │ [] │ all_4_4_0 │ +│ 1 │ ['e'] │ [] │ all_5_5_0 │ +└─────────┴───────────────┴───────────────────┴───────────┘ + +``` + +Now, let's merge all parts into one and see what will happen: + +```sql +SYSTEM START MERGES test; +OPTIMIZE TABLE test FINAL; +SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; +``` + +```text +┌─count()─┬─dynamic_paths─┬─shared_data_paths─┬─_part─────┠+│ 1 │ ['a','b','c'] │ ['e'] │ all_1_5_2 │ +│ 2 │ ['a','b','c'] │ ['d'] │ all_1_5_2 │ +│ 12 │ ['a','b','c'] │ [] │ all_1_5_2 │ +└─────────┴───────────────┴───────────────────┴───────────┘ +``` + +As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and moved paths `e` and `d` to shared data structure. + + + +## Tips for better usage of the JSON type + +Before creating `JSON` column and loading data into it, consider the following tips: + +- Investigate your data and specify as many path hints with types as you can. It will make the storage and the reading much more efficient. +- Think about what paths you will need and what paths you will never need. Specify paths that you won't need in the SKIP section, use SKIP PREFIX and SKIP REGEXP if needed. It will improve the storage. +- Don't set `max_dynamic_paths` parameter to very high values, it can make the storage and reading less efficient. diff --git a/docs/en/sql-reference/data-types/object-json.md b/docs/en/sql-reference/data-types/object-json.md new file mode 100644 index 00000000000..36835a42db8 --- /dev/null +++ b/docs/en/sql-reference/data-types/object-json.md @@ -0,0 +1,83 @@ +--- +slug: /en/sql-reference/data-types/object-data-type +sidebar_position: 26 +sidebar_label: Object Data Type +keywords: [object, data type] +--- + +# Object Data Type (deprecated) + +**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864). + +
+ +Stores JavaScript Object Notation (JSON) documents in a single column. + +`JSON` is an alias for `Object('json')`. + +## Example + +**Example 1** + +Creating a table with a `JSON` column and inserting data into it: + +```sql +CREATE TABLE json +( + o JSON +) +ENGINE = Memory +``` + +```sql +INSERT INTO json VALUES ('{"a": 1, "b": { "c": 2, "d": [1, 2, 3] }}') +``` + +```sql +SELECT o.a, o.b.c, o.b.d[3] FROM json +``` + +```text +┌─o.a─┬─o.b.c─┬─arrayElement(o.b.d, 3)─┠+│ 1 │ 2 │ 3 │ +└─────┴───────┴────────────────────────┘ +``` + +**Example 2** + +To be able to create an ordered `MergeTree` family table, the sorting key has to be extracted into its column. For example, to insert a file of compressed HTTP access logs in JSON format: + +```sql +CREATE TABLE logs +( + timestamp DateTime, + message JSON +) +ENGINE = MergeTree +ORDER BY timestamp +``` + +```sql +INSERT INTO logs +SELECT parseDateTimeBestEffort(JSONExtractString(json, 'timestamp')), json +FROM file('access.json.gz', JSONAsString) +``` + +## Displaying JSON columns + +When displaying a `JSON` column, ClickHouse only shows the field values by default (because internally, it is represented as a tuple). You can also display the field names by setting `output_format_json_named_tuples_as_objects = 1`: + +```sql +SET output_format_json_named_tuples_as_objects = 1 + +SELECT * FROM json FORMAT JSONEachRow +``` + +```text +{"o":{"a":1,"b":{"c":2,"d":[1,2,3]}}} +``` + +## Related Content + +- [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) +- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) \ No newline at end of file diff --git a/docs/en/sql-reference/data-types/object_json.md b/docs/en/sql-reference/data-types/object_json.md new file mode 100644 index 00000000000..39e37abad82 --- /dev/null +++ b/docs/en/sql-reference/data-types/object_json.md @@ -0,0 +1,86 @@ +--- +slug: /en/sql-reference/data-types/json +sidebar_position: 26 +sidebar_label: JSON +--- + +# JSON + +:::note +This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. +::: + +Stores JavaScript Object Notation (JSON) documents in a single column. + +`JSON` is an alias for `Object('json')`. + +:::note +The JSON data type is an obsolete feature. Do not use it. +If you want to use it, set `allow_experimental_object_type = 1`. +::: + +## Example + +**Example 1** + +Creating a table with a `JSON` column and inserting data into it: + +```sql +CREATE TABLE json +( + o JSON +) +ENGINE = Memory +``` + +```sql +INSERT INTO json VALUES ('{"a": 1, "b": { "c": 2, "d": [1, 2, 3] }}') +``` + +```sql +SELECT o.a, o.b.c, o.b.d[3] FROM json +``` + +```text +┌─o.a─┬─o.b.c─┬─arrayElement(o.b.d, 3)─┠+│ 1 │ 2 │ 3 │ +└─────┴───────┴────────────────────────┘ +``` + +**Example 2** + +To be able to create an ordered `MergeTree` family table the sorting key has to be extracted into its column. For example, to insert a file of compressed HTTP access logs in JSON format: + +```sql +CREATE TABLE logs +( + timestamp DateTime, + message JSON +) +ENGINE = MergeTree +ORDER BY timestamp +``` + +```sql +INSERT INTO logs +SELECT parseDateTimeBestEffort(JSONExtractString(json, 'timestamp')), json +FROM file('access.json.gz', JSONAsString) +``` + +## Displaying JSON columns + +When displaying a `JSON` column ClickHouse only shows the field values by default (because internally, it is represented as a tuple). You can display the field names as well by setting `output_format_json_named_tuples_as_objects = 1`: + +```sql +SET output_format_json_named_tuples_as_objects = 1 + +SELECT * FROM json FORMAT JSONEachRow +``` + +```text +{"o":{"a":1,"b":{"c":2,"d":[1,2,3]}}} +``` + +## Related Content + +- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) From 4ad6597ab61fda8f62d9b78713503d1e5c5517d7 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 15:15:39 +0000 Subject: [PATCH 0265/1722] Add more comments --- src/DataTypes/DataTypeObject.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 1a66222cc98..8f580f03b57 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -75,11 +75,17 @@ public: private: SchemaFormat schema_format; + /// Set of paths with types that were specified in type declaration. std::unordered_map typed_paths; + /// Set of paths that should be skipped during data parsing. std::unordered_set paths_to_skip; + /// List of path prefixes that should be skipped during data parsing. std::vector path_prefixes_to_skip; + /// List of regular expressions that should be used to skip paths during data parsing. std::vector path_regexps_to_skip; + /// Limit on the number of paths that can be stored as subcolumn. size_t max_dynamic_paths; + /// Limit of dynamic types that should be used for Dynamic columns. size_t max_dynamic_types; }; From 83037f534e3075e9ebaa7e2457c6c96b98d44363 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 16:07:38 +0000 Subject: [PATCH 0266/1722] Fix style --- src/Columns/ColumnObject.cpp | 4 ++-- src/DataTypes/DataTypeObject.cpp | 3 +-- src/DataTypes/Serializations/SerializationJSON.cpp | 5 +---- src/DataTypes/Serializations/SerializationObject.cpp | 4 ++-- src/DataTypes/Serializations/SerializationObjectTypedPath.h | 2 +- src/DataTypes/Serializations/SerializationSubObject.cpp | 2 +- src/DataTypes/Serializations/SerializationSubObject.h | 5 +++++ src/Formats/JSONExtractTree.cpp | 4 ++-- src/Functions/FunctionsConversion.cpp | 2 +- src/Functions/JSONPaths.cpp | 2 +- 10 files changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 50b2bcf90b8..4c4fa931fbc 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -481,7 +481,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co shared_data_offsets.push_back(shared_data_paths->size()); } } - + /// Insert default values in all remaining dynamic paths. for (auto & [_, column] : dynamic_paths) { @@ -550,7 +550,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co void ColumnObject::serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const String & path, const IColumn & column, size_t n) { - /// Don't store Null values in shared data. We consider Null value equivalent to the absense + /// Don't store Null values in shared data. We consider Null value equivalent to the absence /// of this path in the row because we cannot distinguish these 2 cases for dynamic paths. if (column.isNullAt(n)) return; diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 338883c3ae4..8f3baa05925 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -31,7 +31,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNEXPECTED_AST_STRUCTURE; extern const int BAD_ARGUMENTS; } @@ -403,7 +402,7 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: { if (!arguments || arguments->children.empty()) return std::make_shared(schema_format); - + std::unordered_map typed_paths; std::unordered_set paths_to_skip; /// Collect prefixes in unordered_set to avoid duplicate prefixes diff --git a/src/DataTypes/Serializations/SerializationJSON.cpp b/src/DataTypes/Serializations/SerializationJSON.cpp index b5bf8d1421b..51b12568981 100644 --- a/src/DataTypes/Serializations/SerializationJSON.cpp +++ b/src/DataTypes/Serializations/SerializationJSON.cpp @@ -123,7 +123,7 @@ void SerializationJSON::serializeTextImpl(const IColumn & column, size_t sorted_paths.emplace_back(path); for (const auto & [path, dynamic_column] : dynamic_paths) { - /// We consider null value and absense of the path in a row as equivalent cases, because we cannot actually distinguish them. + /// We consider null value and absence of the path in a row as equivalent cases, because we cannot actually distinguish them. /// So, we don't output null values at all. if (!dynamic_column->isNullAt(row_num)) sorted_paths.emplace_back(path); @@ -403,7 +403,4 @@ template class SerializationJSON; template class SerializationJSON; #endif - - - } diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index d4483703c6b..8e6f6f105ea 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -285,7 +285,7 @@ void SerializationObject::deserializeBinaryBulkStatePrefix( shared_data_serialization->deserializeBinaryBulkStatePrefix(settings, object_state->shared_data_state, cache); settings.path.pop_back(); settings.path.pop_back(); - + state = std::move(object_state); } @@ -605,7 +605,7 @@ void SerializationObject::deserializeBinary(IColumn & col, ReadBuffer & istr, co typed_it->second->deserializeBinary(*typed_column, istr, settings); } } - /// Check if we have this path in dynamc paths. + /// Check if we have this path in dynamic paths. else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) { /// Check if we already had this path. diff --git a/src/DataTypes/Serializations/SerializationObjectTypedPath.h b/src/DataTypes/Serializations/SerializationObjectTypedPath.h index a893e07f798..997e14bd145 100644 --- a/src/DataTypes/Serializations/SerializationObjectTypedPath.h +++ b/src/DataTypes/Serializations/SerializationObjectTypedPath.h @@ -49,7 +49,7 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; - + private: String path; }; diff --git a/src/DataTypes/Serializations/SerializationSubObject.cpp b/src/DataTypes/Serializations/SerializationSubObject.cpp index 62ebd4669e1..9084d46f9b2 100644 --- a/src/DataTypes/Serializations/SerializationSubObject.cpp +++ b/src/DataTypes/Serializations/SerializationSubObject.cpp @@ -39,7 +39,7 @@ void SerializationSubObject::enumerateStreams( settings.path.push_back(Substream::ObjectStructure); callback(settings.path); settings.path.pop_back(); - + const auto * column_object = data.column ? &assert_cast(*data.column) : nullptr; const auto * type_object = data.type ? &assert_cast(*data.type) : nullptr; const auto * deserialize_state = data.deserialize_state ? checkAndGetState(data.deserialize_state) : nullptr; diff --git a/src/DataTypes/Serializations/SerializationSubObject.h b/src/DataTypes/Serializations/SerializationSubObject.h index 8d037720f1f..8a5582536e7 100644 --- a/src/DataTypes/Serializations/SerializationSubObject.h +++ b/src/DataTypes/Serializations/SerializationSubObject.h @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + /// Serialization of a sub-object Object subcolumns. /// For example, if we have type JSON and data {"a" : {"b" : {"c" : 42, "d" : "Hello"}}, "c" : [1, 2, 3], "d" : 42} /// this class will be responsible for reading sub-object a.b and will read JSON column with data {"c" : 43, "d" : "Hello"}. diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index db984d67819..e68e98a6a06 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1636,7 +1636,7 @@ private: auto & typed_paths = column_object.getTypedPaths(); auto & dynamic_paths = column_object.getDynamicPaths(); - /// Check if we have this path in typed pahts. + /// Check if we have this path in typed paths. if (auto typed_it = typed_paths.find(current_path); typed_it != typed_paths.end()) { /// Check if we already had this path. @@ -1683,7 +1683,7 @@ private: } /// Otherwise this path should go to the shared data. /// Don't insert null values into shared data. - /// We consider null equivalent to the absense of this path. + /// We consider null equivalent to the absence of this path. else if (!element.isNull()) { auto tmp_dynamic_column = ColumnDynamic::create(); diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 3b0179dff2b..88cd2e382f2 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4004,7 +4004,7 @@ private: throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName()); } - + WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_object) const { if (checkAndGetDataType(from_type.get())) diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index bbdf1385cf2..b7a161901f0 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -300,7 +300,7 @@ private: offsets.push_back(paths_column->size()); } - + return ColumnMap::create(ColumnPtr(std::move(paths_column)), ColumnPtr(std::move(types_column)), ColumnPtr(std::move(offsets_column))); } From 8394c8ee67d41009dbc72348b64faae98cdfa165 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 16:43:08 +0000 Subject: [PATCH 0267/1722] Fix build --- src/Columns/ColumnObject.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 4c4fa931fbc..dce6b006ff2 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1132,10 +1132,15 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const ColumnString & shared_data_paths, size_t start, size_t end) { /// Simple random access iterator over values in ColumnString in specified range. - class Iterator : public std::iterator + class Iterator { public: using difference_type = size_t; + using value_type = StringRef; + using iterator_category = std::random_access_iterator_tag; + using pointer = StringRef*; + using reference = StringRef&; + Iterator() = delete; Iterator(const ColumnString * data_, size_t index_) : data(data_), index(index_) {} Iterator(const Iterator & rhs) : data(rhs.data), index(rhs.index) {} From b77cfc2724f551cc359e7fbccf3a074c7510cdd0 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 18:47:24 +0000 Subject: [PATCH 0268/1722] Fix compound identifier parser --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 1 - src/Parsers/ExpressionElementParsers.cpp | 24 ++++++++++++++++++++---- 3 files changed, 21 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4bbe034cdc0..db2c8572266 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1111,7 +1111,7 @@ class IColumn; M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ - M(Bool, input_format_json_try_infer_numbers_from_strings, true, "Try to infer numbers from string fields while schema inference", 0) \ + M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ M(Bool, input_format_json_validate_types_from_metadata, true, "For JSON/JSONCompact/JSONColumnsWithMetadata input formats this controls whether format parser should check if data types from input metadata match data types of the corresponding columns from the table", 0) \ M(Bool, input_format_json_read_numbers_as_strings, true, "Allow to parse numbers as strings in JSON input formats", 0) \ M(Bool, input_format_json_read_objects_as_strings, true, "Allow to parse JSON objects as strings in JSON input formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b5a48c3bf7e..8e42c3a3ea8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,7 +81,6 @@ static std::initializer_listparse(pos, element, expected)) - return false; + { + if (is_first) + return false; + pos = begin; + break; + } + + is_first = false; + if (last_special_delimiter != SpecialDelimiter::NONE) parts.push_back(static_cast(last_special_delimiter) + backQuote(getIdentifierName(element))); else @@ -309,7 +318,8 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (parts.back().empty()) params.push_back(element->as()->getParam()); - parsed_delimiter = false; + begin = pos; + bool parsed_delimiter = false; for (const auto & [parser, special_delimiter] : delimiter_parsers) { if (parser->check(pos, expected)) @@ -319,6 +329,12 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex break; } } + + if (!parsed_delimiter) + { + pos = begin; + break; + } } ParserKeyword s_uuid(Keyword::UUID); From 11d56a08cd82aae5b37f30fdbf5d218ef8446a63 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 Jul 2024 20:16:47 +0000 Subject: [PATCH 0269/1722] Fix clickhouse-format --- docs/en/sql-reference/data-types/json.md | 2 +- programs/format/Format.cpp | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index d35a3f90ef3..23bb57e5a6c 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -4,7 +4,7 @@ sidebar_position: 63 sidebar_label: JSON --- -# JSON` +# JSON Stores JavaScript Object Notation (JSON) documents in a single column. diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 1b91e7ceaf3..a434c9171e9 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -175,6 +175,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv) hash_func.update(options["seed"].as()); } + SharedContextHolder shared_context = Context::createShared(); + auto context = Context::createGlobal(shared_context.get()); + auto context_const = WithContext(context).getContext(); + context->makeGlobalContext(); + registerInterpreters(); registerFunctions(); registerAggregateFunctions(); From 201f813516e1283a4d0528bf71753e8291526ccf Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 13 Jul 2024 02:37:09 +0000 Subject: [PATCH 0270/1722] add prep for rebuild --- .../MergeTree/MergeMutateSelectedEntry.h | 5 +- .../MergeTree/MergeTreeMutationEntry.cpp | 4 +- .../MergeTree/MergeTreeMutationEntry.h | 6 ++- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 + src/Storages/MergeTree/MutateTask.cpp | 52 +++++++++++++------ src/Storages/StorageMergeTree.cpp | 11 +++- 6 files changed, 60 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index c420cbca12b..116c7d26552 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -40,12 +40,15 @@ struct MergeMutateSelectedEntry CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; + Field lightweight_delete_projection_mode; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, - MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR) + MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR, + const Field & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) , txn(txn_) + , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) {} }; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 4dbccb91620..06f4875d120 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -48,7 +48,8 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) } MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings) + const TransactionID & tid_, const WriteSettings & settings, + const Field & lightweight_delete_projection_mode_) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -56,6 +57,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) + , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) { try { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 04297f2852a..cbc7e2d4274 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -36,9 +36,13 @@ struct MergeTreeMutationEntry /// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction. CSN csn = Tx::UnknownCSN; + /// From query context. + Field lightweight_delete_projection_mode; + /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings); + const TransactionID & tid_, const WriteSettings & settings, + const Field & lightweight_delete_projection_mode_); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 20f387137e7..1bf337973ff 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,6 +140,8 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); + if (merge_mutate_entry) + context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 2adcb49d6a3..ed603abd9c3 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -656,7 +656,9 @@ static NameSet collectFilesToSkip( const std::set & indices_to_recalc, const String & mrk_extension, const std::set & projections_to_recalc, - const std::set & stats_to_recalc) + const std::set & stats_to_recalc, + const StorageMetadataPtr & metadata_snapshot, + bool lightweight_delete_mode) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -680,8 +682,16 @@ static NameSet collectFilesToSkip( } } - for (const auto & projection : projections_to_recalc) - files_to_skip.insert(projection->getDirectoryName()); + if (lightweight_delete_mode) + { + for (const auto & projection : metadata_snapshot->getProjections()) + files_to_skip.insert(projection.getDirectoryName()); + } + else + { + for (const auto & projection : projections_to_recalc) + files_to_skip.insert(projection->getDirectoryName()); + } for (const auto & stat : stats_to_recalc) files_to_skip.insert(stat->getFileName() + STATS_FILE_SUFFIX); @@ -1042,8 +1052,6 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; - - bool lightweight_delete_mode; }; using MutationContextPtr = std::shared_ptr; @@ -1573,7 +1581,7 @@ private: } else { - if (!ctx->lightweight_delete_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!ctx->updated_header.has(RowExistsColumn::name) && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } @@ -1843,8 +1851,7 @@ private: hardlinked_files.insert(it->name()); } } - /// Ignore projection tmp merge dir, and under lightweight delete mode ignore projection files. - else if (!endsWith(it->name(), ".tmp_proj") && !ctx->lightweight_delete_mode) + else if (!endsWith(it->name(), ".tmp_proj")) // ignore projection tmp merge dir { // it's a projection part directory ctx->new_data_part->getDataPartStorage().createProjection(destination); @@ -2193,6 +2200,7 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); + context_for_reading->setSetting("lightweight_mutation_projection_mode", Field(ctx->context->getSettingsRef().lightweight_mutation_projection_mode)); MutationHelpers::splitAndModifyMutationCommands( ctx->source_part, ctx->metadata_snapshot, @@ -2217,6 +2225,15 @@ bool MutateTask::prepare() ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); + + // ctx->updated_header.has(RowExistsColumn::name); + // for (const auto & projection : ctx->metadata_snapshot->getProjections()) + // { + // if (!ctx->source_part->hasProjection(projection.name)) + // continue; + + // ctx->materialized_projections.insert(projection.name); + // } } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); @@ -2258,8 +2275,8 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - ctx->lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); - if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && ctx->lightweight_delete_mode) + bool lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && lightweight_delete_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, /// Reset existing_rows_count of new data part to 0 and it will be updated while writing _row_exists column @@ -2296,10 +2313,13 @@ bool MutateTask::prepare() ctx->context, ctx->materialized_indices); - ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( - ctx->source_part, - ctx->metadata_snapshot, - ctx->materialized_projections); + if (!lightweight_delete_mode) + { + ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( + ctx->source_part, + ctx->metadata_snapshot, + ctx->materialized_projections); + } ctx->stats_to_recalc = MutationHelpers::getStatisticsToRecalculate(ctx->metadata_snapshot, ctx->materialized_statistics); @@ -2310,7 +2330,9 @@ bool MutateTask::prepare() ctx->indices_to_recalc, ctx->mrk_extension, ctx->projections_to_recalc, - ctx->stats_to_recalc); + ctx->stats_to_recalc, + ctx->metadata_snapshot, + lightweight_delete_mode); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 611289ffd78..063e3b7f064 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -517,7 +517,8 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), + Field(query_context->getSettingsRef().lightweight_mutation_projection_mode)); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -1282,12 +1283,18 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; + + /// Trying to grab it from query context. + Field lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; + for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; + lightweight_delete_projection_mode = it->second.lightweight_delete_projection_mode; + size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -1364,7 +1371,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( future_part->part_format = part->getFormat(); tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}, false), *this, metadata_snapshot, true); - return std::make_shared(future_part, std::move(tagger), commands, txn); + return std::make_shared(future_part, std::move(tagger), commands, txn, lightweight_delete_projection_mode); } } From 6f03ff20d88398c2c097d031b00c1407c6624ca3 Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 13 Jul 2024 14:06:24 +0000 Subject: [PATCH 0271/1722] Add local storage --- .../ObjectStorage/Local/Configuration.cpp | 75 +++++++++ .../ObjectStorage/Local/Configuration.h | 55 +++++++ .../test_local_storage/__init__.py | 0 tests/integration/test_local_storage/test.py | 153 ++++++++++++++++++ 4 files changed, 283 insertions(+) create mode 100644 src/Storages/ObjectStorage/Local/Configuration.cpp create mode 100644 src/Storages/ObjectStorage/Local/Configuration.h create mode 100644 tests/integration/test_local_storage/__init__.py create mode 100644 tests/integration/test_local_storage/test.py diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp new file mode 100644 index 00000000000..5b66dd2470b --- /dev/null +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -0,0 +1,75 @@ +#include + +#include +#include +#include +#include "Common/NamedCollections/NamedCollections.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int LOGICAL_ERROR; +} + +void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) +{ + path = collection.get("path"); + format = collection.getOrDefault("format", "auto"); + compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + structure = collection.getOrDefault("structure", "auto"); +} + + +void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + const size_t max_args_num = with_structure ? 4 : 3; + if (args.empty() || args.size() > max_args_num) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected not more than {} arguments", max_args_num); + } + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + path = checkAndGetLiteralArgument(args[0], "path"); + + if (args.size() > 1) + { + format = checkAndGetLiteralArgument(args[1], "format_name"); + } + + if (with_structure) + { + if (args.size() > 2) + { + structure = checkAndGetLiteralArgument(args[2], "structure"); + } + if (args.size() > 3) + { + compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + } + } + else if (args.size() > 2) + { + compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + } +} + +StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.engine_file_truncate_on_insert, + .create_new_file_on_insert = false, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.engine_file_skip_empty_files, + .list_object_keys_size = 0, + .throw_on_zero_files_match = false, + .ignore_non_existent_file = false}; +} + +} diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h new file mode 100644 index 00000000000..a305fc43825 --- /dev/null +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include "Disks/ObjectStorages/Local/LocalObjectStorage.h" +#include "config.h" + +#if USE_AWS_S3 +# include +# include + +namespace DB +{ + +class StorageLocalConfiguration : public StorageObjectStorage::Configuration +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto type_name = "local"; + + StorageLocalConfiguration() = default; + StorageLocalConfiguration(const StorageLocalConfiguration & other) = default; + + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return "Local"; } + + Path getPath() const override { return path; } + void setPath(const Path & path_) override { path = path_; } + + const Paths & getPaths() const override { return paths; } + void setPaths(const Paths & paths_) override { paths = paths_; } + + String getNamespace() const override { return ""; } + String getDataSourceDescription() const override { return ""; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + + void check(ContextPtr) const override { } + void validateNamespace(const String &) const override { } + ConfigurationPtr clone() override { return std::make_shared(*this); } + bool isStaticConfiguration() const override { return true; } + + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared(path); } + + void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } + +private: + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + Path path; + Paths paths; +}; + +} + +#endif diff --git a/tests/integration/test_local_storage/__init__.py b/tests/integration/test_local_storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py new file mode 100644 index 00000000000..96d116ec6a2 --- /dev/null +++ b/tests/integration/test_local_storage/test.py @@ -0,0 +1,153 @@ +import logging +import os +import random +import string + +import pytest + +from helpers.cluster import ClickHouseCluster +from azure.storage.blob import BlobServiceClient +from test_storage_azure_blob_storage.test import azure_query + +NODE_NAME = "node" + + +def generate_cluster_def(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/disk_storage_conf.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write( + f""" + + + + object_storage + azure_blob_storage + plain_rewritable + http://azurite1:{port}/devstoreaccount1 + cont + true + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + 100000 + 100000 + 10 + 10 + + + + + +
+ blob_storage_disk +
+
+
+
+
+
+""" + ) + return path + + +insert_values = [ + "(0,'data'),(1,'data')", + ",".join( + f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" + for i in range(10) + ), +] + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_cluster_def(port) + cluster.add_instance( + NODE_NAME, + main_configs=[ + path, + ], + with_azurite=True, + stay_alive=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_insert_select(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + azure_query( + node, + """ + CREATE TABLE test_{} ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='blob_storage_policy' + """.format( + index + ), + ) + + azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_restart_server(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + node.restart_clickhouse() + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_drop_table(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) + + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + assert len(list(container_client.list_blobs())) == 0 From f54a4b073a3fd1c0d31b26c3085eedabb667687a Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 13 Jul 2024 14:07:36 +0000 Subject: [PATCH 0272/1722] Add local table function --- .../registerStorageObjectStorage.cpp | 24 +++++++++++++++++-- .../TableFunctionObjectStorage.h | 22 +++++++++++++---- 2 files changed, 40 insertions(+), 6 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index bf595b2f5d4..04c3417948c 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,9 +1,10 @@ +#include #include -#include #include +#include +#include #include #include -#include namespace DB { @@ -138,8 +139,27 @@ void registerStorageHDFS(StorageFactory & factory) } #endif +void registerStorageLocal(StorageFactory & factory) +{ + factory.registerStorage( + "Local", + [=](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::FILE, + }); +} + + void registerStorageObjectStorage(StorageFactory & factory) { + registerStorageLocal(factory); #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 86b8f0d5e14..3468e5c5007 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include "config.h" -#include -#include #include -#include +#include #include +#include +#include +#include "config.h" namespace DB { @@ -14,6 +14,7 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; +class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -90,6 +91,17 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; +struct LocalDefinition +{ + static constexpr auto name = "local"; + static constexpr auto storage_type_name = "Local"; + static constexpr auto signature = " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif + +using TableFunctionLocal = TableFunctionObjectStorage; } From 0fc14520c821f22b493d32657fede6be10832d60 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 13 Jul 2024 23:06:37 +0000 Subject: [PATCH 0273/1722] add server termination on exit --- programs/server/fuzzers/tcp_protocol_fuzzer.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/programs/server/fuzzers/tcp_protocol_fuzzer.cpp b/programs/server/fuzzers/tcp_protocol_fuzzer.cpp index 950ea09669a..7cebdc2ad65 100644 --- a/programs/server/fuzzers/tcp_protocol_fuzzer.cpp +++ b/programs/server/fuzzers/tcp_protocol_fuzzer.cpp @@ -10,6 +10,7 @@ #include #include +#include #include @@ -25,6 +26,12 @@ static int64_t port = 9000; using namespace std::chrono_literals; +void on_exit() +{ + BaseDaemon::terminate(); + main_app.wait(); +} + extern "C" int LLVMFuzzerInitialize(int * argc, char ***argv) { @@ -60,6 +67,8 @@ int LLVMFuzzerInitialize(int * argc, char ***argv) exit(-1); } + atexit(on_exit); + return 0; } From 3ccc2aed4c76eba20e0fc88768412bbfacafbb95 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 13 Jul 2024 23:44:13 +0000 Subject: [PATCH 0274/1722] add fuzzer_arguments to fuzzer runner --- docker/test/libfuzzer/run_libfuzzer.py | 7 +++++++ tests/fuzz/tcp_protocol_fuzzer.options | 4 ++++ 2 files changed, 11 insertions(+) create mode 100644 tests/fuzz/tcp_protocol_fuzzer.options diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py index 5ed019490d5..cdd09dfa3be 100755 --- a/docker/test/libfuzzer/run_libfuzzer.py +++ b/docker/test/libfuzzer/run_libfuzzer.py @@ -20,6 +20,7 @@ def run_fuzzer(fuzzer: str): options_file = f"{fuzzer}.options" custom_libfuzzer_options = "" + fuzzer_arguments = "" with Path(options_file) as path: if path.exists() and path.is_file(): @@ -47,6 +48,12 @@ def run_fuzzer(fuzzer: str): for key, value in parser["libfuzzer"].items() ) + if parser.has_section("fuzzer_arguments"): + fuzzer_arguments = " ".join( + ("%s" % key) if value == "" else ("%s=%s" % (key, value)) + for key, value in parser["fuzzer_arguments"].items() + ) + cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" if custom_libfuzzer_options: cmd_line += f" {custom_libfuzzer_options}" diff --git a/tests/fuzz/tcp_protocol_fuzzer.options b/tests/fuzz/tcp_protocol_fuzzer.options new file mode 100644 index 00000000000..4885669d91d --- /dev/null +++ b/tests/fuzz/tcp_protocol_fuzzer.options @@ -0,0 +1,4 @@ +[fuzzer_arguments] +--log-file=tcp_protocol_fuzzer.log +--= +--logging.terminal=0 From d4116aeaeaeec3b17cd813d686a815476a794bed Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 01:31:40 +0000 Subject: [PATCH 0275/1722] fix --- src/Core/SettingsEnums.h | 2 +- src/Storages/MergeTree/MergeMutateSelectedEntry.h | 4 ++-- src/Storages/MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/MergeTreeMutationEntry.h | 4 ++-- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 3 +-- src/Storages/StorageMergeTree.cpp | 4 ++-- 6 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3611dfa72be..67fbce31be8 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -342,7 +342,7 @@ DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType) enum class LightweightMutationProjectionMode : uint8_t { THROW, - DROP + DROP, }; DECLARE_SETTING_ENUM(LightweightMutationProjectionMode) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index 116c7d26552..bf2d1a7f677 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -40,10 +40,10 @@ struct MergeMutateSelectedEntry CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; - Field lightweight_delete_projection_mode; + LightweightMutationProjectionMode lightweight_delete_projection_mode; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR, - const Field & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) + const LightweightMutationProjectionMode & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 06f4875d120..d1bd8efa7a5 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -49,7 +49,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings, - const Field & lightweight_delete_projection_mode_) + const LightweightMutationProjectionMode & lightweight_delete_projection_mode_) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index cbc7e2d4274..3aca744aa15 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -37,12 +37,12 @@ struct MergeTreeMutationEntry CSN csn = Tx::UnknownCSN; /// From query context. - Field lightweight_delete_projection_mode; + LightweightMutationProjectionMode lightweight_delete_projection_mode; /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings, - const Field & lightweight_delete_projection_mode_); + const LightweightMutationProjectionMode & lightweight_delete_projection_mode_); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 1bf337973ff..666dbe7e61e 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,8 +140,7 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); - if (merge_mutate_entry) - context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); + context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 063e3b7f064..7f210779916 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -518,7 +518,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context std::lock_guard lock(currently_processing_in_background_mutex); MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), - Field(query_context->getSettingsRef().lightweight_mutation_projection_mode)); + query_context->getSettingsRef().lightweight_mutation_projection_mode); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -1285,7 +1285,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto last_mutation_to_apply = mutations_end_it; /// Trying to grab it from query context. - Field lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; + LightweightMutationProjectionMode lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { From 3c09d585cde8068e1f57a1b2adfcdf8b126a8574 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 02:14:58 +0000 Subject: [PATCH 0276/1722] fix --- src/Storages/MergeTree/MergeMutateSelectedEntry.h | 1 + src/Storages/MergeTree/MergeTreeMutationEntry.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index bf2d1a7f677..f75d10d9ecb 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 3aca744aa15..dbb17654ddd 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB From d2eded16aaa62fe66e93bac171c25a1f031b3881 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 15 Jul 2024 10:53:22 +0000 Subject: [PATCH 0277/1722] Debug commit --- src/CMakeLists.txt | 1 + .../Local/LocalObjectStorage.cpp | 12 ++-- src/IO/ReadHelpers.cpp | 10 ++- .../Executors/PullingPipelineExecutor.cpp | 13 ++++ .../ObjectStorage/Local/Configuration.cpp | 2 + .../ObjectStorage/Local/Configuration.h | 20 +++--- .../ObjectStorage/StorageObjectStorage.cpp | 12 ++++ .../StorageObjectStorageSource.cpp | 62 ++++++++++++++++--- .../TableFunctionObjectStorage.cpp | 15 +++-- 9 files changed, 116 insertions(+), 31 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d985595154c..dc32687e058 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -110,6 +110,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage) add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) +add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index a247d86ddce..6c21de96371 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,15 +1,17 @@ #include -#include -#include -#include +#include +#include +#include +#include #include #include -#include #include #include +#include +#include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..ddc2e912fb3 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -80,6 +80,7 @@ UUID parseUUID(std::span src) return uuid; } + void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; @@ -88,12 +89,15 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) if (buf.eof()) out << " at end of stream."; else - out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); + out << " before: " << quote << String(buf.position(), std::min(static_cast(1000), buf.buffer().end() - buf.position())); - throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); + throw Exception( + ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, + "Cannot parse input: expected {} {}", + out.str(), + buf.buffer().end() - buf.position()); } - bool checkString(const char * s, ReadBuffer & buf) { for (; *s; ++s) diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index cbf73c5cb07..a4db27d3f37 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -42,19 +42,32 @@ const Block & PullingPipelineExecutor::getHeader() const bool PullingPipelineExecutor::pull(Chunk & chunk) { + LOG_DEBUG(&Poco::Logger::get("Pulling 1"), "Pulling 1"); + if (!executor) { executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } + LOG_DEBUG(&Poco::Logger::get("Pulling 2"), "Pulling 2"); + if (!executor->checkTimeLimitSoft()) return false; + LOG_DEBUG(&Poco::Logger::get("Pulling 3"), "Pulling 3"); + + if (!executor->executeStep(&has_data_flag)) return false; + LOG_DEBUG(&Poco::Logger::get("Pulling 4"), "Pulling 4"); + + chunk = pulling_format->getChunk(); + + LOG_DEBUG(&Poco::Logger::get("Pulling 5"), "Pulling 5"); + return true; } diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 5b66dd2470b..3abdb4d530b 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -20,6 +20,7 @@ void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & coll format = collection.getOrDefault("format", "auto"); compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); + paths = {path}; } @@ -56,6 +57,7 @@ void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool wi { compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); } + paths = {path}; } StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index a305fc43825..11165fcd574 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -2,11 +2,13 @@ #include #include "Disks/ObjectStorages/Local/LocalObjectStorage.h" -#include "config.h" -#if USE_AWS_S3 -# include -# include +#include + +#include + + +namespace fs = std::filesystem; namespace DB { @@ -34,12 +36,12 @@ public: String getDataSourceDescription() const override { return ""; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; - void check(ContextPtr) const override { } - void validateNamespace(const String &) const override { } ConfigurationPtr clone() override { return std::make_shared(*this); } - bool isStaticConfiguration() const override { return true; } - ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared(path); } + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override + { + return std::make_shared(fs::path{path}.parent_path()); + } void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } @@ -51,5 +53,3 @@ private: }; } - -#endif diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 683473006e3..8c5b4f71d4d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -124,6 +124,12 @@ public: , num_streams(num_streams_) , distributed_processing(distributed_processing_) { + LOG_DEBUG(&Poco::Logger::get("Read step created"), "Read step created"); + LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); + for (auto && key : configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); + } } std::string getName() const override { return name; } @@ -422,6 +428,12 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c DEFAULT_SCHEMA_CACHE_ELEMENTS)); return schema_cache; } + else if (storage_type_name == "local") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_local", DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index a9a7e062076..54ddaec0140 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -1,17 +1,18 @@ #include "StorageObjectStorageSource.h" -#include #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include "Common/logger_useful.h" #include namespace fs = std::filesystem; @@ -69,6 +70,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { + LOG_DEBUG(&Poco::Logger::get("Source created"), "Source created"); } StorageObjectStorageSource::~StorageObjectStorageSource() @@ -131,6 +133,7 @@ std::shared_ptr StorageObjectStorageSourc { ConfigurationPtr copy_configuration = configuration->clone(); auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (filter_dag) { auto keys = configuration->getPaths(); @@ -142,6 +145,19 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } + LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); + for (auto && key : configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); + } + + LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Keys size: {}", copy_configuration->getPaths().size()); + for (auto && key : copy_configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Current key: {}", key); + } + + iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); @@ -170,8 +186,11 @@ Chunk StorageObjectStorageSource::generate() { lazyInitialize(); + while (true) { + LOG_DEBUG(&Poco::Logger::get("Generating"), "Generating reader: {}", !(!reader)); + if (isCancelled() || !reader) { if (reader) @@ -179,10 +198,15 @@ Chunk StorageObjectStorageSource::generate() break; } + LOG_DEBUG(&Poco::Logger::get("Generating 2"), "Generating 2"); + Chunk chunk; if (reader->pull(chunk)) { + LOG_DEBUG(&Poco::Logger::get("Generating 3"), "Generating 3"); + UInt64 num_rows = chunk.getNumRows(); + LOG_DEBUG(&Poco::Logger::get("Creating_chunk"), "Chunk size: {}", num_rows); total_rows_in_file += num_rows; size_t chunk_size = 0; @@ -230,6 +254,9 @@ Chunk StorageObjectStorageSource::generate() return chunk; } + LOG_DEBUG(&Poco::Logger::get("Generating 4"), "Generating 4"); + + if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); @@ -300,6 +327,8 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); + LOG_DEBUG(&Poco::Logger::get("Unreached point 1"), ""); + QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; @@ -324,11 +353,17 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); }; + LOG_DEBUG(&Poco::Logger::get("Unreached point 2"), ""); + + std::optional num_rows_from_cache = need_only_count && context_->getSettingsRef().use_cache_for_count_from_files ? try_get_num_rows_from_cache() : std::nullopt; + LOG_DEBUG(&Poco::Logger::get("Unreached point 3"), ""); + + if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -341,6 +376,8 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { + LOG_DEBUG(&Poco::Logger::get("Unreached point 4"), ""); + CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { @@ -625,11 +662,18 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( , keys(configuration->getPaths()) , ignore_non_existent_files(ignore_non_existent_files_) { + LOG_DEBUG(&Poco::Logger::get("Keys size"), "Keys size: {}", keys.size()); + for (auto && key : keys) + { + LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); + } if (read_keys_) { /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? + for (auto && key : keys) { + LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..d156afb81a1 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -12,12 +12,13 @@ #include -#include #include -#include -#include -#include #include +#include +#include +#include +#include +#include namespace DB @@ -203,6 +204,11 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif + factory.registerFunction>( + {.documentation + = {.description = R"(The table function can be used to read the data stored locally.)", + .examples{{"local", "SELECT * FROM local(file_path, [format, compression, structure])", ""}}}, + .allow_readonly = false}); } #if USE_AZURE_BLOB_STORAGE @@ -223,4 +229,5 @@ template class TableFunctionObjectStorage; #endif +template class TableFunctionObjectStorage; } From 36fb1cc3e79c9570ba43f81e4e47041100a63d0d Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 13:15:14 +0000 Subject: [PATCH 0278/1722] temporarily disable the setting in taskcontext --- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 666dbe7e61e..19aa63d90a2 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,7 +140,7 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); - context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); + // context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } From 43d86f9066ec75d3747e85b297a155690a974e45 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 Jul 2024 14:46:08 +0000 Subject: [PATCH 0279/1722] Unify SKIP and SKIP PREFIX hints, improve JSON parsing --- src/Columns/ColumnObject.cpp | 1 + src/DataTypes/DataTypeObject.cpp | 42 ++--- src/DataTypes/DataTypeObject.h | 4 - src/DataTypes/DataTypesBinaryEncoding.cpp | 16 -- src/DataTypes/DataTypesBinaryEncoding.h | 106 ++++++------- .../Serializations/SerializationDynamic.cpp | 2 + .../Serializations/SerializationJSON.cpp | 3 +- .../Serializations/SerializationJSON.h | 1 - .../Serializations/SerializationObject.cpp | 12 +- .../Serializations/SerializationObject.h | 3 +- src/Formats/JSONExtractTree.cpp | 108 +++++++++---- src/Formats/SchemaInferenceUtils.cpp | 145 ++++++++++++------ src/Parsers/ASTObjectTypeArgument.cpp | 11 -- src/Parsers/ASTObjectTypeArgument.h | 2 - src/Parsers/ParserDataType.cpp | 15 +- src/Storages/VirtualColumnUtils.cpp | 1 + 16 files changed, 265 insertions(+), 207 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index dce6b006ff2..bd6f759ee30 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -424,6 +424,7 @@ void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t len void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) #endif { + /// TODO: try to parallelize doInsertRangeFrom over typed/dynamic paths. const auto & src_object_column = assert_cast(src); /// First, insert typed paths, they must be the same for both columns. diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 8f3baa05925..751a19beca5 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -39,22 +39,29 @@ DataTypeObject::DataTypeObject( const SchemaFormat & schema_format_, const std::unordered_map & typed_paths_, const std::unordered_set & paths_to_skip_, - const std::vector & path_prefixes_to_skip_, const std::vector & path_regexps_to_skip_, size_t max_dynamic_paths_, size_t max_dynamic_types_) : schema_format(schema_format_) , typed_paths(typed_paths_) , paths_to_skip(paths_to_skip_) - , path_prefixes_to_skip(path_prefixes_to_skip_) , path_regexps_to_skip(path_regexps_to_skip_) , max_dynamic_paths(max_dynamic_paths_) , max_dynamic_types(max_dynamic_types_) { - for (const auto & path : paths_to_skip) + for (const auto & [typed_path, type] : typed_paths) { - if (typed_paths.contains(path)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified both with the data type ('{}') and in the SKIP section", path, typed_paths[path]->getName()); + for (const auto & path_to_skip : paths_to_skip) + { + if (typed_path.starts_with(path_to_skip)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified with the data type ('{}') and matches the SKIP path prefix '{}'", typed_path, type->getName(), path_to_skip); + } + + for (const auto & path_regext_to_skip : paths_to_skip) + { + if (re2::RE2::FullMatch(typed_path, re2::RE2(path_regext_to_skip))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified with the data type ('{}') and matches the SKIP REGEXP '{}'", typed_path, type->getName(), path_regext_to_skip); + } } } @@ -81,7 +88,7 @@ bool DataTypeObject::equals(const IDataType & rhs) const return false; } - return schema_format == object->schema_format && paths_to_skip == object->paths_to_skip && path_prefixes_to_skip == object->path_prefixes_to_skip && path_regexps_to_skip == object->path_regexps_to_skip + return schema_format == object->schema_format && paths_to_skip == object->paths_to_skip && path_regexps_to_skip == object->path_regexps_to_skip && max_dynamic_types == object->max_dynamic_types && max_dynamic_paths == object->max_dynamic_paths; } @@ -102,7 +109,6 @@ SerializationPtr DataTypeObject::doGetDefaultSerialization() const return std::make_shared>( std::move(typed_path_serializations), paths_to_skip, - path_prefixes_to_skip, path_regexps_to_skip, buildJSONExtractTree(getPtr(), "JSON serialization")); #elif USE_RAPIDJSON @@ -165,12 +171,6 @@ String DataTypeObject::doGetName() const out << "SKIP " << skip_path; } - for (const auto & skip_prefix : path_prefixes_to_skip) - { - write_separator(); - out << "SKIP PREFIX " << skip_prefix; - } - for (const auto & skip_regexp : path_regexps_to_skip) { write_separator(); @@ -275,7 +275,7 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu std::unique_ptr res = std::make_unique(std::make_shared(prefix, typed_paths_serializations)); /// Keep all current constrains like limits and skip paths/prefixes/regexps. - res->type = std::make_shared(schema_format, typed_sub_paths, paths_to_skip, path_prefixes_to_skip, path_prefixes_to_skip, max_dynamic_paths, max_dynamic_types); + res->type = std::make_shared(schema_format, typed_sub_paths, paths_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); /// If column was provided, we should create a column for the requested subcolumn. if (data.column) { @@ -405,8 +405,6 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: std::unordered_map typed_paths; std::unordered_set paths_to_skip; - /// Collect prefixes in unordered_set to avoid duplicate prefixes - std::unordered_set path_prefixes_to_skip_set; std::vector path_regexps_to_skip; size_t max_dynamic_types = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES; @@ -458,14 +456,6 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: paths_to_skip.insert(identifier->name()); } - else if (object_type_argument->skip_path_prefix) - { - const auto * identifier = object_type_argument->skip_path_prefix->as(); - if (!identifier) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST in SKIP section of {} type arguments: {}. Expected identifier with path name", magic_enum::enum_name(schema_format), object_type_argument->skip_path->formatForErrorMessage()); - - path_prefixes_to_skip_set.insert(identifier->name()); - } else if (object_type_argument->skip_path_regexp) { const auto * literal = object_type_argument->skip_path_regexp->as(); @@ -476,10 +466,8 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: } } - std::vector path_prefixes_to_skip(path_prefixes_to_skip_set.begin(), path_prefixes_to_skip_set.end()); - std::sort(path_prefixes_to_skip.begin(), path_prefixes_to_skip.end()); std::sort(path_regexps_to_skip.begin(), path_regexps_to_skip.end()); - return std::make_shared(schema_format, typed_paths, paths_to_skip, path_prefixes_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); + return std::make_shared(schema_format, typed_paths, paths_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); } static DataTypePtr createJSON(const ASTPtr & arguments) diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 8f580f03b57..61364dfe5ed 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -32,7 +32,6 @@ public: const SchemaFormat & schema_format_, const std::unordered_map & typed_paths_ = {}, const std::unordered_set & paths_to_skip_ = {}, - const std::vector & path_prefixes_to_skip_ = {}, const std::vector & path_regexps_to_skip_ = {}, size_t max_dynamic_paths_ = DEFAULT_MAX_SEPARATELY_STORED_PATHS, size_t max_dynamic_types_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES); @@ -67,7 +66,6 @@ public: const SchemaFormat & getSchemaFormat() const { return schema_format; } const std::unordered_map & getTypedPaths() const { return typed_paths; } const std::unordered_set & getPathsToSkip() const { return paths_to_skip; } - const std::vector & getPathPrefixesToSkip() const { return path_prefixes_to_skip; } const std::vector & getPathRegexpsToSkip() const { return path_regexps_to_skip; } size_t getMaxDynamicTypes() const { return max_dynamic_types; } @@ -79,8 +77,6 @@ private: std::unordered_map typed_paths; /// Set of paths that should be skipped during data parsing. std::unordered_set paths_to_skip; - /// List of path prefixes that should be skipped during data parsing. - std::vector path_prefixes_to_skip; /// List of regular expressions that should be used to skip paths during data parsing. std::vector path_regexps_to_skip; /// Limit on the number of paths that can be stored as subcolumn. diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index 102e5eaede0..bd97bceff4c 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -507,10 +507,6 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) writeVarUInt(paths_to_skip.size(), buf); for (const auto & path : paths_to_skip) writeStringBinary(path, buf); - const auto & path_prefixes_to_skip = object_type.getPathPrefixesToSkip(); - writeVarUInt(path_prefixes_to_skip.size(), buf); - for (const auto & prefix : path_prefixes_to_skip) - writeStringBinary(prefix, buf); const auto & path_regexps_to_skip = object_type.getPathRegexpsToSkip(); writeVarUInt(path_regexps_to_skip.size(), buf); for (const auto & regexp : path_regexps_to_skip) @@ -754,17 +750,6 @@ DataTypePtr decodeDataType(ReadBuffer & buf) paths_to_skip.insert(path); } - size_t path_prefixes_to_skip_size; - readVarUInt(path_prefixes_to_skip_size, buf); - std::vector path_prefixes_to_skip; - path_prefixes_to_skip.reserve(path_prefixes_to_skip_size); - for (size_t i = 0; i != path_prefixes_to_skip_size; ++i) - { - String prefix; - readStringBinary(prefix, buf); - path_prefixes_to_skip.push_back(prefix); - } - size_t path_regexps_to_skip_size; readVarUInt(path_regexps_to_skip_size, buf); std::vector path_regexps_to_skip; @@ -779,7 +764,6 @@ DataTypePtr decodeDataType(ReadBuffer & buf) DataTypeObject::SchemaFormat::JSON, typed_paths, paths_to_skip, - path_prefixes_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); diff --git a/src/DataTypes/DataTypesBinaryEncoding.h b/src/DataTypes/DataTypesBinaryEncoding.h index de001966aee..5ac21471a29 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.h +++ b/src/DataTypes/DataTypesBinaryEncoding.h @@ -8,59 +8,59 @@ namespace DB /** Binary encoding for ClickHouse data types: -|---------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| ClickHouse data type | Binary encoding | -|---------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| Nothing | 0x00 | -| UInt8 | 0x01 | -| UInt16 | 0x02 | -| UInt32 | 0x03 | -| UInt64 | 0x04 | -| UInt128 | 0x05 | -| UInt256 | 0x06 | -| Int8 | 0x07 | -| Int16 | 0x08 | -| Int32 | 0x09 | -| Int64 | 0x0A | -| Int128 | 0x0B | -| Int256 | 0x0C | -| Float32 | 0x0D | -| Float64 | 0x0E | -| Date | 0x0F | -| Date32 | 0x10 | -| DateTime | 0x11 | -| DateTime(time_zone) | 0x12 | -| DateTime64(P) | 0x13 | -| DateTime64(P, time_zone) | 0x14 | -| String | 0x15 | -| FixedString(N) | 0x16 | -| Enum8 | 0x17... | -| Enum16 | 0x18...> | -| Decimal32(P, S) | 0x19 | -| Decimal64(P, S) | 0x1A | -| Decimal128(P, S) | 0x1B | -| Decimal256(P, S) | 0x1C | -| UUID | 0x1D | -| Array(T) | 0x1E | -| Tuple(T1, ..., TN) | 0x1F... | -| Tuple(name1 T1, ..., nameN TN) | 0x20... | -| Set | 0x21 | -| Interval | 0x22 | -| Nullable(T) | 0x23 | -| Function | 0x24... | -| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | -| LowCardinality(T) | 0x26 | -| Map(K, V) | 0x27 | -| IPv4 | 0x28 | -| IPv6 | 0x29 | -| Variant(T1, ..., TN) | 0x2A... | -| Dynamic(max_types=N) | 0x2B | -| Custom type (Ring, Polygon, etc) | 0x2C | -| Bool | 0x2D | -| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | -| Nested(name1 T1, ..., nameN TN) | 0x2F... | -| JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP PREFIX skip_path_prefix, SKIP REGEXP skip_path_regexp) | 0x30............ | -|---------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +|---------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| ClickHouse data type | Binary encoding | +|---------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Nothing | 0x00 | +| UInt8 | 0x01 | +| UInt16 | 0x02 | +| UInt32 | 0x03 | +| UInt64 | 0x04 | +| UInt128 | 0x05 | +| UInt256 | 0x06 | +| Int8 | 0x07 | +| Int16 | 0x08 | +| Int32 | 0x09 | +| Int64 | 0x0A | +| Int128 | 0x0B | +| Int256 | 0x0C | +| Float32 | 0x0D | +| Float64 | 0x0E | +| Date | 0x0F | +| Date32 | 0x10 | +| DateTime | 0x11 | +| DateTime(time_zone) | 0x12 | +| DateTime64(P) | 0x13 | +| DateTime64(P, time_zone) | 0x14 | +| String | 0x15 | +| FixedString(N) | 0x16 | +| Enum8 | 0x17... | +| Enum16 | 0x18...> | +| Decimal32(P, S) | 0x19 | +| Decimal64(P, S) | 0x1A | +| Decimal128(P, S) | 0x1B | +| Decimal256(P, S) | 0x1C | +| UUID | 0x1D | +| Array(T) | 0x1E | +| Tuple(T1, ..., TN) | 0x1F... | +| Tuple(name1 T1, ..., nameN TN) | 0x20... | +| Set| 0x21 | +| Interval | 0x22 | +| Nullable(T) | 0x23 | +| Function | 0x24... | +| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | +| LowCardinality(T) | 0x26 | +| Map(K, V) | 0x27 | +| IPv4 | 0x28 | +| IPv6 | 0x29 | +| Variant(T1, ..., TN) | 0x2A... | +| Dynamic(max_types=N) | 0x2B | +| Custom type (Ring, Polygon, etc) | 0x2C | +| Bool | 0x2D | +| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | +| Nested(name1 T1, ..., nameN TN) | 0x2F... | +| JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP REGEXP skip_path_regexp) | 0x30......... | +|---------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| Interval kind binary encoding: |---------------|-----------------| diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 6cc7a7e2e74..c1ef67b2aaa 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -338,6 +338,8 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu } const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(global_discr); + if (!variant_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Nullptr type in variant {}", variant_info.variant_name); encodeDataType(variant_type, ostr); variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); } diff --git a/src/DataTypes/Serializations/SerializationJSON.cpp b/src/DataTypes/Serializations/SerializationJSON.cpp index 51b12568981..346e6785abc 100644 --- a/src/DataTypes/Serializations/SerializationJSON.cpp +++ b/src/DataTypes/Serializations/SerializationJSON.cpp @@ -22,10 +22,9 @@ template SerializationJSON::SerializationJSON( std::unordered_map typed_paths_serializations_, const std::unordered_set & paths_to_skip_, - const std::vector & path_prefixes_to_skip_, const std::vector & path_regexps_to_skip_, std::unique_ptr> json_extract_tree_) - : SerializationObject(std::move(typed_paths_serializations_), paths_to_skip_, path_prefixes_to_skip_, path_regexps_to_skip_) + : SerializationObject(std::move(typed_paths_serializations_), paths_to_skip_, path_regexps_to_skip_) , json_extract_tree(std::move(json_extract_tree_)) { } diff --git a/src/DataTypes/Serializations/SerializationJSON.h b/src/DataTypes/Serializations/SerializationJSON.h index 0c6d2bd164b..aee1413bdd3 100644 --- a/src/DataTypes/Serializations/SerializationJSON.h +++ b/src/DataTypes/Serializations/SerializationJSON.h @@ -15,7 +15,6 @@ public: SerializationJSON( std::unordered_map typed_paths_serializations_, const std::unordered_set & paths_to_skip_, - const std::vector & path_prefixes_to_skip_, const std::vector & path_regexps_to_skip_, std::unique_ptr> json_extract_tree_); diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 8e6f6f105ea..0fa1ef7d2ca 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -19,11 +19,9 @@ namespace ErrorCodes SerializationObject::SerializationObject( std::unordered_map typed_path_serializations_, const std::unordered_set & paths_to_skip_, - const std::vector & path_prefixes_to_skip_, const std::vector & path_regexps_to_skip_) : typed_path_serializations(std::move(typed_path_serializations_)) , paths_to_skip(paths_to_skip_) - , path_prefixes_to_skip(path_prefixes_to_skip_) , dynamic_serialization(std::make_shared()) , shared_data_serialization(getTypeOfSharedData()->getDefaultSerialization()) { @@ -32,6 +30,8 @@ SerializationObject::SerializationObject( for (const auto & [path, _] : typed_path_serializations) sorted_typed_paths.emplace_back(path); std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); + sorted_paths_to_skip.assign(paths_to_skip.begin(), paths_to_skip.end()); + std::sort(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end()); for (const auto & regexp_str : path_regexps_to_skip_) path_regexps_to_skip.emplace_back(regexp_str); } @@ -48,11 +48,9 @@ bool SerializationObject::shouldSkipPath(const String & path) const if (paths_to_skip.contains(path)) return true; - for (const auto & prefix : path_prefixes_to_skip) - { - if (path.starts_with(prefix)) - return true; - } + auto it = std::lower_bound(sorted_typed_paths.begin(), sorted_typed_paths.end(), path); + if (it != sorted_paths_to_skip.end() && it != sorted_paths_to_skip.begin() && path.starts_with(*std::prev(it))) + return true; for (const auto & regexp : path_regexps_to_skip) { diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 8702b0c7f64..b279fab2603 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -32,7 +32,6 @@ public: SerializationObject( std::unordered_map typed_path_serializations_, const std::unordered_set & paths_to_skip_, - const std::vector & path_prefixes_to_skip_, const std::vector & path_regexps_to_skip_); void enumerateStreams( @@ -114,7 +113,7 @@ protected: std::unordered_map typed_path_serializations; std::unordered_set paths_to_skip; - std::vector path_prefixes_to_skip; + std::vector sorted_paths_to_skip; std::list path_regexps_to_skip; SerializationPtr dynamic_serialization; diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index e68e98a6a06..2eaddddb567 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1386,10 +1386,13 @@ public: if (column_dynamic.addNewVariant(element_type)) { - auto node = buildJSONExtractTree(element_type, "Dynamic inference"); - auto global_discriminator = variant_info.variant_name_to_discriminator.at(element_type->getName()); + auto element_type_name = element_type->getName(); + auto it = json_extract_nodes_cache.find(element_type_name); + if (it == json_extract_nodes_cache.end()) + it = json_extract_nodes_cache.emplace(element_type_name, buildJSONExtractTree(element_type, "Dynamic inference")).first; + auto global_discriminator = variant_info.variant_name_to_discriminator.at(element_type_name); auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discriminator); - if (!node->insertResultToColumn(variant, element, insert_settings, format_settings, error)) + if (!it->second->insertResultToColumn(variant, element, insert_settings, format_settings, error)) return false; variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(global_discriminator)); variant_column.getOffsets().push_back(variant.size() - 1); @@ -1397,8 +1400,10 @@ public: } /// We couldn't add new variant. Try to insert element into current variants. - auto variant_node = buildJSONExtractTree(variant_info.variant_type, "Dynamic inference"); - if (variant_node->insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) + auto it = json_extract_nodes_cache.find(variant_info.variant_name); + if (it == json_extract_nodes_cache.end()) + it = json_extract_nodes_cache.emplace(variant_info.variant_name, buildJSONExtractTree(variant_info.variant_type, "Dynamic inference")).first; + if (it->second->insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) return true; /// We couldn't insert element into any existing variant, add String variant and read value as String. @@ -1424,7 +1429,7 @@ public: JSONInferenceInfo json_inference_info; auto type = elementToDataTypeImpl(element, format_settings, json_inference_info); transformFinalInferredJSONTypeIfNeeded(type, format_settings, &json_inference_info); - if (format_settings.schema_inference_make_columns_nullable) + if (format_settings.schema_inference_make_columns_nullable && type->haveSubtypes()) type = makeNullableRecursively(type); return type; } @@ -1435,20 +1440,20 @@ private: switch (element.type()) { case ElementType::NULL_VALUE: - return makeNullable(std::make_shared()); + return getNullType(); case ElementType::BOOL: - return DataTypeFactory::instance().get("Bool"); + return getBoolType(); case ElementType::INT64: { - auto type = std::make_shared(); + auto type = getInt64Type(); if (element.getInt64() < 0) json_inference_info.negative_integers.insert(type.get()); return type; } case ElementType::UINT64: - return std::make_shared(); + return getUInt64Type(); case ElementType::DOUBLE: - return std::make_shared(); + return getFloat64Type(); case ElementType::STRING: { auto data = element.getString(); @@ -1465,7 +1470,7 @@ private: } } - return std::make_shared(); + return getStringType(); } case ElementType::ARRAY: { @@ -1476,7 +1481,7 @@ private: types.push_back(elementToDataTypeImpl(value, format_settings, json_inference_info)); if (types.empty()) - return std::make_shared(std::make_shared()); + return getEmptyArrayType(); if (checkIfTypesAreEqual(types)) return std::make_shared(types.back()); @@ -1505,13 +1510,67 @@ private: } case ElementType::OBJECT: { - return std::make_shared(DataTypeObject::SchemaFormat::JSON, max_dynamic_paths_for_object, max_dynamic_types_for_object); + return getObjectType(); } } } + /// During schema inference we create shared_ptr to the some data types quite a lot. + /// Single creating of such shared_ptr is not expensive, but when it happens on each + /// column on each row, it can be noticeable. + const DataTypePtr & getBoolType() const + { + static const DataTypePtr bool_type = DataTypeFactory::instance().get("Bool"); + return bool_type; + } + + const DataTypePtr & getStringType() const + { + static const DataTypePtr string_type = std::make_shared(); + return string_type; + } + + const DataTypePtr & getInt64Type() const + { + static const DataTypePtr int64_type = std::make_shared(); + return int64_type; + } + + const DataTypePtr & getUInt64Type() const + { + static const DataTypePtr uint64_type = std::make_shared(); + return uint64_type; + } + + const DataTypePtr & getFloat64Type() const + { + static const DataTypePtr float64_type = std::make_shared(); + return float64_type; + } + + const DataTypePtr & getObjectType() const + { + static const DataTypePtr object_type = std::make_shared(DataTypeObject::SchemaFormat::JSON, max_dynamic_paths_for_object, max_dynamic_types_for_object); + return object_type; + } + + const DataTypePtr & getNullType() const + { + static const DataTypePtr null_type = std::make_shared(std::make_shared()); + return null_type; + } + + const DataTypePtr & getEmptyArrayType() const + { + static const DataTypePtr empty_array_type = std::make_shared(std::make_shared()); + return empty_array_type; + } + size_t max_dynamic_paths_for_object; size_t max_dynamic_types_for_object; + + /// Avoid building JSONExtractTreeNode for the same data types on each row by using cache. + mutable std::unordered_map>> json_extract_nodes_cache; }; template @@ -1521,18 +1580,18 @@ public: ObjectJSONNode( std::unordered_map>> typed_path_nodes_, const std::unordered_set & paths_to_skip_, - const std::vector & path_prefixes_to_skip_, const std::vector & path_regexps_to_skip_, size_t max_dynamic_paths_, size_t max_dynamic_types) : typed_path_nodes(std::move(typed_path_nodes_)) , paths_to_skip(paths_to_skip_) - , path_prefixes_to_skip(path_prefixes_to_skip_) , dynamic_node(std::make_unique>( max_dynamic_paths_ / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))) , dynamic_serialization(std::make_shared()) { + sorted_paths_to_skip.assign(paths_to_skip.begin(), paths_to_skip.end()); + std::sort(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end()); for (const auto & regexp : path_regexps_to_skip_) path_regexps_to_skip.emplace_back(regexp); } @@ -1616,6 +1675,9 @@ private: size_t current_size, String & error) const { + if (shouldSkipPath(current_path)) + return true; + if (element.isObject() && !typed_path_nodes.contains(current_path)) { for (auto [key, value] : element.getObject()) @@ -1631,9 +1693,6 @@ private: return true; } - if (shouldSkipPath(current_path)) - return true; - auto & typed_paths = column_object.getTypedPaths(); auto & dynamic_paths = column_object.getDynamicPaths(); /// Check if we have this path in typed paths. @@ -1707,11 +1766,9 @@ private: if (paths_to_skip.contains(path)) return true; - for (const auto & prefix : path_prefixes_to_skip) - { - if (path.starts_with(prefix)) - return true; - } + auto it = std::lower_bound(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end(), path); + if (it != sorted_paths_to_skip.end() && it != sorted_paths_to_skip.begin() && path.starts_with(*std::prev(it))) + return true; for (const auto & regexp : path_regexps_to_skip) { @@ -1724,7 +1781,7 @@ private: std::unordered_map>> typed_path_nodes; std::unordered_set paths_to_skip; - std::vector path_prefixes_to_skip; + std::vector sorted_paths_to_skip; std::list path_regexps_to_skip; std::unique_ptr> dynamic_node; std::shared_ptr dynamic_serialization; @@ -1889,7 +1946,6 @@ std::unique_ptr> buildJSONExtractTree(const Data return std::make_unique>( std::move(typed_path_nodes), object_type.getPathsToSkip(), - object_type.getPathPrefixesToSkip(), object_type.getPathRegexpsToSkip(), object_type.getMaxDynamicPaths(), object_type.getMaxDynamicTypes()); diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 15dc75c0f3a..d245200eefa 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -36,6 +36,63 @@ namespace ErrorCodes namespace { + /// During schema inference we create shared_ptr to the some data types quite a lot. + /// Single creating of such shared_ptr is not expensive, but when it happens on each + /// column on each row, it can be noticeable. + const DataTypePtr & getBoolType() + { + static const DataTypePtr bool_type = DataTypeFactory::instance().get("Bool"); + return bool_type; + } + + const DataTypePtr & getStringType() + { + static const DataTypePtr string_type = std::make_shared(); + return string_type; + } + + const DataTypePtr & getInt64Type() + { + static const DataTypePtr int64_type = std::make_shared(); + return int64_type; + } + + const DataTypePtr & getUInt64Type() + { + static const DataTypePtr uint64_type = std::make_shared(); + return uint64_type; + } + + const DataTypePtr & getFloat64Type() + { + static const DataTypePtr float64_type = std::make_shared(); + return float64_type; + } + + const DataTypePtr & getDateType() + { + static const DataTypePtr date_type = std::make_shared(); + return date_type; + } + + const DataTypePtr & getDateTime64Type() + { + static const DataTypePtr date_type = std::make_shared(9); + return date_type; + } + + const DataTypePtr & getNullType() + { + static const DataTypePtr null_type = std::make_shared(std::make_shared()); + return null_type; + } + + const DataTypePtr & getEmptyArrayType() + { + static const DataTypePtr empty_array_type = std::make_shared(std::make_shared()); + return empty_array_type; + } + /// Special data type that represents JSON object as a set of paths and their types. /// It supports merging two JSON objects and creating Named Tuple from itself. /// It's used only for schema inference of Named Tuples from JSON objects. @@ -208,7 +265,7 @@ namespace if (leaf_type && !isNothing(removeNullable(leaf_type)) && !nodes.empty()) { if (use_string_type_for_ambiguous_paths) - return std::make_shared(); + return getStringType(); throw Exception( ErrorCodes::INCORRECT_DATA, @@ -274,7 +331,7 @@ namespace bool is_negative = json_info && json_info->negative_integers.contains(type.get()); have_negative_integers |= is_negative; if (!is_negative) - type = std::make_shared(); + type = getUInt64Type(); } } @@ -295,7 +352,7 @@ namespace WhichDataType which(type); if (which.isInt64() || which.isUInt64()) { - auto new_type = std::make_shared(); + auto new_type = getFloat64Type(); if (json_info && json_info->numbers_parsed_from_json_strings.erase(type.get())) json_info->numbers_parsed_from_json_strings.insert(new_type.get()); type = new_type; @@ -319,7 +376,7 @@ namespace for (auto & type : data_types) { if (isDate(type) || isDateTime64(type)) - type = std::make_shared(); + type = getStringType(); } type_indexes.erase(TypeIndex::Date); @@ -333,7 +390,7 @@ namespace for (auto & type : data_types) { if (isDate(type)) - type = std::make_shared(9); + type = getDateTime64Type(); } type_indexes.erase(TypeIndex::Date); @@ -355,7 +412,7 @@ namespace if (isNumber(type) && (settings.json.read_numbers_as_strings || !json_info || json_info->numbers_parsed_from_json_strings.contains(type.get()))) - type = std::make_shared(); + type = getStringType(); } updateTypeIndexes(data_types, type_indexes); @@ -378,11 +435,11 @@ namespace if (isBool(type)) { if (have_signed_integers) - type = std::make_shared(); + type = getInt64Type(); else if (have_unsigned_integers) - type = std::make_shared(); + type = getUInt64Type(); else - type = std::make_shared(); + type = getFloat64Type(); } } @@ -399,7 +456,7 @@ namespace for (auto & type : data_types) { if (isBool(type)) - type = std::make_shared(); + type = getStringType(); } type_indexes.erase(TypeIndex::UInt8); @@ -549,7 +606,7 @@ namespace for (auto & type : data_types) { if (isMap(type)) - type = std::make_shared(); + type = getStringType(); } type_indexes.erase(TypeIndex::Map); @@ -697,37 +754,37 @@ namespace bool tryInferDate(std::string_view field) { - if (field.empty()) + /// Minimum length of Date text representation is 8 (YYYY-M-D) and maximum is 10 (YYYY-MM-DD) + if (field.size() < 8 || field.size() > 10) return false; - ReadBufferFromString buf(field); - Float64 tmp_float; /// Check if it's just a number, and if so, don't try to infer Date from it, /// because we can interpret this number as a Date (for example 20000101 will be 2000-01-01) /// and it will lead to inferring Date instead of simple Int64/UInt64 in some cases. - if (tryReadFloatText(tmp_float, buf) && buf.eof()) + if (std::all_of(field.begin(), field.end(), isNumericASCII)) return false; - buf.seek(0, SEEK_SET); /// Return position to the beginning - + ReadBufferFromString buf(field); DayNum tmp; return tryReadDateText(tmp, buf) && buf.eof(); } bool tryInferDateTime(std::string_view field, const FormatSettings & settings) { - if (field.empty()) + /// Don't try to infer DateTime if string is too long. + /// It's difficult to say what is the real maximum length of + /// DateTime we can parse using BestEffort approach. + /// 40 symbols is more or less valid limit. + if (field.empty() || field.size() > 40) return false; - ReadBufferFromString buf(field); - Float64 tmp_float; /// Check if it's just a number, and if so, don't try to infer DateTime from it, /// because we can interpret this number as a timestamp and it will lead to /// inferring DateTime instead of simple Int64/Float64 in some cases. - if (tryReadFloatText(tmp_float, buf) && buf.eof()) + if (std::all_of(field.begin(), field.end(), isNumericASCII)) return false; - buf.seek(0, SEEK_SET); /// Return position to the beginning + ReadBufferFromString buf(field); DateTime64 tmp; switch (settings.date_time_input_format) { @@ -792,7 +849,7 @@ namespace /// Empty array has type Array(Nothing) if (nested_types.empty()) - return std::make_shared(std::make_shared()); + return getEmptyArrayType(); if (checkIfTypesAreEqual(nested_types)) return std::make_shared(std::move(nested_types.back())); @@ -905,13 +962,13 @@ namespace /// NOTE: it may break parsing of tryReadFloat() != tryReadIntText() + parsing of '.'/'e' /// But, for now it is true if (tryReadFloat(tmp_float, buf, settings, has_fractional) && has_fractional) - return std::make_shared(); + return getFloat64Type(); Int64 tmp_int; buf.position() = number_start; if (tryReadIntText(tmp_int, buf)) { - auto type = std::make_shared(); + auto type = getInt64Type(); if (json_info && tmp_int < 0) json_info->negative_integers.insert(type.get()); return type; @@ -921,7 +978,7 @@ namespace UInt64 tmp_uint; buf.position() = number_start; if (tryReadIntText(tmp_uint, buf)) - return std::make_shared(); + return getUInt64Type(); return nullptr; } @@ -933,13 +990,13 @@ namespace PeekableReadBufferCheckpoint checkpoint(peekable_buf); if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional) && has_fractional) - return std::make_shared(); + return getFloat64Type(); peekable_buf.rollbackToCheckpoint(/* drop= */ false); Int64 tmp_int; if (tryReadIntText(tmp_int, peekable_buf)) { - auto type = std::make_shared(); + auto type = getInt64Type(); if (json_info && tmp_int < 0) json_info->negative_integers.insert(type.get()); return type; @@ -949,11 +1006,11 @@ namespace /// In case of Int64 overflow we can try to infer UInt64. UInt64 tmp_uint; if (tryReadIntText(tmp_uint, peekable_buf)) - return std::make_shared(); + return getUInt64Type(); } else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { - return std::make_shared(); + return getFloat64Type(); } /// This is not a number. @@ -970,7 +1027,7 @@ namespace Int64 tmp_int; if (tryReadIntText(tmp_int, buf) && buf.eof()) { - auto type = std::make_shared(); + auto type = getInt64Type(); if (json_inference_info && tmp_int < 0) json_inference_info->negative_integers.insert(type.get()); return type; @@ -982,7 +1039,7 @@ namespace /// In case of Int64 overflow, try to infer UInt64 UInt64 tmp_uint; if (tryReadIntText(tmp_uint, buf) && buf.eof()) - return std::make_shared(); + return getUInt64Type(); } /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. @@ -991,7 +1048,7 @@ namespace Float64 tmp; bool has_fractional; if (tryReadFloat(tmp, buf, settings, has_fractional) && buf.eof()) - return std::make_shared(); + return getFloat64Type(); return nullptr; } @@ -1015,7 +1072,7 @@ namespace if constexpr (is_json) { if (json_info->is_object_key) - return std::make_shared(); + return getStringType(); } if (auto type = tryInferDateOrDateTimeFromString(field, settings)) @@ -1033,7 +1090,7 @@ namespace } } - return std::make_shared(); + return getStringType(); } bool tryReadJSONObject(ReadBuffer & buf, const FormatSettings & settings, DataTypeJSONPaths::Paths & paths, const std::vector & path, JSONInferenceInfo * json_info, size_t depth) @@ -1190,7 +1247,7 @@ namespace return std::make_shared("json", true); if (settings.json.read_objects_as_strings) - return std::make_shared(); + return getStringType(); transformInferredTypesIfNeededImpl(value_types, settings, json_info); if (!checkIfTypesAreEqual(value_types)) @@ -1256,15 +1313,15 @@ namespace /// Bool if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf)) - return DataTypeFactory::instance().get("Bool"); + return getBoolType(); /// Null or NaN if (checkCharCaseInsensitive('n', buf)) { if (checkStringCaseInsensitive("ull", buf)) - return makeNullable(std::make_shared()); + return getNullType(); else if (checkStringCaseInsensitive("an", buf)) - return std::make_shared(); + return getFloat64Type(); } /// Number @@ -1321,7 +1378,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F if (!remain_nothing_types && isNothing(data_type) && settings.json.infer_incomplete_types_as_strings) { - data_type = std::make_shared(); + data_type = getStringType(); return; } @@ -1338,7 +1395,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F /// If all objects were empty, use type String, so these JSON objects will be read as Strings. if (json_paths->empty() && settings.json.infer_incomplete_types_as_strings) { - data_type = std::make_shared(); + data_type = getStringType(); return; } @@ -1360,7 +1417,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F auto key_type = map_type->getKeyType(); /// If all inferred Maps are empty, use type String, so these JSON objects will be read as Strings. if (isNothing(key_type) && settings.json.infer_incomplete_types_as_strings) - key_type = std::make_shared(); + key_type = getStringType(); auto value_type = map_type->getValueType(); @@ -1437,10 +1494,10 @@ DataTypePtr tryInferJSONNumberFromString(std::string_view field, const FormatSet DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const FormatSettings & settings) { if (settings.try_infer_dates && tryInferDate(field)) - return std::make_shared(); + return getDateType(); if (settings.try_infer_datetimes && tryInferDateTime(field, settings)) - return std::make_shared(9); + return getDateTime64Type(); return nullptr; } diff --git a/src/Parsers/ASTObjectTypeArgument.cpp b/src/Parsers/ASTObjectTypeArgument.cpp index afe875586ad..aa5d23d7881 100644 --- a/src/Parsers/ASTObjectTypeArgument.cpp +++ b/src/Parsers/ASTObjectTypeArgument.cpp @@ -21,11 +21,6 @@ ASTPtr ASTObjectTypeArgument::clone() const res->skip_path = skip_path->clone(); res->children.push_back(res->skip_path); } - else if (skip_path_prefix) - { - res->skip_path_prefix = skip_path_prefix->clone(); - res->children.push_back(res->skip_path_prefix); - } else if (skip_path_regexp) { res->skip_path_regexp = skip_path_regexp->clone(); @@ -56,12 +51,6 @@ void ASTObjectTypeArgument::formatImpl(const FormatSettings & parameters, Format parameters.ostr << indent_str << "SKIP" << ' '; skip_path->formatImpl(parameters, state, frame); } - else if (skip_path_prefix) - { - std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); - parameters.ostr << indent_str << "SKIP PREFIX" << ' '; - skip_path_prefix->formatImpl(parameters, state, frame); - } else if (skip_path_regexp) { std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); diff --git a/src/Parsers/ASTObjectTypeArgument.h b/src/Parsers/ASTObjectTypeArgument.h index 3a6b2bcdd98..ab18d00d770 100644 --- a/src/Parsers/ASTObjectTypeArgument.h +++ b/src/Parsers/ASTObjectTypeArgument.h @@ -9,7 +9,6 @@ namespace DB /** An argument of Object data type declaration (for example for JSON). Can contain one of: * - pair (path, data type) * - path that should be skipped - * - path prefix for paths that should be skipped * - path regexp for paths that should be skipped * - setting in a form of `setting=N` */ @@ -18,7 +17,6 @@ class ASTObjectTypeArgument : public IAST public: ASTPtr path_with_type; ASTPtr skip_path; - ASTPtr skip_path_prefix; ASTPtr skip_path_regexp; ASTPtr parameter; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index f4793c7fc27..0b40ace3ca8 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -69,25 +69,16 @@ private: argument->skip_path_regexp = literal; argument->children.push_back(argument->skip_path_regexp); } - /// SKIP PREFIX some.path.prefix or SKIP some.path + /// SKIP some.path else { - bool is_prefix = ParserKeyword(Keyword::PREFIX).ignore(pos); ParserCompoundIdentifier compound_identifier_parser; ASTPtr compound_identifier; if (!compound_identifier_parser.parse(pos, compound_identifier, expected)) return false; - if (is_prefix) - { - argument->skip_path_prefix = compound_identifier; - argument->children.push_back(argument->skip_path_prefix); - } - else - { - argument->skip_path = compound_identifier; - argument->children.push_back(argument->skip_path); - } + argument->skip_path = compound_identifier; + argument->children.push_back(argument->skip_path); } node = argument; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 27c52124e9c..eb251c75499 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -283,6 +283,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes)) node_copy.children.push_back(child_copy); + if (node_copy.children.empty()) return nullptr; From 4df94a0ef3f8af73328d0a8f45bb217cc70b2e45 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 14:47:52 +0000 Subject: [PATCH 0280/1722] cleanup for setting in mergetree --- src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.cpp | 1 - src/Interpreters/InterpreterDeleteQuery.cpp | 66 +++++-------------- src/Storages/IStorage.h | 3 - .../MergeTree/MergeMutateSelectedEntry.h | 7 +- src/Storages/MergeTree/MergeTreeData.cpp | 15 ----- src/Storages/MergeTree/MergeTreeData.h | 2 - .../MergeTree/MergeTreeMutationEntry.cpp | 4 +- .../MergeTree/MergeTreeMutationEntry.h | 7 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/MutatePlainMergeTreeTask.cpp | 1 - src/Storages/MergeTree/MutateTask.cpp | 10 --- src/Storages/StorageMergeTree.cpp | 10 +-- 13 files changed, 25 insertions(+), 103 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f7b44ea775c..bafc3f93846 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -588,7 +588,6 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(UInt64, lightweight_deletes_sync, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 194292a467e..d6cc0112e0a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -70,7 +70,6 @@ static std::initializer_listlockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); - bool hasProjection = table->hasProjection(); - auto lightweightDelete = [&]() + if (table->supportsDelete()) + { + /// Convert to MutationCommand + MutationCommands mutation_commands; + MutationCommand mut_command; + + mut_command.type = MutationCommand::Type::DELETE; + mut_command.predicate = delete_query.predicate; + + mutation_commands.emplace_back(mut_command); + + table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); + MutationsInterpreter::Settings settings(false); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); + table->mutate(mutation_commands, getContext()); + return {}; + } + else if (table->supportsLightweightDelete()) { if (!getContext()->getSettingsRef().enable_lightweight_delete) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, @@ -89,54 +104,9 @@ BlockIO InterpreterDeleteQuery::execute() context->setSetting("mutations_sync", Field(context->getSettingsRef().lightweight_deletes_sync)); InterpreterAlterQuery alter_interpreter(alter_ast, context); return alter_interpreter.execute(); - }; - - if (table->supportsDelete()) - { - /// Convert to MutationCommand - MutationCommands mutation_commands; - MutationCommand mut_command; - - mut_command.type = MutationCommand::Type::DELETE; - mut_command.predicate = delete_query.predicate; - - mutation_commands.emplace_back(mut_command); - - table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter::Settings settings(false); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); - table->mutate(mutation_commands, getContext()); - return {}; - } - else if (!hasProjection && table->supportsLightweightDelete()) - { - return lightweightDelete(); } else { - if (hasProjection) - { - auto context = Context::createCopy(getContext()); - auto mode = context->getSettingsRef().lightweight_mutation_projection_mode; - - if (mode == LightweightMutationProjectionMode::THROW) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DELETE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", - table->getStorageID().getFullTableName()); - } - else if (mode == LightweightMutationProjectionMode::DROP) - { - return lightweightDelete(); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unrecognized lightweight_mutation_projection_mode, only throw and drop are allowed."); - } - } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "DELETE query is not supported for table {}", table->getStorageID().getFullTableName()); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6217470780d..991c8ff64af 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -262,9 +262,6 @@ public: /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } - /// Return true if storage has any projection. - virtual bool hasProjection() const { return false; } - /// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index f75d10d9ecb..e7efe00741c 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -2,7 +2,7 @@ #include #include -#include + namespace DB { @@ -41,15 +41,12 @@ struct MergeMutateSelectedEntry CurrentlyMergingPartsTaggerPtr tagger; MutationCommandsConstPtr commands; MergeTreeTransactionPtr txn; - LightweightMutationProjectionMode lightweight_delete_projection_mode; MergeMutateSelectedEntry(FutureMergedMutatedPartPtr future_part_, CurrentlyMergingPartsTaggerPtr tagger_, - MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR, - const LightweightMutationProjectionMode & lightweight_delete_projection_mode_ = LightweightMutationProjectionMode::THROW) + MutationCommandsConstPtr commands_, const MergeTreeTransactionPtr & txn_ = NO_TRANSACTION_PTR) : future_part(future_part_) , tagger(std::move(tagger_)) , commands(commands_) , txn(txn_) - , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) {} }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e31f6db5409..5182147350e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6158,21 +6158,6 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } -bool MergeTreeData::hasProjection() const -{ - auto lock = lockParts(); - for (const auto & part : data_parts_by_info) - { - if (part->getState() == MergeTreeDataPartState::Outdated - || part->getState() == MergeTreeDataPartState::Deleting) - continue; - - if (part->hasProjection()) - return true; - } - return false; -} - MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c8b721038c6..7d216f989c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -439,8 +439,6 @@ public: bool supportsLightweightDelete() const override; - bool hasProjection() const override; - bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; } bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index d1bd8efa7a5..4dbccb91620 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -48,8 +48,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) } MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings, - const LightweightMutationProjectionMode & lightweight_delete_projection_mode_) + const TransactionID & tid_, const WriteSettings & settings) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -57,7 +56,6 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") , is_temp(true) , tid(tid_) - , lightweight_delete_projection_mode(lightweight_delete_projection_mode_) { try { diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index dbb17654ddd..04297f2852a 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -37,13 +36,9 @@ struct MergeTreeMutationEntry /// or UnknownCSN if it's not committed (yet) or RolledBackCSN if it's rolled back or PrehistoricCSN if there is no transaction. CSN csn = Tx::UnknownCSN; - /// From query context. - LightweightMutationProjectionMode lightweight_delete_projection_mode; - /// Create a new entry and write it to a temporary file. MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number, - const TransactionID & tid_, const WriteSettings & settings, - const LightweightMutationProjectionMode & lightweight_delete_projection_mode_); + const TransactionID & tid_, const WriteSettings & settings); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c0afd781c7e..a458a21ca1b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -215,6 +215,7 @@ struct Settings; M(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ /** Projection settings. */ \ M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 19aa63d90a2..20f387137e7 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -140,7 +140,6 @@ ContextMutablePtr MutatePlainMergeTreeTask::createTaskContext() const auto queryId = getQueryId(); context->setCurrentQueryId(queryId); context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::MUTATION); - // context->setSetting("lightweight_mutation_projection_mode", merge_mutate_entry->lightweight_delete_projection_mode); return context; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ed603abd9c3..0734174dbef 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2200,7 +2200,6 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); - context_for_reading->setSetting("lightweight_mutation_projection_mode", Field(ctx->context->getSettingsRef().lightweight_mutation_projection_mode)); MutationHelpers::splitAndModifyMutationCommands( ctx->source_part, ctx->metadata_snapshot, @@ -2225,15 +2224,6 @@ bool MutateTask::prepare() ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); - - // ctx->updated_header.has(RowExistsColumn::name); - // for (const auto & projection : ctx->metadata_snapshot->getProjections()) - // { - // if (!ctx->source_part->hasProjection(projection.name)) - // continue; - - // ctx->materialized_projections.insert(projection.name); - // } } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7f210779916..8404e5c9cd9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -517,8 +517,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context { std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings(), - query_context->getSettingsRef().lightweight_mutation_projection_mode); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings()); version = increment.get(); entry.commit(version); String mutation_id = entry.file_name; @@ -1284,17 +1283,12 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; - /// Trying to grab it from query context. - LightweightMutationProjectionMode lightweight_delete_projection_mode = LightweightMutationProjectionMode::THROW; - for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. if (first_mutation_tid != it->second.tid) break; - lightweight_delete_projection_mode = it->second.lightweight_delete_projection_mode; - size_t commands_size = 0; MutationCommands commands_for_size_validation; for (const auto & command : it->second.commands) @@ -1371,7 +1365,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( future_part->part_format = part->getFormat(); tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}, false), *this, metadata_snapshot, true); - return std::make_shared(future_part, std::move(tagger), commands, txn, lightweight_delete_projection_mode); + return std::make_shared(future_part, std::move(tagger), commands, txn); } } From 283de9dc4aca62d18d299bff80cf421c22d02ada Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 Jul 2024 15:00:52 +0000 Subject: [PATCH 0281/1722] Fix JSON type name creation --- src/DataTypes/DataTypeObject.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 751a19beca5..02065bfcc97 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -168,13 +168,13 @@ String DataTypeObject::doGetName() const for (const auto & skip_path : sorted_skip_paths) { write_separator(); - out << "SKIP " << skip_path; + out << "SKIP " << backQuoteIfNeed(skip_path); } for (const auto & skip_regexp : path_regexps_to_skip) { write_separator(); - out << "SKIP REGEXP " << skip_regexp; + out << "SKIP REGEXP " << quoteString(skip_regexp); } if (!first) From 941898a6d9a44487ecdb07eb2e0eb189b79e42e4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 Jul 2024 15:02:12 +0000 Subject: [PATCH 0282/1722] Update documentation --- .../data-types/data-types-binary-encoding.md | 101 +++++++++--------- docs/en/sql-reference/data-types/json.md | 11 +- 2 files changed, 56 insertions(+), 56 deletions(-) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index 812e946e43e..bbd47d6f620 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -12,56 +12,57 @@ This specification describes the binary format that can be used for binary encod The table below describes how each data type is represented in binary format. Each data type encoding consist of 1 byte that indicates the type and some optional additional information. `var_uint` in the binary encoding means that the size is encoded using Variable-Length Quantity compression. -| ClickHouse data type | Binary encoding | -|--------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `Nothing` | `0x00` | -| `UInt8` | `0x01` | -| `UInt16` | `0x02` | -| `UInt32` | `0x03` | -| `UInt64` | `0x04` | -| `UInt128` | `0x05` | -| `UInt256` | `0x06` | -| `Int8` | `0x07` | -| `Int16` | `0x08` | -| `Int32` | `0x09` | -| `Int64` | `0x0A` | -| `Int128` | `0x0B` | -| `Int256` | `0x0C` | -| `Float32` | `0x0D` | -| `Float64` | `0x0E` | -| `Date` | `0x0F` | -| `Date32` | `0x10` | -| `DateTime` | `0x11` | -| `DateTime(time_zone)` | `0x12` | -| `DateTime64(P)` | `0x13` | -| `DateTime64(P, time_zone)` | `0x14` | -| `String` | `0x15` | -| `FixedString(N)` | `0x16` | -| `Enum8` | `0x17...` | -| `Enum16` | `0x18...>` | -| `Decimal32(P, S)` | `0x19` | -| `Decimal64(P, S)` | `0x1A` | -| `Decimal128(P, S)` | `0x1B` | -| `Decimal256(P, S)` | `0x1C` | -| `UUID` | `0x1D` | -| `Array(T)` | `0x1E` | -| `Tuple(T1, ..., TN)` | `0x1F...` | -| `Tuple(name1 T1, ..., nameN TN)` | `0x20...` | -| `Set` | `0x21` | -| `Interval` | `0x22` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | -| `Nullable(T)` | `0x23` | -| `Function` | `0x24...` | -| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x25......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `LowCardinality(T)` | `0x26` | -| `Map(K, V)` | `0x27` | -| `IPv4` | `0x28` | -| `IPv6` | `0x29` | -| `Variant(T1, ..., TN)` | `0x2A...` | -| `Dynamic(max_types=N)` | `0x2B` | -| `Custom type` (`Ring`, `Polygon`, etc) | `0x2C` | -| `Bool` | `0x2D` | -| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2E......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `Nested(name1 T1, ..., nameN TN)` | `0x2F...` | +| ClickHouse data type | Binary encoding | +|-----------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `Nothing` | `0x00` | +| `UInt8` | `0x01` | +| `UInt16` | `0x02` | +| `UInt32` | `0x03` | +| `UInt64` | `0x04` | +| `UInt128` | `0x05` | +| `UInt256` | `0x06` | +| `Int8` | `0x07` | +| `Int16` | `0x08` | +| `Int32` | `0x09` | +| `Int64` | `0x0A` | +| `Int128` | `0x0B` | +| `Int256` | `0x0C` | +| `Float32` | `0x0D` | +| `Float64` | `0x0E` | +| `Date` | `0x0F` | +| `Date32` | `0x10` | +| `DateTime` | `0x11` | +| `DateTime(time_zone)` | `0x12` | +| `DateTime64(P)` | `0x13` | +| `DateTime64(P, time_zone)` | `0x14` | +| `String` | `0x15` | +| `FixedString(N)` | `0x16` | +| `Enum8` | `0x17...` | +| `Enum16` | `0x18...>` | +| `Decimal32(P, S)` | `0x19` | +| `Decimal64(P, S)` | `0x1A` | +| `Decimal128(P, S)` | `0x1B` | +| `Decimal256(P, S)` | `0x1C` | +| `UUID` | `0x1D` | +| `Array(T)` | `0x1E` | +| `Tuple(T1, ..., TN)` | `0x1F...` | +| `Tuple(name1 T1, ..., nameN TN)` | `0x20...` | +| `Set` | `0x21` | +| `Interval` | `0x22` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | +| `Nullable(T)` | `0x23` | +| `Function` | `0x24...` | +| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x25......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `LowCardinality(T)` | `0x26` | +| `Map(K, V)` | `0x27` | +| `IPv4` | `0x28` | +| `IPv6` | `0x29` | +| `Variant(T1, ..., TN)` | `0x2A...` | +| `Dynamic(max_types=N)` | `0x2B` | +| `Custom type` (`Ring`, `Polygon`, etc) | `0x2C` | +| `Bool` | `0x2D` | +| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2E......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `Nested(name1 T1, ..., nameN TN)` | `0x2F...` | +| `JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP REGEXP skip_path_regexp)` | `0x30.........` | ### Interval kind binary encoding diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 23bb57e5a6c..be75f909684 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -16,14 +16,13 @@ If you want to use JSON type, set `allow_experimental_json_type = 1`. To declare a column of `JSON` type, use the following syntax: ``` sql - JSON(max_dynamic_paths=N, max_dynamic_types=M, some.path TypeName, SKIP path.to.skip, SKIP PREFIX path.prefix.to.skip, SKIP REGEXP 'paths_regexp') + JSON(max_dynamic_paths=N, max_dynamic_types=M, some.path TypeName, SKIP path.to.skip, SKIP REGEXP 'paths_regexp') ``` Where: - `max_dynamic_paths` is an optional parameter indicating how many paths can be stored separately as subcolumns across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all other paths will be stored together in a single structure. Default value of `max_dynamic_paths` is `1000`. - `max_dynamic_types` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a single path column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_dynamic_types` is `32`. - `some.path TypeName` is an optional type hint for particular path in the JSON. Such paths will be always stored as subcolumns with specified type. -- `SKIP path.to.skip` is an optional hint for particular path that should be skipped during JSON parsing. Such paths will never be stored in the JSON column. -- `SKIP PREFIX path.prefix.to.skip` is an optional hint for particular path prefix that should be skipped during JSON parsing. All paths with such prefix will never be stored in the JSON column. +- `SKIP path.to.skip` is an optional hint for particular path that should be skipped during JSON parsing. Such paths will never be stored in the JSON column. If specified path is a nested JSON object, the whole nested object will be skipped. - `SKIP REGEXP 'path_regexp'` is an optional hint with a regular expression that is used to skip paths during JSON parsing. All paths that match this regular expression will never be stored in the JSON column. ## Creating JSON @@ -345,7 +344,7 @@ All text formats (JSONEachRow, TSV, CSV, CustomSeparated, Values, etc) supports Examples: ```sql -SELECT json FROM format(JSONEachRow, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP PREFIX d.e, SKIP REGEXP \'b.*\')', ' +SELECT json FROM format(JSONEachRow, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP d.e, SKIP REGEXP \'b.*\')', ' {"json" : {"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}}} {"json" : {"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}}} {"json" : {"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"}} @@ -367,7 +366,7 @@ SELECT json FROM format(JSONEachRow, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP P For text formats like CSV/TSV/etc `JSON` is parsed from a string containing JSON object ```sql -SELECT json FROM format(TSV, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP PREFIX d.e, SKIP REGEXP \'b.*\')', +SELECT json FROM format(TSV, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP REGEXP \'b.*\')', '{"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}} {"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}} {"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"} @@ -478,5 +477,5 @@ As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and move Before creating `JSON` column and loading data into it, consider the following tips: - Investigate your data and specify as many path hints with types as you can. It will make the storage and the reading much more efficient. -- Think about what paths you will need and what paths you will never need. Specify paths that you won't need in the SKIP section, use SKIP PREFIX and SKIP REGEXP if needed. It will improve the storage. +- Think about what paths you will need and what paths you will never need. Specify paths that you won't need in the SKIP section and SKIP REGEXP if needed. It will improve the storage. - Don't set `max_dynamic_paths` parameter to very high values, it can make the storage and reading less efficient. From 68ed5767d795e7b5792fed839198f53d43581c47 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 15:31:17 +0000 Subject: [PATCH 0283/1722] fix merge problem --- src/Storages/MergeTree/MergeTreeData.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 125b2c8c513..38ca0aed9da 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6160,6 +6160,11 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } +bool MergeTreeData::areAsynchronousInsertsEnabled() const +{ + return getSettings()->async_insert; +} + MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; From 7841a8b401d0ddb3f9bce9e5dc03049a65a1067a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 16:27:38 +0000 Subject: [PATCH 0284/1722] fixes after review --- docs/en/operations/settings/settings.md | 2 +- .../table-functions/azureBlobStorage.md | 13 ++++ docs/en/sql-reference/table-functions/file.md | 8 +-- docs/en/sql-reference/table-functions/hdfs.md | 4 +- docs/en/sql-reference/table-functions/s3.md | 8 +-- docs/en/sql-reference/table-functions/url.md | 13 ++++ .../ObjectStorage/StorageObjectStorage.cpp | 16 +++-- .../StorageObjectStorageCluster.cpp | 4 +- .../StorageObjectStorageSource.cpp | 17 ++--- .../StorageObjectStorageSource.h | 4 +- src/Storages/StorageFile.cpp | 20 ++---- src/Storages/StorageURL.cpp | 11 +-- src/Storages/VirtualColumnUtils.cpp | 64 +++++++++--------- src/Storages/VirtualColumnUtils.h | 7 +- .../03203_hive_style_partitioning.reference | 5 ++ .../03203_hive_style_partitioning.sh | 13 ++++ .../column0=Elizabeth/sample.parquet | Bin 0 -> 1308 bytes .../partitioning/identifier=2070/email.csv | 5 ++ 18 files changed, 114 insertions(+), 100 deletions(-) create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 84912b4574f..fb076e76bdd 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5611,6 +5611,6 @@ Default value: `1GiB`. ## use_hive_partitioning -Allows the usage of Hive-style partitioning in queries. When enabled, ClickHouse interprets and maintains table partitions in a way that is consistent with the Hive partitioning scheme, which is commonly used in Hadoop ecosystems. +When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index f59fedeb3a2..104ac4e26df 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -77,3 +77,16 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam **See Also** - [AzureBlobStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) + +## Hive-style partitioning {#hive-style-partitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_partitioning = 1; +SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet', format='Parquet', structure='Date DateTime64, Country String, Code UInt64') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +``` diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 838a7ab61de..0669609a22a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -208,7 +208,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 ## Hive-style partitioning {#hive-style-partitioning} -When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. **Example** @@ -216,11 +216,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT _specified_column from file('/specified_column=specified_data/file.txt'); -``` - -``` reference -specified_data +SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; ``` ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index fc84c431066..6963d4e4b79 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -101,7 +101,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin ## Hive-style partitioning {#hive-style-partitioning} -When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. **Example** @@ -109,7 +109,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); +SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; ``` ``` reference diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 15074a77475..f3ee83afef4 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -276,7 +276,7 @@ FROM s3( ## Hive-style partitioning {#hive-style-partitioning} -When setting `use_hive_partitioning` is set to 1, ClickHouse can introduce virtual columns due to Hive partitioning style if the path has the specific structure. +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. **Example** @@ -284,11 +284,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT _specified_column from HDFS('hdfs://hdfs1:9000/specified_column=specified_data/file.txt'); -``` - -``` reference -specified_data +SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; ``` ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 3bb7aff53a7..596355e2577 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -55,6 +55,19 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +## Hive-style partitioning {#hive-style-partitioning} + +When setting `use_hive_partitioning` is set to 1, ClickHouse will detect Hive-style partitioning in the path (`/name=value/`) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. + +**Example** + +Use virtual column, created with Hive-style partitioning + +``` sql +SET use_hive_partitioning = 1; +SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +``` + ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 48e9118e321..35cd1492642 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -35,16 +35,19 @@ namespace ErrorCodes std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { + auto query_settings = configuration->getQuerySettings(context); + /// We don't want to throw an exception if there are no files with specified path. + query_settings.throw_on_zero_files_match = false; auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, + query_settings, object_storage, distributed_processing, context, {}, // predicate metadata.getColumns().getAll(), // virtual_columns nullptr, // read_keys - {}, // file_progress_callback - true // override_settings_for_hive_partitioning + {} // file_progress_callback ); if (auto file = file_iterator->next(0)) @@ -82,12 +85,10 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - if (sample_path.empty() && context->getSettings().use_hive_partitioning) + if (sample_path.empty()) sample_path = getPathSample(metadata, context); - else if (!context->getSettings().use_hive_partitioning) - sample_path = ""; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings)); setInMemoryMetadata(metadata); } @@ -224,7 +225,7 @@ private: return; auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, distributed_processing, + configuration, configuration->getQuerySettings(context), object_storage, distributed_processing, context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); } }; @@ -376,6 +377,7 @@ std::unique_ptr StorageObjectStorage::createReadBufferIterat { auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, + configuration->getQuerySettings(context), object_storage, false/* distributed_processing */, context, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 92327b4cde0..a88532e1ea9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -84,8 +84,8 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten const ActionsDAG::Node * predicate, const ContextPtr & local_context) const { auto iterator = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, /* distributed_processing */false, local_context, - predicate, virtual_columns, nullptr, local_context->getFileProgressCallback()); + configuration, configuration->getQuerySettings(local_context), object_storage, /* distributed_processing */false, + local_context, predicate, virtual_columns, nullptr, local_context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2e5416d1ffd..707e7603368 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -99,14 +99,14 @@ std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( std::shared_ptr StorageObjectStorageSource::createFileIterator( ConfigurationPtr configuration, + const StorageObjectStorage::QuerySettings & query_settings, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback, - bool override_settings_for_hive_partitioning) + std::function file_progress_callback) { if (distributed_processing) return std::make_shared( @@ -117,20 +117,16 @@ std::shared_ptr StorageObjectStorageSourc throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside {} name", configuration->getNamespaceType()); - auto settings = configuration->getQuerySettings(local_context); const bool is_archive = configuration->isArchive(); std::unique_ptr iterator; if (configuration->isPathWithGlobs()) { - bool throw_on_zero_files_match = settings.throw_on_zero_files_match; - if (override_settings_for_hive_partitioning) - throw_on_zero_files_match = false; /// Iterate through disclosed globs and make a source for each file iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, - local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, - throw_on_zero_files_match, file_progress_callback); + local_context, is_archive ? nullptr : read_keys, query_settings.list_object_keys_size, + query_settings.throw_on_zero_files_match, file_progress_callback); } else { @@ -149,7 +145,7 @@ std::shared_ptr StorageObjectStorageSourc iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, - settings.ignore_non_existent_file, file_progress_callback); + query_settings.ignore_non_existent_file, file_progress_callback); } if (is_archive) @@ -208,8 +204,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - .hive_partitioning_path = object_info->getPath(), - }); + }, read_from_format_info.columns_description, getContext()); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index a99bb068372..ff6d588b364 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -52,14 +52,14 @@ public: static std::shared_ptr createFileIterator( ConfigurationPtr configuration, + const StorageObjectStorage::QuerySettings & query_settings, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - std::function file_progress_callback = {}, - bool override_settings_for_hive_partitioning = false); + std::function file_progress_callback = {}); static std::string getUniqueStoragePathIdentifier( const Configuration & configuration, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 047631cbc54..42e27a13ca9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1108,9 +1108,9 @@ void StorageFile::setStorageMetadata(CommonArguments args) setInMemoryMetadata(storage_metadata); std::string path_for_virtuals; - if (args.getContext()->getSettingsRef().use_hive_partitioning && !paths.empty()) + if (!paths.empty()) path_for_virtuals = paths[0]; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings.value_or(FormatSettings{}))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings)); } @@ -1452,10 +1452,6 @@ Chunk StorageFileSource::generate() chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - std::string hive_partitioning_path; - if (getContext()->getSettingsRef().use_hive_partitioning) - hive_partitioning_path = current_path; - /// Enrich with virtual columns. VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, requested_virtual_columns, @@ -1463,9 +1459,8 @@ Chunk StorageFileSource::generate() .path = current_path, .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), - .last_modified = current_file_last_modified, - .hive_partitioning_path = hive_partitioning_path, - }); + .last_modified = current_file_last_modified + }, columns_description, getContext()); return chunk; } @@ -1648,17 +1643,10 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui size_t num_streams = max_num_streams; size_t files_to_read = 0; - Strings paths; if (storage->archive_info) - { files_to_read = storage->archive_info->paths_to_archives.size(); - paths = storage->archive_info->paths_to_archives; - } else - { files_to_read = storage->paths.size(); - paths = storage->paths; - } if (max_num_streams > files_to_read) num_streams = files_to_read; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 5da42638b87..f7560fa7910 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -164,10 +164,7 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - std::string uri_for_partitioning; - if (context_->getSettingsRef().use_hive_partitioning) - uri_for_partitioning = getSampleURI(uri, context_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, uri_for_partitioning, format_settings.value_or(FormatSettings{}))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings)); } @@ -426,9 +423,6 @@ Chunk StorageURLSource::generate() size_t chunk_size = 0; if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); - std::string hive_partitioning_path; - if (getContext()->getSettingsRef().use_hive_partitioning) - hive_partitioning_path = curr_uri.getPath(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( @@ -436,8 +430,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - .hive_partitioning_path = hive_partitioning_path, - }); + }, columns_description, getContext()); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 0efa9522ac6..fb5a345f424 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -119,15 +119,25 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -std::map parseHivePartitioningKeysAndValues(const std::string& path) +std::unordered_map parseHivePartitioningKeysAndValues(const std::string& path, const ColumnsDescription & storage_columns) { std::string pattern = "/([^/]+)=([^/]+)"; re2::StringPiece input_piece(path); - std::map key_values; + std::unordered_map key_values; std::string key, value; + std::set used_keys; while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) - key_values["_" + key] = value; + { + if (used_keys.contains(key)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Link to file with enabled hive-style partitioning contains duplicated key {}, only unique keys required", key); + used_keys.insert(key); + + auto col_name = "_" + key; + while (storage_columns.has(col_name)) + col_name = "_" + col_name; + key_values[col_name] = value; + } return key_values; } @@ -148,17 +158,20 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_size", makeNullable(std::make_shared())); add_virtual("_time", makeNullable(std::make_shared())); - auto map = parseHivePartitioningKeysAndValues(path); - for (auto& item : map) + if (context->getSettingsRef().use_hive_partitioning) { + auto map = parseHivePartitioningKeysAndValues(path, storage_columns); auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); - auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw); - if (type == nullptr) - type = std::make_shared(); - if (type->canBeInsideLowCardinality()) - add_virtual(item.first, std::make_shared(type)); - else - add_virtual(item.first, type); + for (auto & item : map) + { + auto type = tryInferDataTypeByEscapingRule(item.second, format_settings, FormatSettings::EscapingRule::Raw); + if (type == nullptr) + type = std::make_shared(); + if (type->canBeInsideLowCardinality()) + add_virtual(item.first, std::make_shared(type)); + else + add_virtual(item.first, type); + } } return desc; @@ -207,8 +220,6 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); - if (!getVirtualNamesForFileLikeStorage().contains(column.name)) - block.insert({column.type->createColumn(), column.type, column.name}); } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); @@ -222,9 +233,12 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values) + VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns, ContextPtr context) { - auto hive_map = parseHivePartitioningKeysAndValues(virtual_values.hive_partitioning_path); + std::unordered_map hive_map; + if (context->getSettingsRef().use_hive_partitioning) + hive_map = parseHivePartitioningKeysAndValues(virtual_values.path, columns); + for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -258,23 +272,9 @@ void addRequestedFileLikeStorageVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } - else if (!hive_map.empty()) + else if (auto it = hive_map.find(virtual_column.getNameInStorage()); it != hive_map.end()) { - bool contains_virtual_column = std::any_of(hive_map.begin(), hive_map.end(), - [&](const auto& pair) - { - return requested_virtual_columns.contains(pair.first); - }); - - if (!contains_virtual_column) - hive_map.clear(); // If we cannot find any virtual column in requested, we don't add any of them to chunk - - auto it = hive_map.find(virtual_column.getNameInStorage()); - if (it != hive_map.end()) - { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst()); - hive_map.erase(it); - } + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), convertFieldToType(Field(it->second), *virtual_column.type))->convertToFullColumnIfConst()); } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index aa7d4c4605b..29ec32ab375 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -7,8 +7,6 @@ #include #include -#include -#include #include @@ -81,14 +79,11 @@ struct VirtualsForFileLikeStorage std::optional size { std::nullopt }; const String * filename { nullptr }; std::optional last_modified { std::nullopt }; - const String & hive_partitioning_path = ""; }; -std::map parseFromPath(const std::string& path); - void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values); + VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns = {}, ContextPtr context = {}); } } diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index be43048dd01..fc6da3a55c1 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -33,6 +33,11 @@ Elizabeth Gordon Elizabeth [1,2,3] 42.42 Array(Int64) LowCardinality(Float64) 101 +2070 +4081 +2070 +2070 +1 1 TESTING THE URL PARTITIONING first last Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 58a74a3ca8f..a5f3e36763d 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -36,6 +36,19 @@ SELECT toTypeName(_array), toTypeName(_float) FROM file('$CURDIR/data_hive/parti SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') WHERE _number = 42; """ +$CLICKHOUSE_LOCAL -n -q """ +set use_hive_partitioning = 1; + +SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; +SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; +""" + +$CLICKHOUSE_LOCAL -n -q """ +set use_hive_partitioning = 1; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "INCORRECT_DATA" + $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv b/tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv new file mode 100644 index 00000000000..936d995cc64 --- /dev/null +++ b/tests/queries/0_stateless/data_hive/partitioning/identifier=2070/email.csv @@ -0,0 +1,5 @@ +_login_email,_identifier,_first_name,_last_name +laura@example.com,2070,Laura,Grey +craig@example.com,4081,Craig,Johnson +mary@example.com,9346,Mary,Jenkins +jamie@example.com,5079,Jamie,Smith From 899c5a64e078820caf2e68cbaf892d5d39e0af06 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 17:14:11 +0000 Subject: [PATCH 0285/1722] some more fixes (docs + storageObjectStorage) --- docs/en/sql-reference/table-functions/azureBlobStorage.md | 2 +- docs/en/sql-reference/table-functions/file.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- docs/en/sql-reference/table-functions/url.md | 2 +- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 7 ++++--- 6 files changed, 9 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 104ac4e26df..6936c807f96 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -88,5 +88,5 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet', format='Parquet', structure='Date DateTime64, Country String, Code UInt64') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from azureBlobStorage(config, storage_account_url='...', container='...', blob_path='http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 0669609a22a..7908a3cb934 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -216,7 +216,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from file('data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 6963d4e4b79..73fdc263d68 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -109,7 +109,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ``` reference diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index f3ee83afef4..1bd9f38517e 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -284,7 +284,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 596355e2577..b4027594e7c 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -65,7 +65,7 @@ Use virtual column, created with Hive-style partitioning ``` sql SET use_hive_partitioning = 1; -SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and code = 42; +SELECT * from url('http://data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` ## Storage Settings {#storage-settings} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 35cd1492642..d2cc73f14d7 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -85,7 +85,7 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - if (sample_path.empty()) + if (sample_path.empty() && context->getSettingsRef().use_hive_partitioning) sample_path = getPathSample(metadata, context); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context, sample_path, format_settings)); @@ -412,8 +412,9 @@ std::string StorageObjectStorage::resolveFormatFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto format_and_schema = detectFormatAndReadSchema(format_settings, *iterator, context).second; sample_path = iterator->getLastFilePath(); - return detectFormatAndReadSchema(format_settings, *iterator, context).second; + return format_and_schema; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( @@ -425,8 +426,8 @@ std::pair StorageObjectStorage::resolveSchemaAn { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - sample_path = iterator->getLastFilePath(); auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); + sample_path = iterator->getLastFilePath(); configuration->format = format; return std::pair(columns, format); } From 7d39535c989e16d818370cbbd9cbea891b21d07a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 17:20:46 +0000 Subject: [PATCH 0286/1722] storageObjectStorage small fix --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8291327992c..ee1169d2c5c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -400,8 +400,9 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto schema = readSchemaFromFormat(configuration->format, format_settings, *iterator, context); sample_path = iterator->getLastFilePath(); - return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); + return schema; } std::string StorageObjectStorage::resolveFormatFromData( From 1bd9a1623f246dbf2a3098a4a022b6764aa3094d Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 15 Jul 2024 19:23:11 +0000 Subject: [PATCH 0287/1722] add throw option in low level --- src/Interpreters/MutationsInterpreter.cpp | 15 +++++++++++++++ src/Interpreters/MutationsInterpreter.h | 1 + src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 15 +++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ .../03161_lightweight_delete_projection.sql | 16 ++++++++++------ 6 files changed, 46 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 480c6736bc5..b61f7f78885 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -344,6 +344,11 @@ bool MutationsInterpreter::Source::hasProjection(const String & name) const return part && part->hasProjection(name); } +bool MutationsInterpreter::Source::hasProjection() const +{ + return part && part->hasProjection(); +} + bool MutationsInterpreter::Source::hasBrokenProjection(const String & name) const { return part && part->hasBrokenProjection(name); @@ -491,6 +496,16 @@ static void validateUpdateColumns( { if (!source.supportsLightweightDelete()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); + + if (const MergeTreeData * merge_tree_data = source.getMergeTreeData(); merge_tree_data != nullptr) + { + if (merge_tree_data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW + && merge_tree_data->hasProjection()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DELETE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + source.getStorage()->getStorageID().getFullTableName()); + } } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6aaa233cda3..b792a33f904 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -126,6 +126,7 @@ public: bool materializeTTLRecalculateOnly() const; bool hasSecondaryIndex(const String & name) const; bool hasProjection(const String & name) const; + bool hasProjection() const; bool hasBrokenProjection(const String & name) const; bool isCompactPart() const; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 991c8ff64af..d302fcb26a7 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -259,6 +259,9 @@ public: /// Return true if there is at least one part containing lightweight deleted mask. virtual bool hasLightweightDeletedMask() const { return false; } + /// Return true if storage has any projection. + virtual bool hasProjection() const { return false; } + /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 38ca0aed9da..78a551591a6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6160,6 +6160,21 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } +bool MergeTreeData::hasProjection() const +{ + auto lock = lockParts(); + for (const auto & part : data_parts_by_info) + { + if (part->getState() == MergeTreeDataPartState::Outdated + || part->getState() == MergeTreeDataPartState::Deleting) + continue; + + if (part->hasProjection()) + return true; + } + return false; +} + bool MergeTreeData::areAsynchronousInsertsEnabled() const { return getSettings()->async_insert; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d880928098b..7076b680521 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -439,6 +439,8 @@ public: bool supportsLightweightDelete() const override; + bool hasProjection() const override; + bool areAsynchronousInsertsEnabled() const override; bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index bfeb0127fa4..16a7468234b 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -13,11 +13,13 @@ SETTINGS min_bytes_for_wide_part = 10485760; INSERT INTO users VALUES (1231, 'John', 33); -DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; + +DELETE FROM users WHERE uid = 1231; SYSTEM FLUSH LOGS; @@ -45,11 +47,13 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); -DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'throw'; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } -DELETE FROM users WHERE uid = 1231 SETTINGS lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; + +DELETE FROM users WHERE uid = 1231; SYSTEM FLUSH LOGS; From 12794601921e2d465b27e665b072267b658b8e4c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 22:09:30 +0000 Subject: [PATCH 0288/1722] fix after review --- .../ObjectStorage/StorageObjectStorage.cpp | 2 +- .../ObjectStorage/StorageObjectStorage.h | 2 +- .../StorageObjectStorageCluster.cpp | 29 ++++++++++++++++++- .../StorageObjectStorageCluster.h | 2 ++ src/Storages/StorageFile.cpp | 5 +--- src/Storages/VirtualColumnUtils.cpp | 6 ++-- src/Storages/VirtualColumnUtils.h | 2 +- .../03203_hive_style_partitioning.reference | 17 +++++++++++ .../03203_hive_style_partitioning.sh | 16 ++++++++++ 9 files changed, 70 insertions(+), 11 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ee1169d2c5c..ca0ced8dcd3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::string StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) +String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { auto query_settings = configuration->getQuerySettings(context); /// We don't want to throw an exception if there are no files with specified path. diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 6ee4ce0c16f..cae0db48f31 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -122,7 +122,7 @@ public: protected: virtual void updateConfiguration(ContextPtr local_context); - std::string getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); + String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); virtual ReadFromFormatInfo prepareReadingFromFormat( const Strings & requested_columns, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index a88532e1ea9..7f6b3338f9b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -1,6 +1,8 @@ #include "Storages/ObjectStorage/StorageObjectStorageCluster.h" #include +#include +#include #include #include #include @@ -19,6 +21,28 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) +{ + auto query_settings = configuration->getQuerySettings(context); + /// We don't want to throw an exception if there are no files with specified path. + query_settings.throw_on_zero_files_match = false; + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + query_settings, + object_storage, + false, // distributed_processing + context, + {}, // predicate + metadata.getColumns().getAll(), // virtual_columns + nullptr, // read_keys + {} // file_progress_callback + ); + + if (auto file = file_iterator->next(0)) + return file->getPath(); + return ""; +} + StorageObjectStorageCluster::StorageObjectStorageCluster( const String & cluster_name_, ConfigurationPtr configuration_, @@ -41,7 +65,10 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setColumns(columns); metadata.setConstraints(constraints_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_)); + if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning) + sample_path = getPathSample(metadata, context_); + + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path, getFormatSettings(context_))); setInMemoryMetadata(metadata); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 108aa109616..0088ff28fc2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -27,6 +27,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension( const ActionsDAG::Node * predicate, const ContextPtr & context) const override; + String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); + private: void updateQueryToSendIfNeeded( ASTPtr & query, diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e2b010bf48f..b43fce370a1 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1109,10 +1109,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - std::string path_for_virtuals; - if (!paths.empty()) - path_for_virtuals = paths[0]; - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), path_for_virtuals, format_settings)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), args.getContext(), paths.empty() ? "" : paths[0], format_settings)); } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index fb5a345f424..352fd0d7a76 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -119,18 +119,18 @@ NameSet getVirtualNamesForFileLikeStorage() return {"_path", "_file", "_size", "_time"}; } -std::unordered_map parseHivePartitioningKeysAndValues(const std::string& path, const ColumnsDescription & storage_columns) +std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { std::string pattern = "/([^/]+)=([^/]+)"; re2::StringPiece input_piece(path); std::unordered_map key_values; std::string key, value; - std::set used_keys; + std::unordered_set used_keys; while (RE2::FindAndConsume(&input_piece, pattern, &key, &value)) { if (used_keys.contains(key)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Link to file with enabled hive-style partitioning contains duplicated key {}, only unique keys required", key); + throw Exception(ErrorCodes::INCORRECT_DATA, "Path '{}' to file with enabled hive-style partitioning contains duplicated partition key {}, only unique keys are allowed", path, key); used_keys.insert(key); auto col_name = "_" + key; diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 29ec32ab375..fef32b149ec 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -51,7 +51,7 @@ NameSet getVirtualNamesForFileLikeStorage(); VirtualColumnsDescription getVirtualsForFileLikeStorage( const ColumnsDescription & storage_columns, const ContextPtr & context, - std::string sample_path = "", + const std::string & sample_path = "", std::optional format_settings_ = std::nullopt); ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index fc6da3a55c1..430a3582f65 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -102,3 +102,20 @@ Jeffery Delgado Elizabeth Clara Cross Elizabeth Elizabeth Gordon Elizabeth OK +TESTING THE S3CLUSTER PARTITIONING +first last Elizabeth +Jorge Frank Elizabeth +Hunter Moreno Elizabeth +Esther Guzman Elizabeth +Dennis Stephens Elizabeth +Nettie Franklin Elizabeth +Stanley Gibson Elizabeth +Eugenia Greer Elizabeth +Jeffery Delgado Elizabeth +Clara Cross Elizabeth +Elizabeth Gordon Elizabeth +Eva Schmidt Elizabeth Schmidt +Samuel Schmidt Elizabeth Schmidt +Eva Schmidt Elizabeth +Samuel Schmidt Elizabeth +Elizabeth Gordon Elizabeth Gordon diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index a5f3e36763d..d2b1f31c85f 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -118,3 +118,19 @@ set use_hive_partitioning = 0; SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -F -q "UNKNOWN_IDENTIFIER" && echo "OK" || echo "FAIL"; + +$CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'" + +$CLICKHOUSE_CLIENT -n -q """ +set use_hive_partitioning = 1; + +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; + +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; + +SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; + +SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +""" From 4305a38e7394d7bbf3c3455c3b52b1dc9b86f3c9 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 22:11:08 +0000 Subject: [PATCH 0289/1722] add include --- src/Storages/VirtualColumnUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 352fd0d7a76..938972cffca 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #include "Functions/FunctionsLogical.h" #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" From dcf14e68afbc741fdbf830fc7d01ba84817c0760 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 22:45:21 +0000 Subject: [PATCH 0290/1722] small fix --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 938972cffca..e84979833ab 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -142,7 +142,7 @@ std::unordered_map parseHivePartitioningKeysAndValues( return key_values; } -VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, std::string path, std::optional format_settings_) +VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns, const ContextPtr & context, const std::string & path, std::optional format_settings_) { VirtualColumnsDescription desc; From 177d006307515e62bbe60082c293c03de4d4cc7c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 15 Jul 2024 23:02:24 +0000 Subject: [PATCH 0291/1722] add errorcodes --- src/Storages/VirtualColumnUtils.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index e84979833ab..24d0b7160b2 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -56,6 +56,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + namespace VirtualColumnUtils { From 82d283357755e3b667074596ec254ca54598ee5d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 Jul 2024 23:29:25 +0000 Subject: [PATCH 0292/1722] clang tidy fix --- src/Interpreters/Context.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..2602afd8b78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include From 8aaf9c1d9824c136bbfc0532b040b2dca7564253 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 11:00:55 +0800 Subject: [PATCH 0293/1722] Fix tests --- src/Client/Connection.cpp | 2 +- src/Core/ProtocolDefines.h | 4 +- src/Core/Settings.h | 2 +- src/Formats/JSONUtils.cpp | 2 +- src/Processors/Formats/IOutputFormat.h | 16 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 4 +- src/Processors/IProcessor.h | 8 +- src/Processors/LimitTransform.h | 4 +- src/Processors/OffsetTransform.h | 4 +- src/Processors/RowsBeforeStepCounter.h | 4 +- src/Processors/Sources/DelayedSource.h | 8 +- src/Processors/Sources/RemoteSource.cpp | 4 - src/Processors/Sources/RemoteSource.h | 9 +- .../AggregatingInOrderTransform.cpp | 2 + .../Transforms/AggregatingInOrderTransform.h | 4 + .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/AggregatingTransform.h | 7 +- .../Transforms/PartialSortingTransform.h | 4 +- src/QueryPipeline/ProfileInfo.cpp | 22 +- src/QueryPipeline/ProfileInfo.h | 4 +- src/QueryPipeline/QueryPipeline.cpp | 15 +- src/Server/TCPHandler.cpp | 2 +- ...74_exact_rows_before_aggregation.reference | 196 +++++++++++++++++- .../03174_exact_rows_before_aggregation.sql | 26 ++- 24 files changed, 276 insertions(+), 81 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 19cd8cc4ee5..d545278220a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1279,7 +1279,7 @@ Progress Connection::receiveProgress() const ProfileInfo Connection::receiveProfileInfo() const { ProfileInfo profile_info; - profile_info.read(*in); + profile_info.read(*in, server_revision); return profile_info; } diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 7e6893c6d85..02d54221ed3 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -81,6 +81,8 @@ static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468; +static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -88,6 +90,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468; /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54468; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54469; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2296a880bd6..842aa54f620 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1200,7 +1200,7 @@ class IColumn; M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \ - M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation_at_least statistic, represents the number of rows read before aggregation", 0) \ + M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation", 0) \ M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 363e9344770..017befe5b0e 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -507,7 +507,7 @@ namespace JSONUtils if (applied_aggregation) { writeFieldDelimiter(out, 2); - writeTitle("rows_before_aggregation_at_least", out, 1, " "); + writeTitle("rows_before_aggregation", out, 1, " "); writeIntText(rows_before_aggregation, out); } if (write_statistics) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 40ac1317618..e9af4ca7cf5 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -39,17 +39,17 @@ public: virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) { } /// Counter to calculate rows_before_limit_at_least in processors pipeline. - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_counter.swap(counter); } - /// Value for rows_before_aggregation_at_least field. - virtual void setRowsBeforeAggregation(size_t /*rows_before_limit*/) { } + /// Value for rows_before_aggregation field. + virtual void setRowsBeforeAggregation(size_t /*rows_before_aggregation*/) { } - /// Counter to calculate rows_before_aggregation_at_least in processors pipeline. - void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } + /// Counter to calculate rows_before_aggregation in processors pipeline. + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); } /// Notify about progress. Method could be called from different threads. /// Passed value are delta, that must be summarized. - virtual void onProgress(const Progress & /*progress*/) {} + virtual void onProgress(const Progress & /*progress*/) { } /// Content-Type to set when sending HTTP response. virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; } @@ -192,8 +192,8 @@ protected: bool need_write_prefix = true; bool need_write_suffix = true; - RowsBeforeLimitCounterPtr rows_before_limit_counter; - RowsBeforeAggregationCounterPtr rows_before_aggregation_counter; + RowsBeforeStepCounterPtr rows_before_limit_counter; + RowsBeforeStepCounterPtr rows_before_aggregation_counter; Statistics statistics; private: diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 2fd0536ed02..b19fcfd4a4a 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -224,9 +224,9 @@ void XMLRowOutputFormat::writeRowsBeforeAggregationAtLeast() { if (statistics.applied_aggregation) { - writeCString("\t", *ostr); + writeCString("\t", *ostr); writeIntText(statistics.rows_before_aggregation, *ostr); - writeCString("\n", *ostr); + writeCString("\n", *ostr); } } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index ccdd6308de5..28050691d9e 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -22,9 +22,7 @@ struct StorageLimits; using StorageLimitsList = std::list; class RowsBeforeStepCounter; -using RowsBeforeLimitCounterPtr = std::shared_ptr; - -using RowsBeforeAggregationCounterPtr = std::shared_ptr; +using RowsBeforeStepCounterPtr = std::shared_ptr; class IProcessor; using ProcessorPtr = std::shared_ptr; @@ -366,11 +364,11 @@ public: /// Set rows_before_limit counter for current processor. /// This counter is used to calculate the number of rows right before any filtration of LimitTransform. - virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {} + virtual void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr /* counter */) { } /// Set rows_before_aggregation counter for current processor. /// This counter is used to calculate the number of rows right before AggregatingTransform. - virtual void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr /* counter */) { } + virtual void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr /* counter */) { } protected: virtual void onCancel() {} diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 515203f6829..45ae5b0ce81 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -30,7 +30,7 @@ private: std::vector sort_column_positions; UInt64 rows_read = 0; /// including the last read block - RowsBeforeLimitCounterPtr rows_before_limit_at_least; + RowsBeforeStepCounterPtr rows_before_limit_at_least; /// State of port's pair. /// Chunks from different port pairs are not mixed for better cache locality. @@ -71,7 +71,7 @@ public: InputPort & getInputPort() { return inputs.front(); } OutputPort & getOutputPort() { return outputs.front(); } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; } }; diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h index 7ef16518540..04486a6c940 100644 --- a/src/Processors/OffsetTransform.h +++ b/src/Processors/OffsetTransform.h @@ -16,7 +16,7 @@ private: UInt64 offset; UInt64 rows_read = 0; /// including the last read block - RowsBeforeLimitCounterPtr rows_before_limit_at_least; + RowsBeforeStepCounterPtr rows_before_limit_at_least; /// State of port's pair. /// Chunks from different port pairs are not mixed for better cache locality. @@ -45,7 +45,7 @@ public: InputPort & getInputPort() { return inputs.front(); } OutputPort & getOutputPort() { return outputs.front(); } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } }; } diff --git a/src/Processors/RowsBeforeStepCounter.h b/src/Processors/RowsBeforeStepCounter.h index d9912bfa076..789731f82bd 100644 --- a/src/Processors/RowsBeforeStepCounter.h +++ b/src/Processors/RowsBeforeStepCounter.h @@ -5,7 +5,7 @@ namespace DB { -/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation_at_least. +/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation. class RowsBeforeStepCounter { public: @@ -31,6 +31,6 @@ private: std::atomic_bool has_applied_step = false; }; -using RowsBeforeLimitCounterPtr = std::shared_ptr; +using RowsBeforeStepCounterPtr = std::shared_ptr; } diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 3138a1ab42a..4ee90e34599 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -30,15 +30,15 @@ public: OutputPort * getTotalsPort() { return totals; } OutputPort * getExtremesPort() { return extremes; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: QueryPlanResourceHolder resources; Creator creator; Processors processors; - RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_aggregation; + RowsBeforeStepCounterPtr rows_before_limit; + RowsBeforeStepCounterPtr rows_before_aggregation; /// Outputs for DelayedSource. OutputPort * main = nullptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 9a6fe239ee6..683db2c1c2c 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -50,8 +50,6 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation { if (info.hasAppliedAggregation()) rows_before_aggregation->add(info.getRowsBeforeAggregation()); - else - manually_add_rows_before_aggregation_counter = true; /// Remote subquery doesn't contain a group by } }); } @@ -171,8 +169,6 @@ std::optional RemoteSource::tryGenerate() { if (manually_add_rows_before_limit_counter) rows_before_limit->add(rows); - if (manually_add_rows_before_aggregation_counter) - rows_before_aggregation->add(rows); query_executor->finish(); return {}; } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index bbc563ec5fe..94dfb46e5f6 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -25,8 +25,8 @@ public: void work() override; String getName() const override { return "Remote"; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); } - void setRowsBeforeAggregationCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_aggregation.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit.swap(counter); } + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -45,8 +45,8 @@ private: bool executor_finished = false; bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; - RowsBeforeLimitCounterPtr rows_before_limit; - RowsBeforeLimitCounterPtr rows_before_aggregation; + RowsBeforeStepCounterPtr rows_before_limit; + RowsBeforeStepCounterPtr rows_before_aggregation; const bool async_read; const bool async_query_sending; @@ -54,7 +54,6 @@ private: int fd = -1; size_t rows = 0; bool manually_add_rows_before_limit_counter = false; - bool manually_add_rows_before_aggregation_counter = false; }; /// Totals source from RemoteQueryExecutor. diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 9ffe15d0f85..7e7bf815832 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -81,6 +81,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk) is_consume_started = true; } + if (rows_before_aggregation) + rows_before_aggregation->add(rows); src_rows += rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 5d50e97f552..1fea9bad131 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -42,6 +42,8 @@ public: void work() override; void consume(Chunk chunk); + + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: void generate(); @@ -83,6 +85,8 @@ private: Chunk current_chunk; Chunk to_push_chunk; + RowsBeforeStepCounterPtr rows_before_aggregation; + LoggerPtr log = getLogger("AggregatingInOrderTransform"); }; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 684de0a3e8c..1a0395f0c9a 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -683,8 +683,8 @@ void AggregatingTransform::consume(Chunk chunk) LOG_TRACE(log, "Aggregating"); is_consume_started = true; } - if (rows_before_aggregation_at_least) - rows_before_aggregation_at_least->add(num_rows); + if (rows_before_aggregation) + rows_before_aggregation->add(num_rows); src_rows += num_rows; src_bytes += chunk.bytes(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 9f9638175f0..3f7a698d0e0 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -169,10 +169,7 @@ public: Status prepare() override; void work() override; Processors expandPipeline() override; - void setRowsBeforeAggregationCounter(RowsBeforeAggregationCounterPtr counter) override - { - rows_before_aggregation_at_least.swap(counter); - } + void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } protected: void consume(Chunk chunk); @@ -216,7 +213,7 @@ private: bool is_consume_started = false; - RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least; + RowsBeforeStepCounterPtr rows_before_aggregation; void initGenerate(); }; diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index abb4b290322..73c490d5b92 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -20,7 +20,7 @@ public: String getName() const override { return "PartialSortingTransform"; } - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { read_rows.swap(counter); } + void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { read_rows.swap(counter); } protected: void transform(Chunk & chunk) override; @@ -29,7 +29,7 @@ private: const SortDescription description; SortDescriptionWithPositions description_with_positions; const UInt64 limit; - RowsBeforeLimitCounterPtr read_rows; + RowsBeforeStepCounterPtr read_rows; Columns sort_description_threshold_columns; diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index 87729b7c90e..69575939edc 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -1,14 +1,14 @@ #include +#include +#include #include #include -#include - namespace DB { -void ProfileInfo::read(ReadBuffer & in) +void ProfileInfo::read(ReadBuffer & in, UInt64 server_revision) { readVarUInt(rows, in); readVarUInt(blocks, in); @@ -16,12 +16,15 @@ void ProfileInfo::read(ReadBuffer & in) readBinary(applied_limit, in); readVarUInt(rows_before_limit, in); readBinary(calculated_rows_before_limit, in); - readBinary(applied_aggregation, in); - readVarUInt(rows_before_aggregation, in); + if (server_revision >= DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION) + { + readBinary(applied_aggregation, in); + readVarUInt(rows_before_aggregation, in); + } } -void ProfileInfo::write(WriteBuffer & out) const +void ProfileInfo::write(WriteBuffer & out, UInt64 client_revision) const { writeVarUInt(rows, out); writeVarUInt(blocks, out); @@ -29,8 +32,11 @@ void ProfileInfo::write(WriteBuffer & out) const writeBinary(hasAppliedLimit(), out); writeVarUInt(getRowsBeforeLimit(), out); writeBinary(calculated_rows_before_limit, out); - writeBinary(hasAppliedAggregation(), out); - writeVarUInt(getRowsBeforeAggregation(), out); + if (client_revision >= DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION) + { + writeBinary(hasAppliedAggregation(), out); + writeVarUInt(getRowsBeforeAggregation(), out); + } } diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index e2467afd6f4..92c83c8c3be 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -40,8 +40,8 @@ struct ProfileInfo /// Binary serialization and deserialization of main fields. /// Writes only main fields i.e. fields that required by internal transmission protocol. - void read(ReadBuffer & in); - void write(WriteBuffer & out) const; + void read(ReadBuffer & in, UInt64 server_revision); + void write(WriteBuffer & out, UInt64 client_revision) const; /// Sets main fields from other object (see methods above). /// If skip_block_size_info if true, then rows, bytes and block fields are ignored. diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 311c8a60531..4ec5cca4dc5 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -140,7 +141,7 @@ static void checkCompleted(Processors & processors) static void initRowsBeforeLimit(IOutputFormat * output_format) { - RowsBeforeLimitCounterPtr rows_before_limit_at_least; + RowsBeforeStepCounterPtr rows_before_limit_at_least; std::vector processors; std::map> limit_candidates; std::unordered_set visited; @@ -280,20 +281,20 @@ static void initRowsBeforeAggregation(std::shared_ptr processors, IO if (!processors->empty()) { - RowsBeforeAggregationCounterPtr rows_before_aggregation_at_least = std::make_shared(); + RowsBeforeStepCounterPtr rows_before_aggregation = std::make_shared(); for (auto processor : *processors) { - if (auto transform = std::dynamic_pointer_cast(processor)) + if (typeid_cast(processor.get()) || typeid_cast(processor.get())) { - transform->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + processor->setRowsBeforeAggregationCounter(rows_before_aggregation); has_aggregation = true; } if (typeid_cast(processor.get()) || typeid_cast(processor.get())) - processor->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + processor->setRowsBeforeAggregationCounter(rows_before_aggregation); } if (has_aggregation) - rows_before_aggregation_at_least->add(0); - output_format->setRowsBeforeAggregationCounter(rows_before_aggregation_at_least); + rows_before_aggregation->add(0); + output_format->setRowsBeforeAggregationCounter(rows_before_aggregation); } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 14a2bceebf1..833f84bb866 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1212,7 +1212,7 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); - info.write(*out); + info.write(*out, client_tcp_protocol_version); out->next(); } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference index 36db9721599..2fbdf325760 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -23,7 +23,7 @@ "rows": 10, - "rows_before_aggregation_at_least": 10 + "rows_before_aggregation": 10 } @@ -68,7 +68,7 @@ 10 - 10 + 10 { "meta": @@ -88,7 +88,7 @@ "rows": 3, - "rows_before_aggregation_at_least": 3 + "rows_before_aggregation": 3 } { "meta": @@ -125,7 +125,7 @@ "rows": 20, - "rows_before_aggregation_at_least": 20 + "rows_before_aggregation": 20 } { "meta": @@ -145,7 +145,7 @@ "rows_before_limit_at_least": 1, - "rows_before_aggregation_at_least": 20 + "rows_before_aggregation": 20 } { "meta": @@ -194,7 +194,7 @@ "rows_before_limit_at_least": 60, - "rows_before_aggregation_at_least": 60 + "rows_before_aggregation": 60 } { "meta": @@ -233,7 +233,7 @@ "rows_before_limit_at_least": 40, - "rows_before_aggregation_at_least": 40 + "rows_before_aggregation": 40 } { "meta": @@ -282,7 +282,7 @@ "rows_before_limit_at_least": 30, - "rows_before_aggregation_at_least": 60 + "rows_before_aggregation": 60 } { "meta": @@ -321,7 +321,7 @@ "rows_before_limit_at_least": 20, - "rows_before_aggregation_at_least": 40 + "rows_before_aggregation": 40 } { "meta": @@ -341,7 +341,7 @@ "rows_before_limit_at_least": 1, - "rows_before_aggregation_at_least": 40 + "rows_before_aggregation": 40 } { "meta": @@ -370,5 +370,179 @@ "rows_before_limit_at_least": 10, - "rows_before_aggregation_at_least": 20 + "rows_before_aggregation": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [4], + [3], + [2], + [5], + [1], + [6], + [7], + [9], + [8] + ], + + "rows": 10, + + "rows_before_limit_at_least": 20, + + "rows_before_aggregation": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9] + ], + + "rows": 10, + + "rows_before_aggregation": 10 +} +{ + "meta": + [ + { + "name": "max(i)", + "type": "Int32" + } + ], + + "data": + [ + [19] + ], + + "rows": 1, + + "rows_before_limit_at_least": 1, + + "rows_before_aggregation": 20 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 60, + + "rows_before_aggregation": 60 +} +{ + "meta": + [ + { + "name": "i", + "type": "Int32" + } + ], + + "data": + [ + [0], + [1], + [2], + [3], + [4], + [5], + [6], + [7], + [8], + [9], + [10], + [11], + [12], + [13], + [14], + [15], + [16], + [17], + [18], + [19], + [20], + [21], + [22], + [23], + [24], + [25], + [26], + [27], + [28], + [29] + ], + + "rows": 30, + + "rows_before_limit_at_least": 30, + + "rows_before_aggregation": 60 } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 31b817e8a65..17e3f3c2cef 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -1,18 +1,15 @@ -- Tags: no-parallel, no-random-merge-tree-settings +set rows_before_aggregation = 1, exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; + drop table if exists test; create table test (i int) engine MergeTree order by tuple(); - insert into test select arrayJoin(range(10000)); -set rows_before_aggregation = 1, exact_rows_before_limit = 1, output_format_write_statistics = 0, max_block_size = 100; - select * from test where i < 10 group by i order by i FORMAT JSONCompact; select * from test where i < 10 group by i order by i FORMAT XML; - select * from test group by i having i in (10, 11, 12) order by i FORMAT JSONCompact; - select * from test where i < 20 group by i order by i FORMAT JSONCompact; select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; @@ -23,8 +20,27 @@ select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i set prefer_localhost_replica = 1; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 group by i order by i FORMAT JSONCompact; + select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; +set prefer_localhost_replica = 0; select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; drop table if exists test; + +create table test (i int) engine MergeTree order by i; + +insert into test select arrayJoin(range(10000)); + +set optimize_aggregation_in_order=1; +select * from test where i < 10 group by i order by i FORMAT JSONCompact; +select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; + +set prefer_localhost_replica = 0; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; + +set prefer_localhost_replica = 1; +select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 30 group by i order by i FORMAT JSONCompact; + +drop table if exists test; From b8b076fd5bedc80cf53781a0190aabb60212a7ef Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 11:37:17 +0800 Subject: [PATCH 0294/1722] Fix style --- src/Processors/Transforms/AggregatingInOrderTransform.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 1fea9bad131..9a1ba513f43 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -42,7 +42,6 @@ public: void work() override; void consume(Chunk chunk); - void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); } private: From da5c92e83ef5b485a119ad5a79098bfc9c216d3e Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 12:00:27 +0800 Subject: [PATCH 0295/1722] FIx bug --- src/QueryPipeline/QueryPipeline.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 4ec5cca4dc5..ff2b5aba13d 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -544,7 +544,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - for (const auto context : resources.interpreter_context) + for (const auto & context : resources.interpreter_context) { if (context->getSettingsRef().rows_before_aggregation) { From b863ef83c351bd87007b4cacc66cff3c5276666f Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 13:00:01 +0800 Subject: [PATCH 0296/1722] Fix compile problem --- src/QueryPipeline/QueryPipeline.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index ff2b5aba13d..844b9e3b039 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -544,7 +545,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - for (const auto & context : resources.interpreter_context) + for (const auto context : resources.interpreter_context) { if (context->getSettingsRef().rows_before_aggregation) { From 403afbc77057125560d88239f6db182056303657 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 16:05:08 +0800 Subject: [PATCH 0297/1722] Add settings rows_before_aggregation to src/Core/SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cdc955b38bc..37c91561b0a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,7 +85,10 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static const std::map settings_changes_history = { - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, + {"24.6", + { + {"rows_before_aggregation", true, true, "Provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation"}, + {"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, From 47e3e999241c8084a2b0389d0e854d705fd6e510 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 16 Jul 2024 10:06:21 +0000 Subject: [PATCH 0298/1722] Avoid calling type->getName --- src/Columns/ColumnDynamic.cpp | 6 +++--- src/Columns/ColumnDynamic.h | 3 ++- src/DataTypes/Serializations/ISerialization.h | 3 +++ src/Formats/JSONExtractTree.cpp | 4 ++-- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index c735238f515..b9ee867001a 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -57,10 +57,10 @@ ColumnDynamic::MutablePtr ColumnDynamic::create(MutableColumnPtr variant_column, return create(std::move(variant_column), variant_info, max_dynamic_types_, statistics_); } -bool ColumnDynamic::addNewVariant(const DB::DataTypePtr & new_variant) +bool ColumnDynamic::addNewVariant(const DataTypePtr & new_variant, const String & variant_name) { /// Check if we already have such variant. - if (variant_info.variant_name_to_discriminator.contains(new_variant->getName())) + if (variant_info.variant_name_to_discriminator.contains(variant_name)) return true; /// Check if we reached maximum number of variants. @@ -75,7 +75,7 @@ bool ColumnDynamic::addNewVariant(const DB::DataTypePtr & new_variant) } /// If we have (max_dynamic_types - 1) number of variants and don't have String variant, we can add only String variant. - if (variant_info.variant_names.size() == max_dynamic_types - 1 && new_variant->getName() != "String" && !variant_info.variant_name_to_discriminator.contains("String")) + if (variant_info.variant_names.size() == max_dynamic_types - 1 && variant_name != "String" && !variant_info.variant_name_to_discriminator.contains("String")) return false; const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 630eb8ad7a8..0edd90b29db 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -334,7 +334,8 @@ public: const ColumnVariant & getVariantColumn() const { return assert_cast(*variant_column); } ColumnVariant & getVariantColumn() { return assert_cast(*variant_column); } - bool addNewVariant(const DataTypePtr & new_variant); + bool addNewVariant(const DataTypePtr & new_variant) { return addNewVariant(new_variant, new_variant->getName()); } + bool addNewVariant(const DataTypePtr & new_variant, const String & variant_name); void addStringVariant(); bool hasDynamicStructure() const override { return true; } diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 22bbdb25c43..5d6ac2707ed 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -447,6 +447,9 @@ public: static size_t getArrayLevel(const SubstreamPath & path); static bool hasSubcolumnForPath(const SubstreamPath & path, size_t prefix_len); static SubstreamData createFromPath(const SubstreamPath & path, size_t prefix_len); + + /// Returns true if subcolumn doesn't actually stores any data in column and doen'st require a separate stream + /// for writing/reading data. For example, it's a null-map subcolumn of Variant type (it's always constructed from discriminators);. static bool isFictitiousSubcolumn(const SubstreamPath & path, size_t prefix_len); protected: diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 2eaddddb567..b19cf61e207 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1384,9 +1384,9 @@ public: jsonElementToString(element, format_settings)); } - if (column_dynamic.addNewVariant(element_type)) + auto element_type_name = element_type->getName(); + if (column_dynamic.addNewVariant(element_type, element_type_name)) { - auto element_type_name = element_type->getName(); auto it = json_extract_nodes_cache.find(element_type_name); if (it == json_extract_nodes_cache.end()) it = json_extract_nodes_cache.emplace(element_type_name, buildJSONExtractTree(element_type, "Dynamic inference")).first; From 808acea1b6677c1b6bdbc28ddeb01c28e95e43d8 Mon Sep 17 00:00:00 2001 From: morning-color Date: Tue, 16 Jul 2024 19:45:39 +0800 Subject: [PATCH 0299/1722] Fix compile problem --- src/Core/SettingsChangesHistory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b9b72209103..922946b192e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,7 +57,8 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, + {"24.7", {{"rows_before_aggregation", true, true, "Provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation"}, + {"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, From 9690a5a334b4991eaa9dfa58ce804c91bbff4385 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 Jul 2024 13:37:59 +0000 Subject: [PATCH 0300/1722] fix --- tests/ci/libfuzzer_test_check.py | 2 +- {utils/libfuzzer => tests/fuzz}/runner.py | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename {utils/libfuzzer => tests/fuzz}/runner.py (100%) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index d9e33229932..8f19dd7d023 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -75,7 +75,7 @@ def get_run_command( f"--volume={result_path}:/test_output " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image} " - "python3 ./utils/runner.py" + "python3 /usr/share/clickhouse-test/fuzz/runner.py" ) diff --git a/utils/libfuzzer/runner.py b/tests/fuzz/runner.py similarity index 100% rename from utils/libfuzzer/runner.py rename to tests/fuzz/runner.py From c9e02eee7a1d7b2e7aa85bfc87d6a54a3bcaedfa Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 16 Jul 2024 13:48:52 +0000 Subject: [PATCH 0301/1722] fix after review --- docs/en/sql-reference/table-functions/hdfs.md | 4 ---- .../StorageObjectStorageSource.cpp | 2 +- .../ObjectStorageQueueSource.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageFileCluster.cpp | 2 +- src/Storages/StorageURL.cpp | 19 +------------------ src/Storages/StorageURL.h | 18 ++++++++++++++++++ src/Storages/StorageURLCluster.cpp | 3 +-- src/Storages/VirtualColumnUtils.cpp | 2 +- src/Storages/VirtualColumnUtils.h | 2 +- 10 files changed, 26 insertions(+), 30 deletions(-) diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 73fdc263d68..60c2fd40e6a 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -112,10 +112,6 @@ SET use_hive_partitioning = 1; SELECT * from HDFS('hdfs://hdfs1:9000/data/path/date=*/country=*/code=*/*.parquet') where _date > '2020-01-01' and _country = 'Netherlands' and _code = 42; ``` -``` reference -specified_data -``` - ## Storage Settings {#storage-settings} - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 93f8eaacbc0..d29e33444b0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -204,7 +204,7 @@ Chunk StorageObjectStorageSource::generate() .size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes, .filename = &filename, .last_modified = object_info->metadata->last_modified, - }, read_from_format_info.columns_description, getContext()); + }, getContext(), read_from_format_info.columns_description); const auto & partition_columns = configuration->getPartitionColumns(); if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 4d921003e04..2634a7b2f1e 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -524,7 +524,7 @@ Chunk ObjectStorageQueueSource::generateImpl() { .path = path, .size = reader.getObjectInfo()->metadata->size_bytes - }); + }, getContext(), read_from_format_info.columns_description); return chunk; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index b43fce370a1..5cbc2b38887 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1459,7 +1459,7 @@ Chunk StorageFileSource::generate() .size = current_file_size, .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), .last_modified = current_file_last_modified - }, columns_description, getContext()); + }, getContext(), columns_description); return chunk; } diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index f7684182e79..82ae0b761ae 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -61,7 +61,7 @@ StorageFileCluster::StorageFileCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, paths.empty() ? "" : paths[0])); } void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 87911230819..6e7788cfc1d 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -39,7 +39,6 @@ #include #include -#include #include #include @@ -92,27 +91,11 @@ static const std::vector> optional_regex_keys = { std::make_shared(R"(headers.header\[[0-9]*\].value)"), }; -static bool urlWithGlobs(const String & uri) -{ - return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; -} - static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) { return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout); } -String getSampleURI(String uri, ContextPtr context) -{ - if (urlWithGlobs(uri)) - { - auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); - if (!uris.empty()) - return uris[0]; - } - return uri; -} - IStorageURLBase::IStorageURLBase( const String & uri_, const ContextPtr & context_, @@ -433,7 +416,7 @@ Chunk StorageURLSource::generate() { .path = curr_uri.getPath(), .size = current_file_size, - }, columns_description, getContext()); + }, getContext(), columns_description); return chunk; } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fa7cc6eeeef..a874ca9147c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include namespace DB @@ -267,6 +269,22 @@ private: bool cancelled = false; }; +static bool urlWithGlobs(const String & uri) +{ + return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; +} + +inline String getSampleURI(String uri, ContextPtr context) +{ + if (urlWithGlobs(uri)) + { + auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + if (!uris.empty()) + return uris[0]; + } + return uri; +} + class StorageURL : public IStorageURLBase { public: diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 664f170c17e..e80f4ebcd06 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -76,7 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context)); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context), getFormatSettings(context))); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 24d0b7160b2..31cee485dde 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -239,7 +239,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns, ContextPtr context) + VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns) { std::unordered_map hive_map; if (context->getSettingsRef().use_hive_partitioning) diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fef32b149ec..1bd74189559 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -83,7 +83,7 @@ struct VirtualsForFileLikeStorage void addRequestedFileLikeStorageVirtualsToChunk( Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, - VirtualsForFileLikeStorage virtual_values, ColumnsDescription columns = {}, ContextPtr context = {}); + VirtualsForFileLikeStorage virtual_values, ContextPtr context, const ColumnsDescription & columns); } } From e7e62b358360083eda6d2ec983fb5a1b733d1eba Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 Jul 2024 14:17:51 +0000 Subject: [PATCH 0302/1722] fix style --- tests/fuzz/runner.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index bbe648dbbc2..0862ea29e42 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -11,7 +11,7 @@ FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") def run_fuzzer(fuzzer: str): - logging.info(f"Running fuzzer {fuzzer}...") + logging.info("Running fuzzer %s...", fuzzer) corpus_dir = f"{fuzzer}.in" with Path(corpus_dir) as path: @@ -29,28 +29,28 @@ def run_fuzzer(fuzzer: str): if parser.has_section("asan"): os.environ["ASAN_OPTIONS"] = ( - f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}" + f"{os.environ['ASAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['asan'].items())}" ) if parser.has_section("msan"): os.environ["MSAN_OPTIONS"] = ( - f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}" + f"{os.environ['MSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['msan'].items())}" ) if parser.has_section("ubsan"): os.environ["UBSAN_OPTIONS"] = ( - f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}" + f"{os.environ['UBSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['ubsan'].items())}" ) if parser.has_section("libfuzzer"): custom_libfuzzer_options = " ".join( - "-%s=%s" % (key, value) + f"-{key}={value}" for key, value in parser["libfuzzer"].items() ) if parser.has_section("fuzzer_arguments"): fuzzer_arguments = " ".join( - ("%s" % key) if value == "" else ("%s=%s" % (key, value)) + (f"{key}") if value == "" else (f"{key}={value}") for key, value in parser["fuzzer_arguments"].items() ) @@ -65,7 +65,7 @@ def run_fuzzer(fuzzer: str): cmd_line += " < /dev/null" - logging.info(f"...will execute: {cmd_line}") + logging.info("...will execute: %s", cmd_line) subprocess.check_call(cmd_line, shell=True) From c974430e68bff97986379410e9a94c1ea641d1bd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 Jul 2024 15:01:43 +0000 Subject: [PATCH 0303/1722] fix --- tests/fuzz/runner.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 0862ea29e42..047a2245bfa 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -29,17 +29,17 @@ def run_fuzzer(fuzzer: str): if parser.has_section("asan"): os.environ["ASAN_OPTIONS"] = ( - f"{os.environ['ASAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['asan'].items())}" + f"{os.environ['ASAN_OPTIONS']}:{':'.join(f'{key}={value}' for key, value in parser['asan'].items())}" ) if parser.has_section("msan"): os.environ["MSAN_OPTIONS"] = ( - f"{os.environ['MSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['msan'].items())}" + f"{os.environ['MSAN_OPTIONS']}:{':'.join(f'{key}={value}' for key, value in parser['msan'].items())}" ) if parser.has_section("ubsan"): os.environ["UBSAN_OPTIONS"] = ( - f"{os.environ['UBSAN_OPTIONS']}:{':'.join(f"{key}={value}" for key, value in parser['ubsan'].items())}" + f"{os.environ['UBSAN_OPTIONS']}:{':'.join(f'{key}={value}' for key, value in parser['ubsan'].items())}" ) if parser.has_section("libfuzzer"): From 8660aec5d79f7a16ab3bcac2aaab291e4bcf0c2d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Jul 2024 15:16:11 +0000 Subject: [PATCH 0304/1722] Automatic style fix --- tests/fuzz/runner.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 047a2245bfa..44259228f60 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -44,8 +44,7 @@ def run_fuzzer(fuzzer: str): if parser.has_section("libfuzzer"): custom_libfuzzer_options = " ".join( - f"-{key}={value}" - for key, value in parser["libfuzzer"].items() + f"-{key}={value}" for key, value in parser["libfuzzer"].items() ) if parser.has_section("fuzzer_arguments"): From b6672b9952caeff523b2836a710dd3be3d6ed4e8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 16 Jul 2024 15:20:01 +0000 Subject: [PATCH 0305/1722] add rebuild for compact part --- src/Core/SettingsEnums.cpp | 3 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/MutationsInterpreter.cpp | 5 -- src/Interpreters/MutationsInterpreter.h | 1 - .../MergeTree/MergeMutateSelectedEntry.h | 1 - src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 24 ++++++++-- src/Storages/StorageMergeTree.cpp | 1 - ...61_lightweight_delete_projection.reference | 5 ++ .../03161_lightweight_delete_projection.sql | 46 +++++++++++++++++-- 10 files changed, 69 insertions(+), 20 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 82e7d6db410..6c000d83254 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -175,7 +175,8 @@ IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGU IMPLEMENT_SETTING_ENUM(LightweightMutationProjectionMode, ErrorCodes::BAD_ARGUMENTS, {{"throw", LightweightMutationProjectionMode::THROW}, - {"drop", LightweightMutationProjectionMode::DROP}}) + {"drop", LightweightMutationProjectionMode::DROP}, + {"rebuild", LightweightMutationProjectionMode::REBUILD}}) IMPLEMENT_SETTING_AUTO_ENUM(LocalFSReadMethod, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index f6d9593ca56..0281176417a 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -311,6 +311,7 @@ enum class LightweightMutationProjectionMode : uint8_t { THROW, DROP, + REBUILD, }; DECLARE_SETTING_ENUM(LightweightMutationProjectionMode) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index b61f7f78885..db4ea9c0754 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -344,11 +344,6 @@ bool MutationsInterpreter::Source::hasProjection(const String & name) const return part && part->hasProjection(name); } -bool MutationsInterpreter::Source::hasProjection() const -{ - return part && part->hasProjection(); -} - bool MutationsInterpreter::Source::hasBrokenProjection(const String & name) const { return part && part->hasBrokenProjection(name); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index b792a33f904..6aaa233cda3 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -126,7 +126,6 @@ public: bool materializeTTLRecalculateOnly() const; bool hasSecondaryIndex(const String & name) const; bool hasProjection(const String & name) const; - bool hasProjection() const; bool hasBrokenProjection(const String & name) const; bool isCompactPart() const; diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index e7efe00741c..c420cbca12b 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -3,7 +3,6 @@ #include #include - namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c84ca9956fc..74e7a7f43bc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -215,7 +215,7 @@ struct Settings; M(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ /** Projection settings. */ \ M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts.", 0) \ + M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts, or rebuild the projections.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8790ce6628e..092a6d0d6ed 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -662,7 +662,7 @@ static NameSet collectFilesToSkip( const std::set & projections_to_recalc, const std::set & stats_to_recalc, const StorageMetadataPtr & metadata_snapshot, - bool lightweight_delete_mode) + bool skip_all_projections) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -686,7 +686,7 @@ static NameSet collectFilesToSkip( } } - if (lightweight_delete_mode) + if (skip_all_projections) { for (const auto & projection : metadata_snapshot->getProjections()) files_to_skip.insert(projection.getDirectoryName()); @@ -2211,6 +2211,8 @@ bool MutateTask::prepare() ctx->stage_progress = std::make_unique(1.0); + bool lightweight_delete_mode = false; + if (!ctx->for_interpreter.empty()) { /// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the @@ -2228,6 +2230,16 @@ bool MutateTask::prepare() ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); + + lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + /// If under the condition of lightweight delete mode with rebuild option, add projections again here as we can only know + /// the condition as early as from here. + if (lightweight_delete_mode + && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::REBUILD) + { + for (const auto & projection : ctx->metadata_snapshot->getProjections()) + ctx->materialized_projections.insert(projection.name); + } } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); @@ -2269,7 +2281,6 @@ bool MutateTask::prepare() if (ctx->mutating_pipeline_builder.initialized()) ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - bool lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); if (ctx->data->getSettings()->exclude_deleted_rows_for_part_size_in_merge && lightweight_delete_mode) { /// This mutation contains lightweight delete and we need to count the deleted rows, @@ -2307,7 +2318,10 @@ bool MutateTask::prepare() ctx->context, ctx->materialized_indices); - if (!lightweight_delete_mode) + bool lightweight_delete_projection_drop = lightweight_delete_mode + && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP; + /// Under lightweight delete mode, if option is drop, projections_to_recalc should be empty. + if (!lightweight_delete_projection_drop) { ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( ctx->source_part, @@ -2326,7 +2340,7 @@ bool MutateTask::prepare() ctx->projections_to_recalc, ctx->stats_to_recalc, ctx->metadata_snapshot, - lightweight_delete_mode); + lightweight_delete_projection_drop); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d8c61da2a98..40b3a12297b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1285,7 +1285,6 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; - for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { /// Do not squash mutations from different transactions to be able to commit/rollback them independently. diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index e69de29bb2d..bc7e1faecff 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -0,0 +1,5 @@ +8888 Alice 50 +p1 +p2 +p1 +p2 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 16a7468234b..b63341f5371 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -13,25 +13,43 @@ SETTINGS min_bytes_for_wide_part = 10485760; INSERT INTO users VALUES (1231, 'John', 33); +-- testing throw default mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +-- testing drop mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; +SELECT * FROM users ORDER BY uid; + SYSTEM FLUSH LOGS; -- expecting no projection SELECT - name, - `table` + name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +-- testing rebuild mode +INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); + +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; + +DELETE FROM users WHERE uid = 6666; + SELECT * FROM users ORDER BY uid; +SYSTEM FLUSH LOGS; + +-- expecting projection p1, p2 in 2 parts +SELECT + name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + DROP TABLE users; @@ -47,23 +65,41 @@ SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); +-- testing throw default mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +-- testing drop mode ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; +SELECT * FROM users ORDER BY uid; + SYSTEM FLUSH LOGS; -- expecting no projection SELECT - name, - `table` + name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); -SELECT * FROM users ORDER BY uid; +-- -- testing rebuild mode +-- INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); + +-- ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; + +-- DELETE FROM users WHERE uid = 6666; + +-- SELECT * FROM users ORDER BY uid; + +-- SYSTEM FLUSH LOGS; + +-- -- expecting projection p1, p2 in 2 parts +-- SELECT +-- name +-- FROM system.projection_parts +-- WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); DROP TABLE users; \ No newline at end of file From c3507979cfc0359ab38762525ab0306904a387b8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 16 Jul 2024 15:41:54 +0000 Subject: [PATCH 0306/1722] fix --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 092a6d0d6ed..489c8863a8a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2234,7 +2234,7 @@ bool MutateTask::prepare() lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); /// If under the condition of lightweight delete mode with rebuild option, add projections again here as we can only know /// the condition as early as from here. - if (lightweight_delete_mode + if (lightweight_delete_mode && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::REBUILD) { for (const auto & projection : ctx->metadata_snapshot->getProjections()) From 3ae4211b3af575cf8d7186a4cc915f9ecb6b4182 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 17:59:57 +0200 Subject: [PATCH 0307/1722] fix tests --- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index d2b1f31c85f..0f687d532b0 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -39,8 +39,8 @@ SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01 $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; -SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identitier=*/email.csv') LIMIT 2; +SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; +SELECT __identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; """ $CLICKHOUSE_LOCAL -n -q """ From d91cb40bbdbd18a2bef811002033d0c99fe693d3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 16 Jul 2024 16:15:24 +0000 Subject: [PATCH 0308/1722] fix include and remove unused getFormatSettings --- src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp | 2 +- src/Storages/StorageFile.cpp | 1 - src/Storages/StorageURLCluster.cpp | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 7f6b3338f9b..c214665f7e0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -68,7 +68,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( if (sample_path.empty() && context_->getSettingsRef().use_hive_partitioning) sample_path = getPathSample(metadata, context_); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path, getFormatSettings(context_))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns(), context_, sample_path)); setInMemoryMetadata(metadata); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5cbc2b38887..ed05f57b418 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,7 +52,6 @@ #include #include #include -#include "Formats/FormatSettings.h" #include #include diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index e80f4ebcd06..1522a18a083 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -75,7 +75,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context), getFormatSettings(context))); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context, getSampleURI(uri, context))); } void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) From d11d1a42bc7101993e5c85f1c1c3298e6334dbf9 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 16 Jul 2024 16:49:33 +0000 Subject: [PATCH 0309/1722] fix for storageURL functions --- src/Storages/StorageURL.cpp | 16 ++++++++++++++++ src/Storages/StorageURL.h | 19 +++---------------- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6e7788cfc1d..1d1deebf9f5 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -91,6 +91,22 @@ static const std::vector> optional_regex_keys = { std::make_shared(R"(headers.header\[[0-9]*\].value)"), }; +bool urlWithGlobs(const String & uri) +{ + return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; +} + +String getSampleURI(String uri, ContextPtr context) +{ + if (urlWithGlobs(uri)) + { + auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + if (!uris.empty()) + return uris[0]; + } + return uri; +} + static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) { return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index a874ca9147c..cd48ecb767b 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -143,6 +143,9 @@ private: virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0; }; +bool urlWithGlobs(const String & uri); + +String getSampleURI(String uri, ContextPtr context); class StorageURLSource : public SourceWithKeyCondition, WithContext { @@ -269,22 +272,6 @@ private: bool cancelled = false; }; -static bool urlWithGlobs(const String & uri) -{ - return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; -} - -inline String getSampleURI(String uri, ContextPtr context) -{ - if (urlWithGlobs(uri)) - { - auto uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); - if (!uris.empty()) - return uris[0]; - } - return uri; -} - class StorageURL : public IStorageURLBase { public: From 14dcb97e353fb4739c7f7d37b9c3c11c9ad40923 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 19:09:18 +0200 Subject: [PATCH 0310/1722] Update src/Storages/StorageURL.h Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Storages/StorageURL.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index cd48ecb767b..1f3d63b4c85 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -12,8 +12,6 @@ #include #include #include -#include -#include namespace DB From 771b39fa2179a9a548580c41859bbecf0165000d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 19:44:10 +0200 Subject: [PATCH 0311/1722] Update StorageURLCluster.cpp --- src/Storages/StorageURLCluster.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 1522a18a083..7c7a299c64e 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include From 9d332911fb0f4b25bbdb67a9c10c5f3b42db4ea6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 20:10:19 +0200 Subject: [PATCH 0312/1722] Update StorageURL.cpp --- src/Storages/StorageURL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1d1deebf9f5..4cf191f7e8a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -39,6 +39,7 @@ #include #include +#include #include #include From 094a1aa970e371a71446ea93bbd4292864257854 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 16 Jul 2024 19:07:15 +0000 Subject: [PATCH 0313/1722] Fix build and JSON getName() method --- src/DataTypes/DataTypeObject.cpp | 4 +++- src/DataTypes/DataTypeObjectDeprecated.cpp | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 02065bfcc97..0e7b46ffbd2 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -15,6 +15,8 @@ #include #include +#include + #if USE_SIMDJSON #include #endif @@ -157,7 +159,7 @@ String DataTypeObject::doGetName() const for (const auto & path : sorted_typed_paths) { write_separator(); - out << path << " " << typed_paths.at(path)->getName(); + out << backQuoteIfNeed(path) << " " << typed_paths.at(path)->getName(); } std::vector sorted_skip_paths; diff --git a/src/DataTypes/DataTypeObjectDeprecated.cpp b/src/DataTypes/DataTypeObjectDeprecated.cpp index 8b2b5e72f6c..e3c1a458eda 100644 --- a/src/DataTypes/DataTypeObjectDeprecated.cpp +++ b/src/DataTypes/DataTypeObjectDeprecated.cpp @@ -8,6 +8,7 @@ #include #include +#include namespace DB { From 949e69c0573354417e21bd83d446e1ea085db04d Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 16 Jul 2024 21:58:48 +0200 Subject: [PATCH 0314/1722] add documentation for getSubcolumn and getTypeSerializationStreams --- .../functions/other-functions.md | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 260457b3be1..40f1b82562d 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -4055,3 +4055,94 @@ Result: │ 32 │ └─────────────────────────────┘ ``` + +## getSubcolumn + +Takes a table expression or identifier and constant string with the name of the sub-column, and returns the requested sub-column extracted from the expression. + +**Syntax** + +```sql +getSubcolumn(col_name, subcol_name) +``` + +**Arguments** + +- `col_name` — Table expression or identifier. [Expression](../syntax.md/#expressions), [Identifier](../syntax.md/#identifiers). +- `subcol_name` — The name of the sub-column. [String](../data-types/string.md). + +**Returned value** + +- Returns the extracted sub-colum. + +**Example** + +Query: + +```sql +CREATE TABLE t_arr (arr Array(Tuple(subcolumn1 UInt32, subcolumn2 String))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_arr VALUES ([(1, 'Hello'), (2, 'World')]), ([(3, 'This'), (4, 'is'), (5, 'subcolumn')]); +SELECT getSubcolumn(arr, 'subcolumn1'), getSubcolumn(arr, 'subcolumn2') FROM t_arr; +``` + +Result: + +```response + ┌─getSubcolumn(arr, 'subcolumn1')─┬─getSubcolumn(arr, 'subcolumn2')─┠+1. │ [1,2] │ ['Hello','World'] │ +2. │ [3,4,5] │ ['This','is','subcolumn'] │ + └─────────────────────────────────┴─────────────────────────────────┘ +``` + +## getTypeSerializationStreams + +Enumerates stream paths of a data type. + +:::note +This function is intended for use by developers. +::: + +**Syntax** + +```sql +getTypeSerializationStreams(col) +``` + +**Arguments** + +- `col` — Column or string representation of a data-type from which the data type will be detected. + +**Returned value** + +- Returns an array with all the serialization sub-stream paths.[Array](../data-types/array.md)([String](../data-types/string.md)). + +**Examples** + +Query: + +```sql +SELECT getTypeSerializationStreams(tuple('a', 1, 'b', 2)); +``` + +Result: + +```response + ┌─getTypeSerializationStreams(('a', 1, 'b', 2))─────────────────────────────────────────────────────────────────────────┠+1. │ ['{TupleElement(1), Regular}','{TupleElement(2), Regular}','{TupleElement(3), Regular}','{TupleElement(4), Regular}'] │ + └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Query: + +```sql +SELECT getTypeSerializationStreams('Map(String, Int64)'); +``` + +Result: + +```response + ┌─getTypeSerializationStreams('Map(String, Int64)')────────────────────────────────────────────────────────────────┠+1. │ ['{ArraySizes}','{ArrayElements, TupleElement(keys), Regular}','{ArrayElements, TupleElement(values), Regular}'] │ + └──────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + From 122673592b21a2a0e60d1cedc9f9337b471ebcb8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 17 Jul 2024 02:37:48 +0000 Subject: [PATCH 0315/1722] add rebuild for wide part --- src/Storages/MergeTree/MutateTask.cpp | 5 ++++ src/Storages/StorageInMemoryMetadata.cpp | 12 +++++++-- ...61_lightweight_delete_projection.reference | 1 + .../03161_lightweight_delete_projection.sql | 25 ++++++++++--------- 4 files changed, 29 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 489c8863a8a..fe14c5a4f05 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2238,7 +2238,12 @@ bool MutateTask::prepare() && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::REBUILD) { for (const auto & projection : ctx->metadata_snapshot->getProjections()) + { + if (!ctx->source_part->hasProjection(projection.name)) + continue; + ctx->materialized_projections.insert(projection.name); + } } } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 2226de3e64f..4a655cac566 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -334,10 +335,17 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( NameSet required_ttl_columns; NameSet updated_ttl_columns; - auto add_dependent_columns = [&updated_columns](const Names & required_columns, auto & to_set) + auto add_dependent_columns = [&updated_columns](const Names & required_columns, auto & to_set, bool is_projection = false) { for (const auto & dependency : required_columns) { + /// useful in the case of lightweight delete with wide part and option of rebuild projection + if (is_projection && updated_columns.contains(RowExistsColumn::name)) + { + to_set.insert(required_columns.begin(), required_columns.end()); + return true; + } + if (updated_columns.contains(dependency)) { to_set.insert(required_columns.begin(), required_columns.end()); @@ -357,7 +365,7 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( for (const auto & projection : getProjections()) { if (has_dependency(projection.name, ColumnDependency::PROJECTION)) - add_dependent_columns(projection.getRequiredColumns(), projections_columns); + add_dependent_columns(projection.getRequiredColumns(), projections_columns, true); } auto add_for_rows_ttl = [&](const auto & expression, auto & to_set) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index bc7e1faecff..3401eaf6162 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,5 +1,6 @@ 8888 Alice 50 p1 p2 +8888 Alice 50 p1 p2 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index b63341f5371..2c60d83d74d 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -44,7 +44,7 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting projection p1, p2 in 2 parts +-- expecting projection p1, p2 SELECT name FROM system.projection_parts @@ -85,21 +85,22 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- -- testing rebuild mode --- INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +-- testing rebuild mode +INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); --- ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; --- DELETE FROM users WHERE uid = 6666; +DELETE FROM users WHERE uid = 6666; --- SELECT * FROM users ORDER BY uid; +SELECT * FROM users ORDER BY uid; --- SYSTEM FLUSH LOGS; +SYSTEM FLUSH LOGS; + +-- expecting projection p1, p2 +SELECT + name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- -- expecting projection p1, p2 in 2 parts --- SELECT --- name --- FROM system.projection_parts --- WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); DROP TABLE users; \ No newline at end of file From 0bfe345a460766e8113934801609230410db1836 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 17 Jul 2024 09:24:33 +0000 Subject: [PATCH 0316/1722] Debug code --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 7 ++++--- src/Storages/ObjectStorage/Local/Configuration.h | 5 +---- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 7 +++++++ 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 6c21de96371..dfbc1a77d3c 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -50,10 +50,11 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = - [=] (bool /* restricted_seek */, const StoredObject & object) - -> std::unique_ptr + auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr { + LOG_DEBUG(&Poco::Logger::get("Read"), "Remote Path: {}", object.remote_path); + auto from_file_storage = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index 11165fcd574..ba4de63ac47 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -38,10 +38,7 @@ public: ConfigurationPtr clone() override { return std::make_shared(*this); } - ObjectStoragePtr createObjectStorage(ContextPtr, bool) override - { - return std::make_shared(fs::path{path}.parent_path()); - } + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared("/"); } void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 54ddaec0140..715f93c7f7e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -388,7 +388,12 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); + LOG_DEBUG(&Poco::Logger::get("Info relative path"), "Info: {}", object_info->relative_path); read_buf = createReadBuffer(*object_info, object_storage, context_, log); + auto new_read_buf = createReadBuffer(*object_info, object_storage, context_, log); + std::string answer(1000, ' '); + size_t read_bytes = new_read_buf->read(answer.data(), 1000); + LOG_DEBUG(&Poco::Logger::get("Read buffer"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); } auto input_format = FormatFactory::instance().getInput( @@ -471,6 +476,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + LOG_DEBUG(&Poco::Logger::get("Read"), "Path: {}, object size: {}", object_info.getPath(), object_size); + auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); From f29700e04d3ca4d455908ed354472945597da4f5 Mon Sep 17 00:00:00 2001 From: morning-color Date: Wed, 17 Jul 2024 20:01:01 +0800 Subject: [PATCH 0317/1722] Fix flaky tests --- ...74_exact_rows_before_aggregation.reference | 26 +++++++++---------- .../03174_exact_rows_before_aggregation.sql | 4 +-- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference index 2fbdf325760..5f7fedbbcb3 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.reference @@ -355,15 +355,15 @@ "data": [ [0], - [4], - [3], - [2], - [5], [1], + [2], + [3], + [4], + [5], [6], [7], - [9], - [8] + [8], + [9] ], "rows": 10, @@ -384,20 +384,20 @@ "data": [ [0], - [4], - [3], - [2], - [5], [1], + [2], + [3], + [4], + [5], [6], [7], - [9], - [8] + [8], + [9] ], "rows": 10, - "rows_before_limit_at_least": 20, + "rows_before_limit_at_least": 10, "rows_before_aggregation": 20 } diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 17e3f3c2cef..0afc0be4370 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -23,9 +23,9 @@ select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i select max(i) from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 20 FORMAT JSONCompact; -select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i order by i limit 10 FORMAT JSONCompact; set prefer_localhost_replica = 0; -select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i limit 10 FORMAT JSONCompact; +select * from (select * from cluster(test_cluster_two_shards, currentDatabase(), test) where i < 10) group by i order by i limit 10 FORMAT JSONCompact; drop table if exists test; From 63936364b1abf345349403e656c4cf58c44715bc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 17 Jul 2024 16:53:35 +0200 Subject: [PATCH 0318/1722] fixes of tests --- src/Storages/VirtualColumnUtils.cpp | 2 +- .../0_stateless/03203_hive_style_partitioning.sh | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 31cee485dde..87c1aecc3a7 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -127,7 +127,7 @@ NameSet getVirtualNamesForFileLikeStorage() std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { - std::string pattern = "/([^/]+)=([^/]+)"; + std::string pattern = "([^/]+)=([^/]+)/"; re2::StringPiece input_piece(path); std::unordered_map key_values; diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 0f687d532b0..db1f073d736 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -124,13 +124,13 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; -SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0, _column1 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; """ From 938071cd55913c3bb2b8781750ef37bf6307acab Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 17 Jul 2024 14:56:00 +0000 Subject: [PATCH 0319/1722] add ci_include_fuzzer to PR body template --- .github/PULL_REQUEST_TEMPLATE.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index e045170561d..146542e980c 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -59,6 +59,8 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage - [ ] Exclude: All with aarch64, release, debug --- +- [ ] Run only libFuzzer related jobs +--- - [ ] Do not test - [ ] Woolen Wolfdog - [ ] Upload binaries for special builds From 542542b44d4688bc125887f811843249a4024379 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 17 Jul 2024 14:58:58 +0000 Subject: [PATCH 0320/1722] fix test --- .../queries/0_stateless/03161_lightweight_delete_projection.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 2c60d83d74d..3bf459cc32d 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,4 +1,6 @@ +SET lightweight_deletes_sync = 2; + DROP TABLE IF EXISTS users; -- compact part From bb01920370e1dd5faa7b17694f74175190537445 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 17 Jul 2024 16:19:02 +0000 Subject: [PATCH 0321/1722] add ci_exclude_ast to PR body template --- .github/PULL_REQUEST_TEMPLATE.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 146542e980c..8b6e957e1d8 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -59,7 +59,8 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage - [ ] Exclude: All with aarch64, release, debug --- -- [ ] Run only libFuzzer related jobs +- [ ] Run only fuzzers related jobs (libFuzzer fuzzers, AST fuzzers, etc.) +- [ ] Exclude AST fuzzers --- - [ ] Do not test - [ ] Woolen Wolfdog From 275b3666dadece731e368dd672e8d6e83ec22d8f Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 18 Jul 2024 01:18:34 +0000 Subject: [PATCH 0322/1722] try to fix the test --- .../03161_lightweight_delete_projection.sql | 114 +++++++++++++++++- 1 file changed, 111 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 3bf459cc32d..9d577f8a701 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,5 +1,73 @@ -SET lightweight_deletes_sync = 2; +SET lightweight_deletes_sync = 2, alter_sync = 2; + +Set max_insert_threads = 2, +group_by_two_level_threshold = 704642, +group_by_two_level_threshold_bytes = 49659607, +distributed_aggregation_memory_efficient = 0, +fsync_metadata = 0, +output_format_parallel_formatting = 0, +input_format_parallel_parsing = 1, +min_chunk_bytes_for_parallel_parsing = 14437539, +max_read_buffer_size = 887507, +prefer_localhost_replica = 0, +max_block_size = 73908, +max_joined_block_size_rows = 21162, +max_threads = 2, +optimize_append_index = 0, +optimize_if_chain_to_multiif = 1, +optimize_if_transform_strings_to_enum = 0, +optimize_read_in_order = 0, +optimize_or_like_chain = 1, +optimize_substitute_columns = 1, +enable_multiple_prewhere_read_steps = 1, +read_in_order_two_level_merge_threshold = 13, +optimize_aggregation_in_order = 1, +aggregation_in_order_max_block_bytes = 37110261, +use_uncompressed_cache = 0, +min_bytes_to_use_direct_io = 10737418240, +min_bytes_to_use_mmap_io = 1, +local_filesystem_read_method ='pread', +remote_filesystem_read_method ='threadpool', +local_filesystem_read_prefetch = 0, +filesystem_cache_segments_batch_size = 3, +read_from_filesystem_cache_if_exists_otherwise_bypass_cache = 1, +throw_on_error_from_cache_on_write_operations = 0, +remote_filesystem_read_prefetch = 1, +allow_prefetched_read_pool_for_remote_filesystem = 0, +filesystem_prefetch_max_memory_usage = '32Mi', +filesystem_prefetches_limit = 0, +filesystem_prefetch_min_bytes_for_single_read_task ='16Mi', +filesystem_prefetch_step_marks = 50, +filesystem_prefetch_step_bytes = 0, +compile_aggregate_expressions = 0, +compile_sort_description = 1, +merge_tree_coarse_index_granularity = 16, +optimize_distinct_in_order = 0, +max_bytes_before_external_sort = 0, +max_bytes_before_external_group_by = 0, +max_bytes_before_remerge_sort = 820113150, +min_compress_block_size = 1262249, +max_compress_block_size = 1472188, +merge_tree_compact_parts_min_granules_to_multibuffer_read = 56, +optimize_sorting_by_input_stream_properties = 1, +http_response_buffer_size = 1883022, +http_wait_end_of_query = False, +enable_memory_bound_merging_of_aggregation_results = 1, +min_count_to_compile_expression = 0, +min_count_to_compile_aggregate_expression = 0, +min_count_to_compile_sort_description = 0, +session_timezone ='Africa/Khartoum', +prefer_warmed_unmerged_parts_seconds = 10, +use_page_cache_for_disks_without_file_cache = True, +page_cache_inject_eviction = False, +merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.02, +prefer_external_sort_block_bytes = 100000000, +cross_join_min_rows_to_compress = 100000000, +cross_join_min_bytes_to_compress = 1, +min_external_table_block_size_bytes = 100000000, +max_parsing_threads = 0; + DROP TABLE IF EXISTS users; @@ -11,7 +79,27 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 10485760; +SETTINGS min_bytes_for_wide_part = 10485760, +ratio_of_defaults_for_sparse_serialization = 1.0, +prefer_fetch_merged_part_size_threshold = 1, +vertical_merge_algorithm_min_rows_to_activate = 1, +vertical_merge_algorithm_min_columns_to_activate = 100, +allow_vertical_merges_from_compact_to_wide_parts = 0, +min_merge_bytes_to_use_direct_io = 114145183, +index_granularity_bytes = 2660363, +merge_max_block_size = 13460, +index_granularity = 51768, +marks_compress_block_size = 59418, +primary_key_compress_block_size = 88795, +replace_long_file_name_to_hash = 0, +max_file_name_length = 0, +min_bytes_for_full_part_storage = 536870912, +compact_parts_max_bytes_to_buffer = 378557913, +compact_parts_max_granules_to_buffer = 254, +compact_parts_merge_max_bytes_to_prefetch_part = 26969686, +cache_populated_by_fetch = 0, +concurrent_part_removal_threshold = 38, +old_parts_lifetime = 480; INSERT INTO users VALUES (1231, 'John', 33); @@ -63,7 +151,27 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 0; +SETTINGS min_bytes_for_wide_part = 0, +ratio_of_defaults_for_sparse_serialization = 1.0, +prefer_fetch_merged_part_size_threshold = 1, +vertical_merge_algorithm_min_rows_to_activate = 1, +vertical_merge_algorithm_min_columns_to_activate = 100, +allow_vertical_merges_from_compact_to_wide_parts = 0, +min_merge_bytes_to_use_direct_io = 114145183, +index_granularity_bytes = 2660363, +merge_max_block_size = 13460, +index_granularity = 51768, +marks_compress_block_size = 59418, +primary_key_compress_block_size = 88795, +replace_long_file_name_to_hash = 0, +max_file_name_length = 0, +min_bytes_for_full_part_storage = 536870912, +compact_parts_max_bytes_to_buffer = 378557913, +compact_parts_max_granules_to_buffer = 254, +compact_parts_merge_max_bytes_to_prefetch_part = 26969686, +cache_populated_by_fetch = 0, +concurrent_part_removal_threshold = 38, +old_parts_lifetime = 480; INSERT INTO users VALUES (1231, 'John', 33); From 84559ab31cac8d994fbccf9085884c1c2ea8f4b8 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 18 Jul 2024 10:06:06 +0000 Subject: [PATCH 0323/1722] Debug code --- .../Local/LocalObjectStorage.cpp | 100 +++++++++--------- .../ObjectStorages/S3/S3ObjectStorage.cpp | 75 ++++++++----- .../ObjectStorage/StorageObjectStorage.h | 2 +- .../StorageObjectStorageSource.cpp | 36 ++++--- 4 files changed, 120 insertions(+), 93 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 3704e4bb672..607ce41dd0f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -58,27 +59,28 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL std::string answer(1000, ' '); size_t read_bytes = reader->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase 00"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; + LOG_DEBUG(&Poco::Logger::get("Method"), "Method read: {}", read_settings.remote_fs_method == RemoteFSReadMethod::read); switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); return std::make_unique( std::move(read_buffer_creator), objects, "file:", modified_settings, @@ -86,56 +88,58 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL } case RemoteFSReadMethod::threadpool: { - LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); + // LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); auto impl = std::make_unique( std::move(read_buffer_creator), objects, "file:", modified_settings, global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - auto impl3 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl3 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); + // auto & reader2 = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + + // auto async_reader = std::make_unique( + // std::move(impl3), + // reader2, + // read_settings, + // global_context->getAsyncReadCounters(), + // global_context->getFilesystemReadPrefetchesLog()); + + // answer = std::string(1000, ' '); + // read_bytes = async_reader->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("AsynchronousBoundedReadBuffer"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - std::move(impl3), + return std::make_unique( + std::move(impl), reader, read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); - - answer = std::string(1000, ' '); - read_bytes = async_reader->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("AsynchronousBoundedReadBuffer"), - "Read bytes: {}, string: {}", - read_bytes, - answer.substr(0, read_bytes)); - - // reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - return std::make_unique( - std::move(impl), reader, read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); } } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 267c3eaaea4..351f167b6b9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -200,21 +200,41 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT restricted_seek); }; + // auto read_buffer_creator2 = [this, settings_ptr, disk_read_settings]( + // bool restricted_seek, const StoredObject & object_) -> std::unique_ptr + // { + // return std::make_unique( + // client.get(), + // uri.bucket, + // object_.remote_path, + // uri.version_id, + // settings_ptr->request_settings, + // disk_read_settings, + // /* use_external_buffer */ true, + // /* offset */ 0, + // /* read_until_position */ 0, + // restricted_seek); + // }; + + switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "s3:" + uri.bucket + "/", - disk_read_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator2), + // objects, + // "s3:" + uri.bucket + "/", + // disk_read_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ false); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather 000"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); return std::make_unique( std::move(read_buffer_creator), @@ -224,17 +244,24 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */false); } - case RemoteFSReadMethod::threadpool: - { + case RemoteFSReadMethod::threadpool: { + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator2), + // objects, + // "s3:" + uri.bucket + "/", + // disk_read_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); + + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather 001"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); + auto impl = std::make_unique( - std::move(read_buffer_creator), - objects, - "s3:" + uri.bucket + "/", - disk_read_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */true); - - auto impl2 = std::make_unique( std::move(read_buffer_creator), objects, "s3:" + uri.bucket + "/", @@ -242,12 +269,6 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( std::move(impl), reader, disk_read_settings, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 818ce055c77..7a13ada224c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -157,7 +157,7 @@ public: ContextPtr local_context, bool with_table_structure); - /// Storage type: s3, hdfs, azure. + /// Storage type: s3, hdfs, azure, local. virtual std::string getTypeName() const = 0; /// Engine name: S3, HDFS, Azure. virtual std::string getEngineName() const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index f664bed0204..8b800cbc343 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -461,6 +461,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); + read_settings.remote_fs_method = RemoteFSReadMethod::read; + LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); read_settings.enable_filesystem_cache = false; @@ -477,28 +479,28 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. - if (use_prefetch) - { - LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + // if (use_prefetch) + // { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); + LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); - auto async_reader = object_storage->readObjects( - StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + auto async_reader + = object_storage->readObjects(StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - return async_reader; - } - else - { - /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); + return async_reader; + // } + // else + // { + // /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + // LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); - return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - } + // return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + // } } StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) From 8fad2860251abf3fc71b525522c7ce51bd4687aa Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 18 Jul 2024 15:03:12 +0000 Subject: [PATCH 0324/1722] Remove asyncronous processing for local storage --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 5 +- .../Local/LocalObjectStorage.cpp | 96 ++----------------- .../ObjectStorage/Local/Configuration.cpp | 2 +- .../StorageObjectStorageSource.cpp | 2 +- src/Storages/ObjectStorage/Utils.cpp | 5 +- 6 files changed, 20 insertions(+), 91 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 92b1c750a55..10b869e58f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,6 +114,7 @@ class IColumn; M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ + M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 73fb28dc67a..5de4630758d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,8 +79,9 @@ static std::initializer_list LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr + auto read_buffer_creator = + [=] (bool /* restricted_seek */, const StoredObject & object) + -> std::unique_ptr { - LOG_DEBUG(&Poco::Logger::get("Read"), "Remote Path: {}", object.remote_path); - - auto reader = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); - - std::string answer(1000, ' '); - size_t read_bytes = reader->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; - LOG_DEBUG(&Poco::Logger::get("Method"), "Method read: {}", read_settings.remote_fs_method == RemoteFSReadMethod::read); - switch (read_settings.remote_fs_method) - { - case RemoteFSReadMethod::read: - { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - return std::make_unique( - std::move(read_buffer_creator), objects, "file:", modified_settings, - global_context->getFilesystemCacheLog(), /* use_external_buffer */false); - } - case RemoteFSReadMethod::threadpool: - { - // LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); - auto impl = std::make_unique( - std::move(read_buffer_creator), objects, "file:", modified_settings, - global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - // auto impl3 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - - // auto & reader2 = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - // auto async_reader = std::make_unique( - // std::move(impl3), - // reader2, - // read_settings, - // global_context->getAsyncReadCounters(), - // global_context->getFilesystemReadPrefetchesLog()); - - // answer = std::string(1000, ' '); - // read_bytes = async_reader->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("AsynchronousBoundedReadBuffer"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - return std::make_unique( - std::move(impl), - reader, - read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - } + return std::make_unique( + std::move(read_buffer_creator), + objects, + "file:", + modified_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */ false); } ReadSettings LocalObjectStorage::patchSettings(const ReadSettings & read_settings) const diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 740132f0981..364bd21a64e 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -66,7 +66,7 @@ StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings( const auto & settings = context->getSettingsRef(); return StorageObjectStorage::QuerySettings{ .truncate_on_insert = settings.engine_file_truncate_on_insert, - .create_new_file_on_insert = false, + .create_new_file_on_insert = settings.local_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.engine_file_skip_empty_files, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8b800cbc343..3053a48b75b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -461,7 +461,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - read_settings.remote_fs_method = RemoteFSReadMethod::read; + // read_settings.remote_fs_method = RemoteFSReadMethod::read; LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index e49e14d2a0c..6491deef440 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -40,7 +40,10 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", - configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); + configuration.getNamespace(), + key, + configuration.getTypeName() == "local" ? "engine_file" : configuration.getTypeName(), + configuration.getTypeName()); } void resolveSchemaAndFormat( From 436f6463c03b5cfc56af30e49faa6e192750a8f5 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:06:30 +0000 Subject: [PATCH 0325/1722] Add test --- contrib/grpc | 2 +- .../test_local_storage/configs/config.xml | 3 + tests/integration/test_local_storage/test.py | 243 +++++++++--------- tests/integration/tmp_data/example.csv | 5 + 4 files changed, 131 insertions(+), 122 deletions(-) create mode 100644 tests/integration/test_local_storage/configs/config.xml create mode 100644 tests/integration/tmp_data/example.csv diff --git a/contrib/grpc b/contrib/grpc index f5b7fdc2dff..1716359d2e2 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df +Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml new file mode 100644 index 00000000000..bcbb107f0a2 --- /dev/null +++ b/tests/integration/test_local_storage/configs/config.xml @@ -0,0 +1,3 @@ + + /var/lib/clickhouse/ + \ No newline at end of file diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index 96d116ec6a2..d27fb746d12 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -6,148 +6,149 @@ import string import pytest from helpers.cluster import ClickHouseCluster -from azure.storage.blob import BlobServiceClient -from test_storage_azure_blob_storage.test import azure_query -NODE_NAME = "node" - - -def generate_cluster_def(port): - path = os.path.join( - os.path.dirname(os.path.realpath(__file__)), - "./_gen/disk_storage_conf.xml", - ) - os.makedirs(os.path.dirname(path), exist_ok=True) - with open(path, "w") as f: - f.write( - f""" - - - - object_storage - azure_blob_storage - plain_rewritable - http://azurite1:{port}/devstoreaccount1 - cont - true - devstoreaccount1 - Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== - 100000 - 100000 - 10 - 10 - - - - - -
- blob_storage_disk -
-
-
-
-
-
-""" - ) - return path - - -insert_values = [ - "(0,'data'),(1,'data')", - ",".join( - f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" - for i in range(10) - ), -] +from pathlib import Path @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): + global cluster try: cluster = ClickHouseCluster(__file__) - port = cluster.azurite_port - path = generate_cluster_def(port) cluster.add_instance( - NODE_NAME, - main_configs=[ - path, - ], - with_azurite=True, - stay_alive=True, + "test_local_storage", main_configs=["configs/config.xml"], stay_alive=True ) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") + cluster.start() yield cluster + finally: cluster.shutdown() -def test_insert_select(cluster): - node = cluster.instances[NODE_NAME] +def process_result(line: str): + return sorted( + list( + map( + lambda x: (int(x.split("\t")[0]), x.split("\t")[1]), + filter(lambda x: len(x) > 0, line.split("\n")), + ) + ) + ) - for index, value in enumerate(insert_values): - azure_query( - node, - """ - CREATE TABLE test_{} ( + +def test_local_engine(started_cluster): + node = started_cluster.instances["test_local_storage"] + node.query( + """ + CREATE TABLE test_0 ( id Int64, data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS storage_policy='blob_storage_policy' - """.format( - index - ), - ) + ) ENGINE=Local('/data/example.csv', 'CSV'); + """ + ) - azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) + node.query( + """ + INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); + """ + ) + result = node.query( + """ + select * from test_0; + """ + ) -def test_restart_server(cluster): - node = cluster.instances[NODE_NAME] + assert [(-1, "7"), (1, "3"), (4, "abc")] == process_result(result) + + error_got = node.query_and_get_error( + """ + INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); + """ + ) + + print("Error got", error_got) + + node.query( + """ + SET engine_file_truncate_on_insert = 1; + """ + ) + + node.query( + """ + INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); + """, + settings={"engine_file_truncate_on_insert": 1}, + ) + + result = node.query( + """ + SELECT * FROM test_0; + """ + ) + + assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) + + node.query( + """ + SET local_create_new_file_on_insert = 1; + """ + ) + + node.query( + """ + INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); + """, + settings={"local_create_new_file_on_insert": 1}, + ) + + result = node.query( + """ + SELECT * FROM test_0; + """ + ) + + assert [ + (-1, "7"), + (0, "15"), + (1, "3"), + (4, "abc"), + (5, "arr"), + (9, "ty"), + ] == process_result(result) - for index, value in enumerate(insert_values): - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) node.restart_clickhouse() - for index, value in enumerate(insert_values): - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) - - -def test_drop_table(cluster): - node = cluster.instances[NODE_NAME] - - for index, value in enumerate(insert_values): - node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) - - port = cluster.env_variables["AZURITE_PORT"] - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + result = node.query( + """ + SELECT * FROM test_0; + """ ) - blob_service_client = BlobServiceClient.from_connection_string(connection_string) - containers = blob_service_client.list_containers() - for container in containers: - container_client = blob_service_client.get_container_client(container) - assert len(list(container_client.list_blobs())) == 0 + + assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) + + +def test_table_function(started_cluster): + with open("/tmp/example.csv", "w") as f: + f.write( + """id,data +1,Str1 +2,Str2""" + ) + node = started_cluster.instances["test_local_storage"] + + node.copy_file_to_container("/tmp/example.csv", "/data/example2.csv") + + result = node.query( + """ + SELECT * FROM local('/data/example2.csv', 'CSV', 'id Int64, data String'); + """ + ) + + print("Res5", result) + + assert [(1, "Str1"), (2, "Str2")] == process_result(result) + + # assert False diff --git a/tests/integration/tmp_data/example.csv b/tests/integration/tmp_data/example.csv new file mode 100644 index 00000000000..93d6fb20f38 --- /dev/null +++ b/tests/integration/tmp_data/example.csv @@ -0,0 +1,5 @@ + +id,data +1,'Str1' +2,'Str2' + \ No newline at end of file From 27ab6aa8b52118b3373aaa9ff3ccb8320a5f5344 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:16:24 +0000 Subject: [PATCH 0326/1722] Remove logs --- .../Local/LocalObjectStorage.cpp | 14 +-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 53 +-------- .../Executors/PullingPipelineExecutor.cpp | 22 +--- .../StorageObjectStorageSource.cpp | 107 ++++-------------- 4 files changed, 35 insertions(+), 161 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 266e39f8f97..01ff2e2baf1 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,18 +1,15 @@ #include -#include -#include -#include -#include +#include +#include +#include #include #include +#include #include #include -#include -#include -#include #include -#include +#include namespace fs = std::filesystem; @@ -58,7 +55,6 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; - return std::make_unique( std::move(read_buffer_creator), objects, diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 351f167b6b9..a6672e14e10 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -200,42 +200,10 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT restricted_seek); }; - // auto read_buffer_creator2 = [this, settings_ptr, disk_read_settings]( - // bool restricted_seek, const StoredObject & object_) -> std::unique_ptr - // { - // return std::make_unique( - // client.get(), - // uri.bucket, - // object_.remote_path, - // uri.version_id, - // settings_ptr->request_settings, - // disk_read_settings, - // /* use_external_buffer */ true, - // /* offset */ 0, - // /* read_until_position */ 0, - // restricted_seek); - // }; - - switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator2), - // objects, - // "s3:" + uri.bucket + "/", - // disk_read_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ false); - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather 000"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - return std::make_unique( std::move(read_buffer_creator), objects, @@ -244,30 +212,15 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */false); } - case RemoteFSReadMethod::threadpool: { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator2), - // objects, - // "s3:" + uri.bucket + "/", - // disk_read_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather 001"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - + case RemoteFSReadMethod::threadpool: + { auto impl = std::make_unique( std::move(read_buffer_creator), objects, "s3:" + uri.bucket + "/", disk_read_settings, global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + /* use_external_buffer */true); auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index a4db27d3f37..25c15d40c9a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -42,32 +42,19 @@ const Block & PullingPipelineExecutor::getHeader() const bool PullingPipelineExecutor::pull(Chunk & chunk) { - LOG_DEBUG(&Poco::Logger::get("Pulling 1"), "Pulling 1"); - if (!executor) { executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } - LOG_DEBUG(&Poco::Logger::get("Pulling 2"), "Pulling 2"); - if (!executor->checkTimeLimitSoft()) return false; - LOG_DEBUG(&Poco::Logger::get("Pulling 3"), "Pulling 3"); - - if (!executor->executeStep(&has_data_flag)) return false; - LOG_DEBUG(&Poco::Logger::get("Pulling 4"), "Pulling 4"); - - chunk = pulling_format->getChunk(); - - LOG_DEBUG(&Poco::Logger::get("Pulling 5"), "Pulling 5"); - return true; } @@ -86,13 +73,10 @@ bool PullingPipelineExecutor::pull(Block & block) } block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 3053a48b75b..086482e330a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -1,18 +1,17 @@ #include "StorageObjectStorageSource.h" +#include #include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include "Common/logger_useful.h" +#include #include #include @@ -71,7 +70,6 @@ StorageObjectStorageSource::StorageObjectStorageSource( , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { - LOG_DEBUG(&Poco::Logger::get("Source created"), "Source created"); } StorageObjectStorageSource::~StorageObjectStorageSource() @@ -134,7 +132,6 @@ std::shared_ptr StorageObjectStorageSourc { ConfigurationPtr copy_configuration = configuration->clone(); auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - if (filter_dag) { auto keys = configuration->getPaths(); @@ -146,19 +143,6 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } - LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); - for (auto && key : configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); - } - - LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Keys size: {}", copy_configuration->getPaths().size()); - for (auto && key : copy_configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Current key: {}", key); - } - - iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); @@ -187,11 +171,8 @@ Chunk StorageObjectStorageSource::generate() { lazyInitialize(); - while (true) { - LOG_DEBUG(&Poco::Logger::get("Generating"), "Generating reader: {}", !(!reader)); - if (isCancelled() || !reader) { if (reader) @@ -199,15 +180,10 @@ Chunk StorageObjectStorageSource::generate() break; } - LOG_DEBUG(&Poco::Logger::get("Generating 2"), "Generating 2"); - Chunk chunk; if (reader->pull(chunk)) { - LOG_DEBUG(&Poco::Logger::get("Generating 3"), "Generating 3"); - UInt64 num_rows = chunk.getNumRows(); - LOG_DEBUG(&Poco::Logger::get("Creating_chunk"), "Chunk size: {}", num_rows); total_rows_in_file += num_rows; size_t chunk_size = 0; @@ -255,9 +231,6 @@ Chunk StorageObjectStorageSource::generate() return chunk; } - LOG_DEBUG(&Poco::Logger::get("Generating 4"), "Generating 4"); - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); @@ -328,8 +301,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); - LOG_DEBUG(&Poco::Logger::get("Unreached point 1"), ""); - QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; @@ -354,17 +325,11 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); }; - LOG_DEBUG(&Poco::Logger::get("Unreached point 2"), ""); - - std::optional num_rows_from_cache = need_only_count && context_->getSettingsRef().use_cache_for_count_from_files ? try_get_num_rows_from_cache() : std::nullopt; - LOG_DEBUG(&Poco::Logger::get("Unreached point 3"), ""); - - if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -377,8 +342,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { - LOG_DEBUG(&Poco::Logger::get("Unreached point 4"), ""); - CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { @@ -389,12 +352,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); - LOG_DEBUG(&Poco::Logger::get("Info relative path"), "Info: {}", object_info->relative_path); read_buf = createReadBuffer(*object_info, object_storage, context_, log); - auto new_read_buf = createReadBuffer(*object_info, object_storage, context_, log); - std::string answer(1000, ' '); - size_t read_bytes = new_read_buf->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("Read buffer"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); } auto input_format = FormatFactory::instance().getInput( @@ -460,17 +418,11 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( const auto & object_size = object_info.metadata->size_bytes; auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - - // read_settings.remote_fs_method = RemoteFSReadMethod::read; - - LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); - read_settings.enable_filesystem_cache = false; /// FIXME: Changing this setting to default value breaks something around parquet reading read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; const bool object_too_small = object_size <= 2 * context_->getSettingsRef().max_download_buffer_size; - const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; /// User's object may change, don't cache it. @@ -479,28 +431,24 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. - // if (use_prefetch) - // { - LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + if (use_prefetch) + { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); + auto async_reader = object_storage->readObjects( + StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); - auto async_reader - = object_storage->readObjects(StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; - // } - // else - // { - // /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - // LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); - - // return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - // } + return async_reader; + } + else + { + /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + } } StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) @@ -678,18 +626,11 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( , keys(configuration->getPaths()) , ignore_non_existent_files(ignore_non_existent_files_) { - LOG_DEBUG(&Poco::Logger::get("Keys size"), "Keys size: {}", keys.size()); - for (auto && key : keys) - { - LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); - } if (read_keys_) { /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? - for (auto && key : keys) { - LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } From 45e0f0350e475503cdd2ac0a30d14ca2466fceb9 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:25:37 +0000 Subject: [PATCH 0327/1722] Fix some stuff --- .../integration/test_local_storage/files/example2.csv | 3 +++ tests/integration/test_local_storage/test.py | 10 +--------- tests/integration/tmp_data/example.csv | 5 ----- 3 files changed, 4 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_local_storage/files/example2.csv delete mode 100644 tests/integration/tmp_data/example.csv diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv new file mode 100644 index 00000000000..1abfb225937 --- /dev/null +++ b/tests/integration/test_local_storage/files/example2.csv @@ -0,0 +1,3 @@ +id,data +1,Str1 +2,Str2 \ No newline at end of file diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index d27fb746d12..54c8016c376 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -68,8 +68,6 @@ def test_local_engine(started_cluster): """ ) - print("Error got", error_got) - node.query( """ SET engine_file_truncate_on_insert = 1; @@ -131,15 +129,9 @@ def test_local_engine(started_cluster): def test_table_function(started_cluster): - with open("/tmp/example.csv", "w") as f: - f.write( - """id,data -1,Str1 -2,Str2""" - ) node = started_cluster.instances["test_local_storage"] - node.copy_file_to_container("/tmp/example.csv", "/data/example2.csv") + node.copy_file_to_container("test_local_storage/files/example2.csv", "/data/example2.csv") result = node.query( """ diff --git a/tests/integration/tmp_data/example.csv b/tests/integration/tmp_data/example.csv deleted file mode 100644 index 93d6fb20f38..00000000000 --- a/tests/integration/tmp_data/example.csv +++ /dev/null @@ -1,5 +0,0 @@ - -id,data -1,'Str1' -2,'Str2' - \ No newline at end of file From 07d03c0c67437eaf70fe70199e71430d643302b6 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:30:46 +0000 Subject: [PATCH 0328/1722] Fix minor things --- src/IO/ReadHelpers.cpp | 10 +++------- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 6 ------ .../integration/test_local_storage/configs/config.xml | 2 +- .../integration/test_local_storage/files/example2.csv | 2 +- 4 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ddc2e912fb3..c771fced73a 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -80,7 +80,6 @@ UUID parseUUID(std::span src) return uuid; } - void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; @@ -89,15 +88,12 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) if (buf.eof()) out << " at end of stream."; else - out << " before: " << quote << String(buf.position(), std::min(static_cast(1000), buf.buffer().end() - buf.position())); + out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); - throw Exception( - ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, - "Cannot parse input: expected {} {}", - out.str(), - buf.buffer().end() - buf.position()); + throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); } + bool checkString(const char * s, ReadBuffer & buf) { for (; *s; ++s) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 0be929bc2c1..10fffd148be 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -125,12 +125,6 @@ public: , num_streams(num_streams_) , distributed_processing(distributed_processing_) { - LOG_DEBUG(&Poco::Logger::get("Read step created"), "Read step created"); - LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); - for (auto && key : configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); - } } std::string getName() const override { return name; } diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml index bcbb107f0a2..b4179ee51df 100644 --- a/tests/integration/test_local_storage/configs/config.xml +++ b/tests/integration/test_local_storage/configs/config.xml @@ -1,3 +1,3 @@ /var/lib/clickhouse/ - \ No newline at end of file + diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv index 1abfb225937..7b6e6d6bab1 100644 --- a/tests/integration/test_local_storage/files/example2.csv +++ b/tests/integration/test_local_storage/files/example2.csv @@ -1,3 +1,3 @@ id,data 1,Str1 -2,Str2 \ No newline at end of file +2,Str2 From f0ef101bc558400c1f38666a390862da71e1c0f2 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:40:30 +0000 Subject: [PATCH 0329/1722] Minor changes --- src/Storages/ObjectStorage/Local/Configuration.cpp | 1 - tests/integration/test_local_storage/test.py | 4 +++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 364bd21a64e..d64b22769ad 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int LOGICAL_ERROR; } void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index 54c8016c376..89c3c17e1f0 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -131,7 +131,9 @@ def test_local_engine(started_cluster): def test_table_function(started_cluster): node = started_cluster.instances["test_local_storage"] - node.copy_file_to_container("test_local_storage/files/example2.csv", "/data/example2.csv") + node.copy_file_to_container( + "test_local_storage/files/example2.csv", "/data/example2.csv" + ) result = node.query( """ From 57181a5a4871b251c3d1aa04a39d467a715645a4 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 14:47:57 +0000 Subject: [PATCH 0330/1722] Fix compilation bug --- src/Storages/ObjectStorage/registerStorageObjectStorage.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 13f2df7a0e7..4bc8cfa6a2f 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -10,8 +10,6 @@ namespace DB { -#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -71,8 +69,6 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } -#endif - #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { From 2504a6c36016b41e33ee5323fca79f5d511fb3ce Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 19 Jul 2024 14:59:38 +0000 Subject: [PATCH 0331/1722] make test output a bit clear --- ...61_lightweight_delete_projection.reference | 8 ++ .../03161_lightweight_delete_projection.sql | 131 ++---------------- 2 files changed, 21 insertions(+), 118 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index 3401eaf6162..cb623ea2b50 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,6 +1,14 @@ +compact part +testing throw default mode +testing drop mode +testing rebuild mode 8888 Alice 50 p1 p2 +wide part +testing throw default mode +testing drop mode +testing rebuild mode 8888 Alice 50 p1 p2 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 9d577f8a701..f2d6dcb164f 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,77 +1,11 @@ SET lightweight_deletes_sync = 2, alter_sync = 2; -Set max_insert_threads = 2, -group_by_two_level_threshold = 704642, -group_by_two_level_threshold_bytes = 49659607, -distributed_aggregation_memory_efficient = 0, -fsync_metadata = 0, -output_format_parallel_formatting = 0, -input_format_parallel_parsing = 1, -min_chunk_bytes_for_parallel_parsing = 14437539, -max_read_buffer_size = 887507, -prefer_localhost_replica = 0, -max_block_size = 73908, -max_joined_block_size_rows = 21162, -max_threads = 2, -optimize_append_index = 0, -optimize_if_chain_to_multiif = 1, -optimize_if_transform_strings_to_enum = 0, -optimize_read_in_order = 0, -optimize_or_like_chain = 1, -optimize_substitute_columns = 1, -enable_multiple_prewhere_read_steps = 1, -read_in_order_two_level_merge_threshold = 13, -optimize_aggregation_in_order = 1, -aggregation_in_order_max_block_bytes = 37110261, -use_uncompressed_cache = 0, -min_bytes_to_use_direct_io = 10737418240, -min_bytes_to_use_mmap_io = 1, -local_filesystem_read_method ='pread', -remote_filesystem_read_method ='threadpool', -local_filesystem_read_prefetch = 0, -filesystem_cache_segments_batch_size = 3, -read_from_filesystem_cache_if_exists_otherwise_bypass_cache = 1, -throw_on_error_from_cache_on_write_operations = 0, -remote_filesystem_read_prefetch = 1, -allow_prefetched_read_pool_for_remote_filesystem = 0, -filesystem_prefetch_max_memory_usage = '32Mi', -filesystem_prefetches_limit = 0, -filesystem_prefetch_min_bytes_for_single_read_task ='16Mi', -filesystem_prefetch_step_marks = 50, -filesystem_prefetch_step_bytes = 0, -compile_aggregate_expressions = 0, -compile_sort_description = 1, -merge_tree_coarse_index_granularity = 16, -optimize_distinct_in_order = 0, -max_bytes_before_external_sort = 0, -max_bytes_before_external_group_by = 0, -max_bytes_before_remerge_sort = 820113150, -min_compress_block_size = 1262249, -max_compress_block_size = 1472188, -merge_tree_compact_parts_min_granules_to_multibuffer_read = 56, -optimize_sorting_by_input_stream_properties = 1, -http_response_buffer_size = 1883022, -http_wait_end_of_query = False, -enable_memory_bound_merging_of_aggregation_results = 1, -min_count_to_compile_expression = 0, -min_count_to_compile_aggregate_expression = 0, -min_count_to_compile_sort_description = 0, -session_timezone ='Africa/Khartoum', -prefer_warmed_unmerged_parts_seconds = 10, -use_page_cache_for_disks_without_file_cache = True, -page_cache_inject_eviction = False, -merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.02, -prefer_external_sort_block_bytes = 100000000, -cross_join_min_rows_to_compress = 100000000, -cross_join_min_bytes_to_compress = 1, -min_external_table_block_size_bytes = 100000000, -max_parsing_threads = 0; - - DROP TABLE IF EXISTS users; --- compact part + +SELECT 'compact part'; + CREATE TABLE users ( uid Int16, name String, @@ -79,36 +13,17 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 10485760, -ratio_of_defaults_for_sparse_serialization = 1.0, -prefer_fetch_merged_part_size_threshold = 1, -vertical_merge_algorithm_min_rows_to_activate = 1, -vertical_merge_algorithm_min_columns_to_activate = 100, -allow_vertical_merges_from_compact_to_wide_parts = 0, -min_merge_bytes_to_use_direct_io = 114145183, -index_granularity_bytes = 2660363, -merge_max_block_size = 13460, -index_granularity = 51768, -marks_compress_block_size = 59418, -primary_key_compress_block_size = 88795, -replace_long_file_name_to_hash = 0, -max_file_name_length = 0, -min_bytes_for_full_part_storage = 536870912, -compact_parts_max_bytes_to_buffer = 378557913, -compact_parts_max_granules_to_buffer = 254, -compact_parts_merge_max_bytes_to_prefetch_part = 26969686, -cache_populated_by_fetch = 0, -concurrent_part_removal_threshold = 38, -old_parts_lifetime = 480; +SETTINGS min_bytes_for_wide_part = 10485760; INSERT INTO users VALUES (1231, 'John', 33); --- testing throw default mode +SELECT 'testing throw default mode'; + ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } --- testing drop mode +SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; @@ -123,7 +38,7 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- testing rebuild mode +SELECT 'testing rebuild mode'; INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; @@ -143,7 +58,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); DROP TABLE users; --- wide part +SELECT 'wide part'; CREATE TABLE users ( uid Int16, name String, @@ -151,36 +66,16 @@ CREATE TABLE users ( projection p1 (select count(), age group by age), projection p2 (select age, name group by age, name) ) ENGINE = MergeTree order by uid -SETTINGS min_bytes_for_wide_part = 0, -ratio_of_defaults_for_sparse_serialization = 1.0, -prefer_fetch_merged_part_size_threshold = 1, -vertical_merge_algorithm_min_rows_to_activate = 1, -vertical_merge_algorithm_min_columns_to_activate = 100, -allow_vertical_merges_from_compact_to_wide_parts = 0, -min_merge_bytes_to_use_direct_io = 114145183, -index_granularity_bytes = 2660363, -merge_max_block_size = 13460, -index_granularity = 51768, -marks_compress_block_size = 59418, -primary_key_compress_block_size = 88795, -replace_long_file_name_to_hash = 0, -max_file_name_length = 0, -min_bytes_for_full_part_storage = 536870912, -compact_parts_max_bytes_to_buffer = 378557913, -compact_parts_max_granules_to_buffer = 254, -compact_parts_merge_max_bytes_to_prefetch_part = 26969686, -cache_populated_by_fetch = 0, -concurrent_part_removal_threshold = 38, -old_parts_lifetime = 480; +SETTINGS min_bytes_for_wide_part = 0; INSERT INTO users VALUES (1231, 'John', 33); --- testing throw default mode +SELECT 'testing throw default mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } --- testing drop mode +SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; DELETE FROM users WHERE uid = 1231; @@ -195,7 +90,7 @@ SELECT FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); --- testing rebuild mode +SELECT 'testing rebuild mode'; INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; From 2df3e99666e22e8838bc500eda37393772e1a4f1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Jul 2024 17:04:16 +0000 Subject: [PATCH 0332/1722] better mutations snapshot --- src/Interpreters/MutationsInterpreter.h | 1 - src/Storages/MergeTree/MergeTask.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 41 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 38 ++++++++++------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 -- .../MergeTree/MergeTreeMutationEntry.cpp | 1 - .../MergeTree/MergeTreeSequentialSource.cpp | 3 +- src/Storages/MergeTree/MutateTask.cpp | 1 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 36 +++++++++------- .../MergeTree/ReplicatedMergeTreeQueue.h | 7 ++-- src/Storages/StorageMergeTree.cpp | 24 +++++++---- src/Storages/StorageMergeTree.h | 7 ++-- 12 files changed, 91 insertions(+), 72 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 8ae438efc19..b3a8dd41736 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -6,7 +6,6 @@ #include #include #include -#include "Storages/MergeTree/AlterConversions.h" namespace DB diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 619e2fc7b2b..180538480fa 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -4,7 +4,6 @@ #include #include -#include "Storages/MergeTree/AlterConversions.h" #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d631a8833d0..cdfa41eef78 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8444,9 +8444,10 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart DB::updateObjectColumns(object_columns, columns, part->getColumns()); } -bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const +bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const { - return !hasLightweightDeletedMask(); + const auto & snapshot_data = assert_cast(*storage_snapshot->data); + return !hasLightweightDeletedMask() && !snapshot_data.mutations_snapshot->hasDataMutations(); } Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) @@ -8698,16 +8699,16 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } static void updateMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, Int64 increment) { - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + if (num_data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply); - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + if (num_metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply); bool has_data_mutation = false; bool has_metadata_mutation = false; @@ -8716,40 +8717,40 @@ static void updateMutationsCounters( { if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - data_mutations_to_apply += increment; + num_data_mutations_to_apply += increment; has_data_mutation = true; - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + if (num_data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply); } if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) { - metadata_mutations_to_apply += increment; + num_metadata_mutations_to_apply += increment; has_metadata_mutation = true; - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + if (num_metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply); } } } void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); + return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, 1); } void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); + return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 984662daff0..d5cfddb70af 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -34,7 +34,6 @@ #include #include #include -#include "Storages/ProjectionsDescription.h" #include #include @@ -444,10 +443,14 @@ public: bool areAsynchronousInsertsEnabled() const override; - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; + bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const override; + /// A snapshot of pending mutations that weren't applied to some of the parts yet + /// and should be applied on the fly (i.e. when reading from the part). + /// Mutations not supported by AlterConversions (supportsMutationCommandType()) can be omitted. struct IMutationsSnapshot { + /// Contains info that doesn't depend on state of mutations. struct Params { Int64 metadata_version = -1; @@ -455,18 +458,25 @@ public: bool need_data_mutations = false; }; + /// Contains info that depends on state of mutations. + struct Info + { + Int64 num_data_mutations = 0; + Int64 num_metadata_mutations = 0; + }; + Params params; + Info info; IMutationsSnapshot() = default; - explicit IMutationsSnapshot(Params params_) : params(std::move(params_)) {} + IMutationsSnapshot(Params params_, Info info_): params(std::move(params_)), info(std::move(info_)) {} - /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly - /// (i.e. when reading from the part). Mutations not supported by AlterConversions - /// (supportsMutationCommandType()) can be omitted. - /// - /// @return list of mutations, in *reverse* order (newest to oldest) + /// Returns mutation commands that are required to be applied to the `part`. + /// @return list of mutation commands, in *reverse* order (newest to oldest) virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; virtual std::shared_ptr cloneEmpty() const = 0; + bool hasDataMutations() const { return params.need_data_mutations && info.num_data_mutations > 0; } + virtual ~IMutationsSnapshot() = default; }; @@ -956,10 +966,10 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } - /// TODO: comment + /// Returns a snapshot of mutations that probably will be applied on the fly to parts during reading. virtual MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const = 0; - /// TODO: comment + /// Returns the minimum version of metadata among parts. static Int64 getMinMetadataVersion(const DataPartsVector & parts); /// Return alter conversions for part which must be applied on fly. @@ -1761,14 +1771,14 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & lock); void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & lock); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 94e00c531ba..1e10e4adc9d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -896,9 +896,6 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return std::make_shared(); std::optional indexes; - /// NOTE: We don't need mutations snapshot because the returned analysis_result is only used for: - /// 1. estimate the number of rows to read; - /// 2. projection reading, which doesn't have alter conversions. return ReadFromMergeTree::selectRangesToRead( std::move(parts), metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 6f06b921031..5970aed497e 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -6,7 +6,6 @@ #include #include #include -#include "Storages/MutationCommands.h" #include diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 8368f4e80f7..31e1c0f235f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -14,7 +14,6 @@ #include #include #include -#include "Storages/MergeTree/AlterConversions.h" #include namespace DB @@ -65,7 +64,7 @@ private: /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; - /// TODO: comment. + /// Alter and mutation commands that are required to be applied to the part on-fly. AlterConversionsPtr alter_conversions; /// Columns we have to read (each Block from read will contain them) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 74069d8b06e..a95cccdc0d2 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -33,7 +33,6 @@ #include #include #include -#include "Storages/MergeTree/AlterConversions.h" #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c2fcfbd1810..c3d91ca0705 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -14,8 +14,6 @@ #include #include #include -#include -#include #include #include @@ -952,7 +950,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, state_lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, state_lock); } else it = mutations_by_znode.erase(it); @@ -1002,7 +1000,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) @@ -1913,7 +1911,7 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo MutationCommands result; - bool seen_all_data_mutations = !params.need_data_mutations; + bool seen_all_data_mutations = !hasDataMutations(); bool seen_all_metadata_mutations = part_metadata_version >= params.metadata_version; if (seen_all_data_mutations && seen_all_metadata_mutations) @@ -1940,9 +1938,9 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo auto alter_version = entry->alter_version; - if (!seen_all_metadata_mutations && alter_version != -1) + if (alter_version != -1) { - if (alter_version > params.metadata_version) + if (seen_all_metadata_mutations || alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version @@ -1965,11 +1963,17 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const { - auto res = std::make_shared(params); - std::lock_guard lock(state_mutex); - bool need_data_mutations = params.need_data_mutations && data_mutations_to_apply > 0; + MutationsSnapshot::Info info + { + .num_data_mutations = num_data_mutations_to_apply, + .num_metadata_mutations = num_metadata_mutations_to_apply, + }; + + auto res = std::make_shared(params, std::move(info)); + + bool need_data_mutations = res->hasDataMutations(); bool need_metatadata_mutations = params.min_part_metadata_version < params.metadata_version; if (!need_data_mutations && !need_metatadata_mutations) @@ -2000,14 +2004,16 @@ MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapsh auto alter_version = status->entry->alter_version; - if (!seen_all_metadata_mutations && alter_version != -1) + if (alter_version != -1) { - if (alter_version > params.metadata_version) + if (seen_all_metadata_mutations || alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version if (alter_version > params.min_part_metadata_version) { + /// Copy a pointer to the whole entry to avoid extracting and copying commands. + /// Required commands will be copied later only for specific parts. if (std::ranges::any_of(status->entry->commands, is_supported_command)) in_partition.emplace(mutation_version, status->entry); } @@ -2020,6 +2026,8 @@ MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapsh { if (!status->is_done) { + /// Copy a pointer to the whole entry to avoid extracting and copying commands. + /// Required commands will be copied later only for specific parts. if (std::ranges::any_of(status->entry->commands, is_supported_command)) in_partition.emplace(mutation_version, status->entry); } @@ -2114,7 +2122,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2171,7 +2179,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index a46fdaf3ac4..af8e2521f81 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,8 +154,8 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_data_mutations_to_apply = 0; + Int64 num_metadata_mutations_to_apply = 0; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; @@ -415,8 +415,9 @@ public: struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot { + public: MutationsSnapshot() = default; - explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} + MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {} using Params = MergeTreeData::IMutationsSnapshot::Params; using MutationsByPartititon = std::unordered_map>; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c7189925812..900b592d32d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -520,7 +520,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -556,7 +556,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands, lock); } } else @@ -838,7 +838,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); } to_kill.emplace(std::move(it->second)); @@ -923,7 +923,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2466,12 +2466,18 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - auto res = std::make_shared(params); - std::lock_guard lock(currently_processing_in_background_mutex); - bool need_data_mutations = res->params.need_data_mutations && data_mutations_to_apply > 0; - bool need_metadata_mutations = metadata_mutations_to_apply > 0; + MutationsSnapshot::Info info + { + .num_data_mutations = num_data_mutations_to_apply, + .num_metadata_mutations = num_metadata_mutations_to_apply, + }; + + auto res = std::make_shared(params, std::move(info)); + + bool need_data_mutations = res->params.need_data_mutations && num_data_mutations_to_apply > 0; + bool need_metadata_mutations = num_metadata_mutations_to_apply > 0; if (!need_data_mutations && !need_metadata_mutations) return res; @@ -2489,6 +2495,8 @@ MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const return false; }); + /// Copy a pointer to all commands to avoid extracting and copying them. + /// Required commands will be copied later only for specific parts. if (has_required_command) res->mutations_by_version.emplace(version, entry.commands); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index a3633284ac6..16ad79d7586 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,7 +17,6 @@ #include #include -#include "Storages/MutationCommands.h" namespace DB @@ -150,8 +149,8 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_data_mutations_to_apply = 0; + Int64 num_metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -314,7 +313,7 @@ private: struct MutationsSnapshot : public IMutationsSnapshot { MutationsSnapshot() = default; - explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} + MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {} using MutationsByVersion = std::map>; MutationsByVersion mutations_by_version; From 4ae0daf5d3149a2e9e4e8494e52164c91c27af0e Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 19 Jul 2024 18:46:37 +0000 Subject: [PATCH 0333/1722] output more info --- ...61_lightweight_delete_projection.reference | 14 +++++--- .../03161_lightweight_delete_projection.sql | 32 ++++++++++++++++--- 2 files changed, 38 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index cb623ea2b50..960fa1dcc33 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,14 +1,20 @@ compact part testing throw default mode testing drop mode +all_1_1_0_2 testing rebuild mode 8888 Alice 50 -p1 -p2 +all_1_1_0_4 +all_3_3_0_4 +p1 all_3_3_0_4 +p2 all_3_3_0_4 wide part testing throw default mode testing drop mode +all_1_1_0_2 testing rebuild mode 8888 Alice 50 -p1 -p2 +all_1_1_0_4 +all_3_3_0_4 +p1 all_3_3_0_4 +p2 all_3_3_0_4 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index f2d6dcb164f..f33653fc652 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -32,9 +32,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting no projection +-- all_1_1_0_2 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting no projection +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); @@ -49,9 +55,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting projection p1, p2 +-- all_1_1_0_4, all_3_3_0_4 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting projection p1, p2 +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); @@ -84,9 +96,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting no projection +-- all_1_1_0_2 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting no projection +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); @@ -101,9 +119,15 @@ SELECT * FROM users ORDER BY uid; SYSTEM FLUSH LOGS; --- expecting projection p1, p2 +-- all_1_1_0_4, all_3_3_0_4 SELECT name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); + +-- expecting projection p1, p2 +SELECT + name, parent_name FROM system.projection_parts WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); From 519494a9d08f4a45bd995b712808b908ace94c0c Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Jul 2024 18:51:20 +0000 Subject: [PATCH 0334/1722] Add tests, docs, implement new syntax sugar for Array(JSON), fix small ugs --- docs/en/operations/settings/settings.md | 7 + docs/en/sql-reference/data-types/json.md | 104 +- .../sql-reference/data-types/object-json.md | 2 +- .../sql-reference/data-types/object_json.md | 86 - .../sql-reference/functions/json-functions.md | 236 ++ src/Columns/ColumnObject.cpp | 2 +- src/Columns/ColumnObject.h | 5 +- src/DataTypes/DataTypeObject.cpp | 4 +- src/DataTypes/DataTypeObject.h | 7 +- .../Serializations/SerializationObject.cpp | 3 + .../SerializationObjectDynamicPath.cpp | 12 + .../SerializationVariantElement.cpp | 19 +- src/Formats/JSONExtractTree.cpp | 8 +- src/Functions/JSONEmpty.cpp | 2 +- src/Functions/JSONPaths.cpp | 228 +- src/Parsers/ExpressionElementParsers.cpp | 51 +- src/Parsers/ExpressionElementParsers.h | 1 + .../0_stateless/00727_concat.reference | 1 + tests/queries/0_stateless/00727_concat.sql | 2 + .../01825_new_type_json_10.reference | 14 + .../0_stateless/01825_new_type_json_10.sql | 16 + .../01825_new_type_json_11.reference | 13 + .../0_stateless/01825_new_type_json_11.sh | 64 + .../01825_new_type_json_12.reference | 14 + .../0_stateless/01825_new_type_json_12.sh | 54 + .../01825_new_type_json_13.reference | 4 + .../0_stateless/01825_new_type_json_13.sh | 50 + .../01825_new_type_json_18.reference | 2 + .../0_stateless/01825_new_type_json_18.sql | 16 + .../01825_new_type_json_2.reference | 24 + .../0_stateless/01825_new_type_json_2.sql | 41 + .../01825_new_type_json_3.reference.j2 | 59 + .../0_stateless/01825_new_type_json_3.sql.j2 | 64 + .../01825_new_type_json_6.reference | 10 + .../0_stateless/01825_new_type_json_6.sh | 59 + ...1825_new_type_json_add_column.reference.j2 | 8 + .../01825_new_type_json_add_column.sql.j2 | 23 + .../01825_new_type_json_btc.reference | 33 + .../0_stateless/01825_new_type_json_btc.sh | 31 + .../01825_new_type_json_ephemeral.reference | 1 + .../01825_new_type_json_ephemeral.sql | 18 + ...w_type_json_ghdata_insert_select.reference | 1 + ...1825_new_type_json_ghdata_insert_select.sh | 27 + .../01825_new_type_json_nbagames.reference | 65 + .../01825_new_type_json_nbagames.sh | 54 + ...25_new_type_json_parallel_insert.reference | 1 + .../01825_new_type_json_parallel_insert.sql | 10 + .../queries/0_stateless/01825_type_json_11.sh | 2 +- .../queries/0_stateless/01825_type_json_12.sh | 2 +- .../queries/0_stateless/01825_type_json_13.sh | 2 +- .../0_stateless/01825_type_json_14.sql | 2 +- .../queries/0_stateless/01825_type_json_15.sh | 2 +- .../queries/0_stateless/01825_type_json_16.sh | 2 +- .../0_stateless/01825_type_json_17.sql | 2 +- .../0_stateless/01825_type_json_3.sql.j2 | 2 +- .../queries/0_stateless/01825_type_json_4.sh | 2 +- .../queries/0_stateless/01825_type_json_5.sql | 2 +- .../queries/0_stateless/01825_type_json_6.sh | 2 +- .../queries/0_stateless/01825_type_json_7.sh | 2 +- .../queries/0_stateless/01825_type_json_8.sh | 2 +- .../queries/0_stateless/01825_type_json_9.sql | 2 +- .../01825_type_json_add_column.sql.j2 | 2 +- .../0_stateless/01825_type_json_bools.sql | 2 +- .../0_stateless/01825_type_json_btc.sh | 2 +- .../0_stateless/01825_type_json_describe.sql | 2 +- .../01825_type_json_distributed.sql | 2 +- .../01825_type_json_empty_string.sql | 2 +- .../0_stateless/01825_type_json_ephemeral.sql | 2 +- .../0_stateless/01825_type_json_field.sql | 2 +- .../0_stateless/01825_type_json_from_map.sql | 2 +- .../01825_type_json_ghdata_insert_select.sh | 4 +- .../0_stateless/01825_type_json_in_array.sql | 10 +- .../01825_type_json_in_other_types.sh | 2 +- .../01825_type_json_insert_select.sql | 4 +- .../01825_type_json_missed_values.sql | 2 +- .../01825_type_json_multiple_files.sh | 8 +- .../0_stateless/01825_type_json_mutations.sql | 2 +- .../0_stateless/01825_type_json_nbagames.sh | 2 +- .../0_stateless/01825_type_json_order_by.sql | 6 +- .../01825_type_json_parallel_insert.sql | 2 +- .../01825_type_json_partitions.sql | 2 +- .../01825_type_json_schema_race_long.sh | 2 +- .../0_stateless/01825_type_json_sparse.sql | 2 +- .../01825_type_json_wide_parts_merge.sql | 2 +- .../0_stateless/02242_subcolumns_sizes.sql | 2 +- .../0_stateless/02246_flatten_tuple.sql | 2 +- .../02286_tuple_numeric_identifier.sql | 2 +- .../02421_type_json_async_insert.sh | 2 +- .../02421_type_json_empty_parts.sh | 6 +- ...02482_json_nested_arrays_with_same_keys.sh | 2 +- .../0_stateless/02513_validate_data_types.sql | 4 +- .../02553_type_json_attach_partition.sql | 2 +- .../02553_type_object_analyzer.sql | 2 +- .../queries/0_stateless/02717_pretty_json.sql | 2 +- ...75_show_columns_called_from_clickhouse.sql | 2 +- ...2775_show_columns_called_from_mysql.expect | 2 +- .../0_stateless/02870_per_column_settings.sql | 2 +- ...2935_format_with_arbitrary_types.reference | 1 + .../02935_format_with_arbitrary_types.sql | 2 + .../02969_mysql_cast_type_aliases.reference | 4 +- .../02969_mysql_cast_type_aliases.sql | 2 +- .../03157_dynamic_type_json.reference | 8 +- .../0_stateless/03157_dynamic_type_json.sql | 4 +- .../03158_dynamic_type_from_variant.sql | 1 - .../03159_dynamic_type_all_types.sql | 1 - .../03205_json_cast_from_string.reference | 18 + .../03205_json_cast_from_string.sql | 22 + .../0_stateless/03205_json_syntax.reference | 0 .../queries/0_stateless/03205_json_syntax.sql | 23 + ...3206_json_parsing_and_formatting.reference | 195 ++ .../03206_json_parsing_and_formatting.sh | 56 + .../03207_json_read_subcolumns_1.reference | 2080 +++++++++++++++++ .../03207_json_read_subcolumns_1.sh | 125 + ..._subcolumns_2_compact_merge_tree.reference | 71 + ...on_read_subcolumns_2_compact_merge_tree.sh | 142 ++ ...07_json_read_subcolumns_2_memory.reference | 35 + .../03207_json_read_subcolumns_2_memory.sh | 137 ++ ...ead_subcolumns_2_wide_merge_tree.reference | 71 + ..._json_read_subcolumns_2_wide_merge_tree.sh | 142 ++ ..._array_of_json_read_subcolumns_1.reference | 560 +++++ .../03208_array_of_json_read_subcolumns_1.sh | 71 + ..._subcolumns_2_compact_merge_tree.reference | 65 + ...on_read_subcolumns_2_compact_merge_tree.sh | 73 + ...of_json_read_subcolumns_2_memory.reference | 32 + ..._array_of_json_read_subcolumns_2_memory.sh | 68 + ...ead_subcolumns_2_wide_merge_tree.reference | 65 + ..._json_read_subcolumns_2_wide_merge_tree.sh | 73 + ...3209_json_type_horizontal_merges.reference | 100 + .../03209_json_type_horizontal_merges.sh | 70 + .../03209_json_type_vertical_merges.reference | 100 + .../03209_json_type_vertical_merges.sh | 70 + ...03210_json_type_alter_add_column.reference | 75 + .../03210_json_type_alter_add_column.sh | 47 + .../03211_nested_json_merges.reference | 88 + .../0_stateless/03211_nested_json_merges.sh | 65 + 135 files changed, 6188 insertions(+), 258 deletions(-) delete mode 100644 docs/en/sql-reference/data-types/object_json.md create mode 100644 tests/queries/0_stateless/01825_new_type_json_10.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_10.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_11.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_11.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_12.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_12.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_13.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_13.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_18.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_18.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_2.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_2.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_3.reference.j2 create mode 100644 tests/queries/0_stateless/01825_new_type_json_3.sql.j2 create mode 100644 tests/queries/0_stateless/01825_new_type_json_6.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_6.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_add_column.reference.j2 create mode 100644 tests/queries/0_stateless/01825_new_type_json_add_column.sql.j2 create mode 100644 tests/queries/0_stateless/01825_new_type_json_btc.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_btc.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_ephemeral.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_ephemeral.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_nbagames.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_nbagames.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_parallel_insert.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_parallel_insert.sql create mode 100644 tests/queries/0_stateless/03205_json_cast_from_string.reference create mode 100644 tests/queries/0_stateless/03205_json_cast_from_string.sql create mode 100644 tests/queries/0_stateless/03205_json_syntax.reference create mode 100644 tests/queries/0_stateless/03205_json_syntax.sql create mode 100644 tests/queries/0_stateless/03206_json_parsing_and_formatting.reference create mode 100755 tests/queries/0_stateless/03206_json_parsing_and_formatting.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1.reference create mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_1.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference create mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference create mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference create mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference create mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference create mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference create mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference create mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh create mode 100644 tests/queries/0_stateless/03209_json_type_horizontal_merges.reference create mode 100755 tests/queries/0_stateless/03209_json_type_horizontal_merges.sh create mode 100644 tests/queries/0_stateless/03209_json_type_vertical_merges.reference create mode 100755 tests/queries/0_stateless/03209_json_type_vertical_merges.sh create mode 100644 tests/queries/0_stateless/03210_json_type_alter_add_column.reference create mode 100755 tests/queries/0_stateless/03210_json_type_alter_add_column.sh create mode 100644 tests/queries/0_stateless/03211_nested_json_merges.reference create mode 100755 tests/queries/0_stateless/03211_nested_json_merges.sh diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c3f697c3bdc..9ddfe985cee 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5608,3 +5608,10 @@ Default value: `10000000`. Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. Default value: `1GiB`. + +## use_json_alias_for_old_object_type + +When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/ob) type instead of the new [JSON](../../sql-reference/data-types/json.md) type. +This setting requires server restart to take effect when changed. + +Default value: `false`. diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index be75f909684..1001ad63999 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -19,7 +19,7 @@ To declare a column of `JSON` type, use the following syntax: JSON(max_dynamic_paths=N, max_dynamic_types=M, some.path TypeName, SKIP path.to.skip, SKIP REGEXP 'paths_regexp') ``` Where: -- `max_dynamic_paths` is an optional parameter indicating how many paths can be stored separately as subcolumns across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all other paths will be stored together in a single structure. Default value of `max_dynamic_paths` is `1000`. +- `max_dynamic_paths` is an optional parameter indicating how many paths can be stored separately as subcolumns across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all other paths will be stored together in a single structure. Default value of `max_dynamic_paths` is `1024`. - `max_dynamic_types` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a single path column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_dynamic_types` is `32`. - `some.path TypeName` is an optional type hint for particular path in the JSON. Such paths will be always stored as subcolumns with specified type. - `SKIP path.to.skip` is an optional hint for particular path that should be skipped during JSON parsing. Such paths will never be stored in the JSON column. If specified path is a nested JSON object, the whole nested object will be skipped. @@ -267,18 +267,18 @@ JSON paths that contains an array of objects are parsed as type `Array(JSON)` an ```sql CREATE TABLE test (json JSON) ENGINE = Memory; INSERT INTO test VALUES -('{"a" : {"b" : [{"c" : 42, "d" : "Hello", "f" : {"g" : 42.42}}, {"c" : 43}, {"e" : [1, 2, 3], "d" : "My", "f" : {"g" : 43.43, "h" : "2020-01-01"}}]}}'), +('{"a" : {"b" : [{"c" : 42, "d" : "Hello", "f" : [[{"g" : 42.42}]], "k" : {"j" : 1000}}, {"c" : 43}, {"e" : [1, 2, 3], "d" : "My", "f" : [[{"g" : 43.43, "h" : "2020-01-01"}]], "k" : {"j" : 2000}}]}}'), ('{"a" : {"b" : [1, 2, 3]}}'), -('{"a" : {"b" : [{"c" : 44, "f" : {"h" : "2020-01-02"}}, {"e" : [4, 5, 6], "d" : "World", "f" : {"g" : 44.44}}]}}'); +('{"a" : {"b" : [{"c" : 44, "f" : [[{"h" : "2020-01-02"}]]}, {"e" : [4, 5, 6], "d" : "World", "f" : [[{"g" : 44.44}]], "k" : {"j" : 3000}}]}}'); SELECT json FROM test; ``` -```text -┌─json──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┠-│ {"a":{"b":[{"c":"42","d":"Hello","f":{"g":42.42}},{"c":"43"},{"d":"My","e":["1","2","3"],"f":{"g":43.43,"h":"2020-01-01"}}]}} │ -│ {"a":{"b":["1","2","3"]}} │ -│ {"a":{"b":[{"c":"44","f":{"h":"2020-01-02"}},{"d":"World","e":["4","5","6"],"f":{"g":44.44}}]}} │ -└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +```text3 +┌─json────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┠+│ {"a":{"b":[{"c":"42","d":"Hello","f":[[{"g":42.42}]],"k":{"j":"1000"}},{"c":"43"},{"d":"My","e":["1","2","3"],"f":[[{"g":43.43,"h":"2020-01-01"}]],"k":{"j":"2000"}}]}} │ +│ {"a":{"b":["1","2","3"]}} │ +│ {"a":{"b":[{"c":"44","f":[[{"h":"2020-01-02"}]]},{"d":"World","e":["4","5","6"],"f":[[{"g":44.44}]],"k":{"j":"3000"}}]}} │ +└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` ```sql @@ -286,55 +286,87 @@ SELECT json.a.b, dynamicType(json.a.b) FROM test; ``` ```text -┌─json.a.b────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─dynamicType(json.a.b)───────────────────────────────────┠-│ ['{"c":"42","d":"Hello","f":{"g":42.42}}','{"c":"43"}','{"d":"My","e":["1","2","3"],"f":{"g":43.43,"h":"2020-01-01"}}'] │ Array(JSON(max_dynamic_types=8, max_dynamic_paths=125)) │ -│ [1,2,3] │ Array(Nullable(Int64)) │ -│ ['{"c":"44","f":{"h":"2020-01-02"}}','{"d":"World","e":["4","5","6"],"f":{"g":44.44}}'] │ Array(JSON(max_dynamic_types=8, max_dynamic_paths=125)) │ -└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────┘ +┌─json.a.b──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─dynamicType(json.a.b)────────────────────────────────────┠+│ ['{"c":"42","d":"Hello","f":[[{"g":42.42}]],"k":{"j":"1000"}}','{"c":"43"}','{"d":"My","e":["1","2","3"],"f":[[{"g":43.43,"h":"2020-01-01"}]],"k":{"j":"2000"}}'] │ Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)) │ +│ [1,2,3] │ Array(Nullable(Int64)) │ +│ ['{"c":"44","f":[[{"h":"2020-01-02"}]]}','{"d":"World","e":["4","5","6"],"f":[[{"g":44.44}]],"k":{"j":"3000"}}'] │ Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)) │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────────────────────────────────────────────────────────┘ ``` As you can notice, the `max_dynamic_types/max_dynamic_paths` parameters of the nested `JSON` type were reduced compared to the default values. It's needed to avoid number of subcolumns to grow uncontrolled on nested arrays of JSON objects. -Let's try to read subcolumns of this nested `JSON` column: +Let's try to read subcolumns from this nested `JSON` column: ```sql -SELECT json.a.b.:`Array(JSON)`.c, json.a.b.:`Array(JSON)`.f.g, json.a.b.:`Array(JSON)`.f.g FROM test; +SELECT json.a.b.:`Array(JSON)`.c, json.a.b.:`Array(JSON)`.f, json.a.b.:`Array(JSON)`.d FROM test; ``` ```text -┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f.g─┬─json.a.b.:`Array(JSON)`.f.g─┠-│ [42,43,NULL] │ [42.42,NULL,43.43] │ [42.42,NULL,43.43] │ -│ [] │ [] │ [] │ -│ [44,NULL] │ [NULL,44.44] │ [NULL,44.44] │ -└───────────────────────────┴─────────────────────────────┴─────────────────────────────┘ +┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f───────────────────────────────────┬─json.a.b.:`Array(JSON)`.d─┠+│ [42,43,NULL] │ [[['{"g":42.42}']],NULL,[['{"g":43.43,"h":"2020-01-01"}']]] │ ['Hello',NULL,'My'] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [[['{"h":"2020-01-02"}']],[['{"g":44.44}']]] │ [NULL,'World'] │ +└───────────────────────────┴─────────────────────────────────────────────────────────────┴───────────────────────────┘ ``` -We can also read subcolumns of `Dynamic` columns: +We can avoid writing `Array(JSON)` subcolumn name using special syntax: ```sql -SELECT json.a.b.:`Array(JSON)`.f.h.:Date FROM test; +SELECT json.a.b[].c, json.a.b[].f, json.a.b[].d FROM test; ``` ```text -┌─json.a.b.:`Array(JSON)`.f.h.:`Date`─┠-│ [NULL,NULL,'2020-01-01'] │ -│ [] │ -│ ['2020-01-02',NULL] │ -└─────────────────────────────────────┘ +┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f───────────────────────────────────┬─json.a.b.:`Array(JSON)`.d─┠+│ [42,43,NULL] │ [[['{"g":42.42}']],NULL,[['{"g":43.43,"h":"2020-01-01"}']]] │ ['Hello',NULL,'My'] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [[['{"h":"2020-01-02"}']],[['{"g":44.44}']]] │ [NULL,'World'] │ +└───────────────────────────┴─────────────────────────────────────────────────────────────┴───────────────────────────┘ +``` + +The number of `[]` after path indicates the array level. `json.path[][]` will be transformed to `json.path.:Array(Array(JSON))` + +Let's check the paths and types inside our `Array(JSON)`: + +```sql +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b[]))) FROM test; +``` + +```text +┌─arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b.:`Array(JSON)`)))──┠+│ ('c','Int64') │ +│ ('d','String') │ +│ ('f','Array(Array(JSON(max_dynamic_types=8, max_dynamic_paths=64)))') │ +│ ('k.j','Int64') │ +│ ('e','Array(Nullable(Int64))') │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +Let's read subcolumns from `Array(JSON)` column: + +```sql +SELECT json.a.b[].c.:Int64, json.a.b[].f[][].g.:Float64, json.a.b[].f[][].h.:Date FROM test; +``` + +```text +┌─json.a.b.:`Array(JSON)`.c.:`Int64`─┬─json.a.b.:`Array(JSON)`.f.:`Array(Array(JSON))`.g.:`Float64`─┬─json.a.b.:`Array(JSON)`.f.:`Array(Array(JSON))`.h.:`Date`─┠+│ [42,43,NULL] │ [[[42.42]],[],[[43.43]]] │ [[[NULL]],[],[['2020-01-01']]] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [[[NULL]],[[44.44]]] │ [[['2020-01-02']],[[NULL]]] │ +└────────────────────────────────────┴──────────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ ``` We can also read sub-object subcolumns from nested `JSON` column: ```sql -SELECT json.a.b.:`Array(JSON)`.^f FROM test +SELECT json.a.b[].^k FROM test ``` ```text -┌─json.a.b.:`Array(JSON)`.^`f`────────────────────────┠-│ ['{"g":42.42}','{}','{"g":43.43,"h":"2020-01-01"}'] │ -│ [] │ -│ ['{"h":"2020-01-02"}','{"g":44.44}'] │ -└─────────────────────────────────────────────────────┘ +┌─json.a.b.:`Array(JSON)`.^`k`─────────┠+│ ['{"j":"1000"}','{}','{"j":"2000"}'] │ +│ [] │ +│ ['{}','{"j":"3000"}'] │ +└──────────────────────────────────────┘ ``` ## Reading JSON type from the data @@ -386,7 +418,7 @@ SELECT json FROM format(TSV, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP REGEXP \' ## Reaching the limit of dynamic paths inside JSON -`JSON` data type can store only limited number of paths as separate subcolumns inside. By default, this limit is 1000, but you can change it in type declaration using parameter `max_dynamic_paths`. +`JSON` data type can store only limited number of paths as separate subcolumns inside. By default, this limit is 1024, but you can change it in type declaration using parameter `max_dynamic_paths`. When the limit is reached, all new paths inserted to `JSON` column will be stored in a single shared data structure. It's still possible to read such paths as subcolumns, but it will require reading the whole shared data structure to extract the values of this path. This limit is needed to avoid the enormous number of different subcolumns that can make the table unusable. @@ -470,7 +502,9 @@ SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) O As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and moved paths `e` and `d` to shared data structure. +## Introspection functions +There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes). ## Tips for better usage of the JSON type diff --git a/docs/en/sql-reference/data-types/object-json.md b/docs/en/sql-reference/data-types/object-json.md index 36835a42db8..c6fa6f0e882 100644 --- a/docs/en/sql-reference/data-types/object-json.md +++ b/docs/en/sql-reference/data-types/object-json.md @@ -13,7 +13,7 @@ keywords: [object, data type] Stores JavaScript Object Notation (JSON) documents in a single column. -`JSON` is an alias for `Object('json')`. +`JSON` can be used as an alias to `Object('json')` when setting [use_json_alias_for_old_object_type](../../operations/settings/settings.md#usejsonaliasforoldobjecttype) is enabled. ## Example diff --git a/docs/en/sql-reference/data-types/object_json.md b/docs/en/sql-reference/data-types/object_json.md deleted file mode 100644 index 39e37abad82..00000000000 --- a/docs/en/sql-reference/data-types/object_json.md +++ /dev/null @@ -1,86 +0,0 @@ ---- -slug: /en/sql-reference/data-types/json -sidebar_position: 26 -sidebar_label: JSON ---- - -# JSON - -:::note -This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. -::: - -Stores JavaScript Object Notation (JSON) documents in a single column. - -`JSON` is an alias for `Object('json')`. - -:::note -The JSON data type is an obsolete feature. Do not use it. -If you want to use it, set `allow_experimental_object_type = 1`. -::: - -## Example - -**Example 1** - -Creating a table with a `JSON` column and inserting data into it: - -```sql -CREATE TABLE json -( - o JSON -) -ENGINE = Memory -``` - -```sql -INSERT INTO json VALUES ('{"a": 1, "b": { "c": 2, "d": [1, 2, 3] }}') -``` - -```sql -SELECT o.a, o.b.c, o.b.d[3] FROM json -``` - -```text -┌─o.a─┬─o.b.c─┬─arrayElement(o.b.d, 3)─┠-│ 1 │ 2 │ 3 │ -└─────┴───────┴────────────────────────┘ -``` - -**Example 2** - -To be able to create an ordered `MergeTree` family table the sorting key has to be extracted into its column. For example, to insert a file of compressed HTTP access logs in JSON format: - -```sql -CREATE TABLE logs -( - timestamp DateTime, - message JSON -) -ENGINE = MergeTree -ORDER BY timestamp -``` - -```sql -INSERT INTO logs -SELECT parseDateTimeBestEffort(JSONExtractString(json, 'timestamp')), json -FROM file('access.json.gz', JSONAsString) -``` - -## Displaying JSON columns - -When displaying a `JSON` column ClickHouse only shows the field values by default (because internally, it is represented as a tuple). You can display the field names as well by setting `output_format_json_named_tuples_as_objects = 1`: - -```sql -SET output_format_json_named_tuples_as_objects = 1 - -SELECT * FROM json FORMAT JSONEachRow -``` - -```text -{"o":{"a":1,"b":{"c":2,"d":[1,2,3]}}} -``` - -## Related Content - -- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 7bff6a6cba5..b02dc3d1d8a 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -1155,3 +1155,239 @@ SELECT jsonMergePatch('{"a":1}', '{"name": "joey"}', '{"name": "tom"}', '{"name" │ {"a":1,"name":"zoey"} │ └───────────────────────┘ ``` + +### JSONAllPaths + +Returns the list of all paths stored in each row in [JSON](../data-types/json.md) column. + +**Syntax** + +``` sql +JSONAllPaths(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- An array of paths. [Array(String)](../data-types/array.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONAllPaths(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONAllPaths(json)─┠+│ {"a":"42"} │ ['a'] │ +│ {"b":"Hello"} │ ['b'] │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ ['a','c'] │ +└──────────────────────────────────────┴────────────────────┘ +``` + +### JSONAllPathsWithTypes + +Returns the map of all paths and their data types stored in each row in [JSON](../data-types/json.md) column. + +**Syntax** + +``` sql +JSONAllPathsWithTypes(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- An array of paths. [Map(String, String)](../data-types/array.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONAllPathsWithTypes(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONAllPathsWithTypes(json)───────────────┠+│ {"a":"42"} │ {'a':'Int64'} │ +│ {"b":"Hello"} │ {'b':'String'} │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'a':'Array(Nullable(Int64))','c':'Date'} │ +└──────────────────────────────────────┴───────────────────────────────────────────┘ +``` + +### JSONDynamicPaths + +Returns the list of dynamic paths that are stored as separate subcolumns in [JSON](../data-types/json.md) column. + +**Syntax** + +``` sql +JSONDynamicPaths(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- An array of paths. [Array(String)](../data-types/array.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONDynamicPaths(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONDynamicPaths(json)─┠+| {"a":"42"} │ ['a'] │ +│ {"b":"Hello"} │ [] │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ ['a'] │ +└──────────────────────────────────────┴────────────────────────┘ +``` + +### JSONDynamicPathsWithTypes + +Returns the map of dynamic paths that are stored as separate subcolumns and their types in each row in [JSON](../data-types/json.md) column. + +**Syntax** + +``` sql +JSONAllPathsWithTypes(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- An array of paths. [Map(String, String)](../data-types/array.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONDynamicPathsWithTypes(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONDynamicPathsWithTypes(json)─┠+│ {"a":"42"} │ {'a':'Int64'} │ +│ {"b":"Hello"} │ {} │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'a':'Array(Nullable(Int64))'} │ +└──────────────────────────────────────┴─────────────────────────────────┘ +``` + +### JSONSharedDataPaths + +Returns the list of paths that are stored in shared data structure in [JSON](../data-types/json.md) column. + +**Syntax** + +``` sql +JSONSharedDataPaths(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- An array of paths. [Array(String)](../data-types/array.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONSharedDataPaths(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONSharedDataPaths(json)─┠+│ {"a":"42"} │ [] │ +│ {"b":"Hello"} │ ['b'] │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ ['c'] │ +└──────────────────────────────────────┴───────────────────────────┘ +``` + +### JSONSharedDataPathsWithTypes + +Returns the map of paths that are stored in shared data structure and their types in each row in [JSON](../data-types/json.md) column. + +**Syntax** + +``` sql +JSONSharedDataPathsWithTypes(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- An array of paths. [Map(String, String)](../data-types/array.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONSharedDataPathsWithTypes(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONSharedDataPathsWithTypes(json)─┠+│ {"a":"42"} │ {} │ +│ {"b":"Hello"} │ {'b':'String'} │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'c':'Date'} │ +└──────────────────────────────────────┴────────────────────────────────────┘ +``` + +### JSONEmpty + +Checks whether the input [JSON](../data-types/json.md) object is empty. + +``` sql +JSONEmpty(json) +``` + +**Arguments** + +- `json` — [JSON](../data-types/json.md). + +**Returned value** + +- Returns `1` for an empty JSON object or `0` for a non-empty JSON object. [UInt8](../data-types/int-uint.md). + +**Example** + +``` sql +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {}}, {"json" : {"a" : [1, 2, 3], "b" : "2020-01-01"}}, {"json" : {}}, +SELECT json, JSONEmpty(json) FROM test; +``` + +```text +┌─json─────────────────────────────────┬─JSONEmpty(json)─┠+│ {} │ 1 │ +│ {"a":["1","2","3"],"b":"2020-01-01"} │ 0 │ +│ {} │ 1 │ +└──────────────────────────────────────┴─────────────────┘ +``` diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index bd6f759ee30..f9a3af601e9 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -424,7 +424,7 @@ void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t len void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) #endif { - /// TODO: try to parallelize doInsertRangeFrom over typed/dynamic paths. + /// TODO: try to parallelize doInsertRangeFrom over typed/dynamic paths if it makes sense. const auto & src_object_column = assert_cast(src); /// First, insert typed paths, they must be the same for both columns. diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 72ec20bf534..b130c7f4468 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -101,14 +101,15 @@ public: ColumnPtr replicate(const Offsets & replicate_offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; + void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &) const override; + void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} + /// Values of ColumnObject are not comparable. #if !defined(ABORT_ON_LOGICAL_ERROR) int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } #else int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } #endif - void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &) const override; - void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} void getExtremes(Field & min, Field & max) const override { min = Object(); diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 0e7b46ffbd2..bae1d1935b5 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -14,9 +14,7 @@ #include #include #include - #include - #if USE_SIMDJSON #include #endif @@ -480,7 +478,7 @@ static DataTypePtr createJSON(const ASTPtr & arguments) void registerDataTypeJSON(DataTypeFactory & factory) { if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) - factory.registerDataType("JSON", createJSON); + factory.registerDataType("JSON", createJSON, DataTypeFactory::CaseInsensitive); } } diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 61364dfe5ed..2984078370b 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -24,7 +24,7 @@ public: }; /// Don't change these constants, it can break backward compatibility. - static constexpr size_t DEFAULT_MAX_SEPARATELY_STORED_PATHS = 1000; + static constexpr size_t DEFAULT_MAX_SEPARATELY_STORED_PATHS = 1024; static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR = 4; static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR = 2; @@ -44,10 +44,7 @@ public: MutableColumnPtr createColumn() const override; - Field getDefault() const override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDefault() is not implemented for data type {}", getName()); - } + Field getDefault() const override { return Object(); } bool isParametric() const override { return true; } bool canBeInsideNullable() const override { return false; } diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 0fa1ef7d2ca..126c2ffec6a 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -423,6 +423,9 @@ void SerializationObject::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { + if (!state) + return; + auto * object_state = checkAndGetState(state); auto * structure_state = checkAndGetState(object_state->structure_state); auto mutable_column = column->assumeMutable(); diff --git a/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp b/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp index 3a635671978..5323079c54b 100644 --- a/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp +++ b/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp @@ -147,6 +147,18 @@ void SerializationObjectDynamicPath::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } /// Otherwise, read the whole shared data column and extract requested path from it. + /// TODO: We can read several subcolumns of the same path located in the shared data + /// and right now we extract the whole path column from shared data every time + /// and then extract the requested subcolumns. We can optimize it and use substreams + /// cache here to avoid extracting the same path from shared data several times. + /// + /// TODO: We can change the serialization of shared data to optimize reading paths from it. + /// Right now we cannot know if shared data contains our path in current range or not, + /// but we can change the serialization and write the list of all paths stored in shared + /// data before each granule, and then replace the column that stores paths with column + /// with indexes in this list. It can also reduce the storage, because we will store + /// each path only once and can replace UInt64 string offset column with indexes column + /// that can have smaller type depending on the number of paths in the list. else { settings.path.push_back(Substream::ObjectSharedData); diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 8ceab17cba4..df70c6394f4 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } struct SerializationVariantElement::DeserializeBinaryBulkStateVariantElement : public ISerialization::DeserializeBinaryBulkState @@ -188,27 +189,21 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( assert_cast(*variant_element_state->variant->assumeMutable()).nestedRemoveNullable(); } - /// If nothing to deserialize, just insert defaults. - if (variant_limit == 0) - { - mutable_column->insertManyDefaults(num_new_discriminators); - return; - } - addVariantToPath(settings.path); nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, *variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); - /// If nothing was deserialized when variant_limit > 0 - /// it means that we don't have a stream for such sub-column. - /// It may happen during ALTER MODIFY column with Variant extension. - /// In this case we should just insert default values. - if (variant_element_state->variant->empty()) + /// If there was nothing to deserialize or nothing was actually deserialized when variant_limit > 0, just insert defaults. + /// The second case means that we don't have a stream for such sub-column. It may happen during ALTER MODIFY column with Variant extension. + if (variant_limit == 0 || variant_element_state->variant->empty()) { mutable_column->insertManyDefaults(num_new_discriminators); return; } + if (variant_element_state->variant->size() < *variant_limit) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of deserialized variant column less than the limit: {} < {}", variant_element_state->variant->size(), *variant_limit); + size_t variant_offset = variant_element_state->variant->size() - *variant_limit; /// If we have only our discriminator in range, insert the whole range to result column. diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index b19cf61e207..e2c127b228d 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1510,7 +1510,7 @@ private: } case ElementType::OBJECT: { - return getObjectType(); + return std::make_shared(DataTypeObject::SchemaFormat::JSON, max_dynamic_paths_for_object, max_dynamic_types_for_object); } } } @@ -1548,12 +1548,6 @@ private: return float64_type; } - const DataTypePtr & getObjectType() const - { - static const DataTypePtr object_type = std::make_shared(DataTypeObject::SchemaFormat::JSON, max_dynamic_paths_for_object, max_dynamic_types_for_object); - return object_type; - } - const DataTypePtr & getNullType() const { static const DataTypePtr null_type = std::make_shared(std::make_shared()); diff --git a/src/Functions/JSONEmpty.cpp b/src/Functions/JSONEmpty.cpp index e932074ccb8..cdd1ac2f237 100644 --- a/src/Functions/JSONEmpty.cpp +++ b/src/Functions/JSONEmpty.cpp @@ -99,7 +99,7 @@ REGISTER_FUNCTION(JSONEmpty) { factory.registerFunction(FunctionDocumentation{ .description = R"( -Checks whether thee input JSON object is empty. +Checks whether the input JSON object is empty. )", .syntax = {"JSONEmpty(json)"}, .arguments = {{"json", "JSON column"}}, diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index b7a161901f0..0bf6b9ffe12 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -76,6 +76,7 @@ struct JSONSharedDataPathsWithTypesImpl }; /// Implements functions that extracts paths and types from JSON object column. +/// Used for introspection of the content of the JSON object column. template class FunctionJSONPaths : public IFunction { @@ -95,9 +96,12 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override { - if (data_types.size() != 1 || data_types[0]->getTypeId() != TypeIndex::Object) + if (data_types.size() != 1 ) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single argument with type JSON", getName()); + if (data_types[0]->getTypeId() != TypeIndex::Object) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires argument with type JSON, got: {}", getName(),data_types[0]->getName()); + if constexpr (Impl::with_types) return std::make_shared(std::make_shared(), std::make_shared()); return std::make_shared(std::make_shared()); @@ -142,14 +146,23 @@ private: /// We want the resulting arrays of paths to be sorted for consistency. std::sort(dynamic_paths.begin(), dynamic_paths.end()); - for (const auto & path : dynamic_paths) - data.insertData(path.data(), path.size()); - offsets.push_back(data.size()); - return res->replicate(IColumn::Offsets(1, column_object.size())); + size_t size = column_object.size(); + for (size_t i = 0; i != size; ++i) + { + for (const auto & path : dynamic_paths) + { + /// Don't include path if it contains NULL, because we consider + /// it to be equivalent to the absense of this path in this row. + if (!dynamic_path_columns.at(path)->isNullAt(i)) + data.insertData(path.data(), path.size()); + } + offsets.push_back(data.size()); + } + return res; } /// Collect all paths: typed, dynamic and paths from shared data. - std::vector sorted_dynamic_and_typed_paths; + std::vector sorted_dynamic_and_typed_paths; const auto & typed_path_columns = column_object.getTypedPaths(); const auto & dynamic_path_columns = column_object.getDynamicPaths(); for (const auto & [path, _] : typed_path_columns) @@ -174,7 +187,9 @@ private: while (sorted_paths_index != sorted_dynamic_and_typed_paths.size() && sorted_dynamic_and_typed_paths[sorted_paths_index] < shared_data_path) { auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; - data.insertData(path.data, path.size); + /// If it's dynamic path include it only if it's not NULL. + if (auto it = dynamic_path_columns.find(path); it == dynamic_path_columns.end() || !it->second->isNullAt(i)) + data.insertData(path.data(), path.size()); ++sorted_paths_index; } @@ -184,7 +199,8 @@ private: for (; sorted_paths_index != sorted_dynamic_and_typed_paths.size(); ++sorted_paths_index) { auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; - data.insertData(path.data, path.size); + if (auto it = dynamic_path_columns.find(path); it == dynamic_path_columns.end() || !it->second->isNullAt(i)) + data.insertData(path.data(), path.size()); } offsets.push_back(data.size()); @@ -215,12 +231,16 @@ private: { for (auto & path : sorted_dynamic_paths) { - auto type = getDynamicValueType(dynamic_path_columns.at(path), i); - paths_column->insertData(path.data(), path.size()); - types_column->insertData(type.data(), type.size()); + auto column = dynamic_path_columns.at(path); + if (!column->isNullAt(i)) + { + auto type = getDynamicValueType(column, i); + paths_column->insertData(path.data(), path.size()); + types_column->insertData(type.data(), type.size()); + } } - offsets.push_back(types_column->size()); + offsets.push_back(paths_column->size()); } return ColumnMap::create(ColumnPtr(std::move(paths_column)), ColumnPtr(std::move(types_column)), ColumnPtr(std::move(offsets_column))); @@ -237,9 +257,11 @@ private: size_t end = shared_data_offsets[ssize_t(i)]; for (size_t j = start; j != end; ++j) { - paths_column->insertFrom(*shared_data_paths, j); - auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j)); - types_column->insertData(type_name.data(), type_name.size()); + if (auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j))) + { + paths_column->insertFrom(*shared_data_paths, j); + types_column->insertData(type_name->data(), type_name->size()); + } } offsets.push_back(paths_column->size()); @@ -273,28 +295,44 @@ private: { auto shared_data_path = shared_data_paths->getDataAt(j); auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j)); + /// Skip NULL values. + if (!type_name) + continue; while (sorted_paths_index != sorted_typed_and_dynamic_paths_with_types.size() && sorted_typed_and_dynamic_paths_with_types[sorted_paths_index].first < shared_data_path) { auto & [path, type] = sorted_typed_and_dynamic_paths_with_types[sorted_paths_index]; - paths_column->insertData(path.data(), path.size()); /// Update type for path from dynamic paths. if (auto it = dynamic_path_columns.find(path); it != dynamic_path_columns.end()) + { + /// Skip NULL values. + if (it->second->isNullAt(i)) + { + ++sorted_paths_index; + continue; + } type = getDynamicValueType(it->second, i); + } + paths_column->insertData(path.data(), path.size()); types_column->insertData(type.data(), type.size()); ++sorted_paths_index; } paths_column->insertData(shared_data_path.data, shared_data_path.size); - types_column->insertData(type_name.data(), type_name.size()); + types_column->insertData(type_name->data(), type_name->size()); } for (; sorted_paths_index != sorted_typed_and_dynamic_paths_with_types.size(); ++sorted_paths_index) { auto & [path, type] = sorted_typed_and_dynamic_paths_with_types[sorted_paths_index]; - paths_column->insertData(path.data(), path.size()); if (auto it = dynamic_path_columns.find(path); it != dynamic_path_columns.end()) + { + /// Skip NULL values. + if (it->second->isNullAt(i)) + continue; type = getDynamicValueType(it->second, i); + } + paths_column->insertData(path.data(), path.size()); types_column->insertData(type.data(), type.size()); } @@ -310,17 +348,18 @@ private: const auto & variant_info = dynamic_column->getVariantInfo(); const auto & variant_column = dynamic_column->getVariantColumn(); auto global_discr = variant_column.globalDiscriminatorAt(i); - if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) - return "None"; - + /// We don't output path with NULL values. It should be checked before calling getDynamicValueType. + chassert(global_discr != ColumnVariant::NULL_DISCRIMINATOR); return variant_info.variant_names[global_discr]; } - String getDynamicValueTypeFromSharedData(StringRef value) const + std::optional getDynamicValueTypeFromSharedData(StringRef value) const { ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); - return isNothing(type) ? "None" : type->getName(); + if (isNothing(type)) + return std::nullopt; + return type->getName(); } }; @@ -328,12 +367,143 @@ private: REGISTER_FUNCTION(JSONPaths) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>(FunctionDocumentation{ + .description = R"( +Returns the list of all paths stored in each row in JSON column. +)", + .syntax = {"JSONAllPaths(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONAllPaths(json) FROM test; +)", + R"( +┌─json─────────────────────────────────┬─JSONAllPaths(json)─┠+│ {"a":"42"} │ ['a'] │ +│ {"b":"Hello"} │ ['b'] │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ ['a','c'] │ +└──────────────────────────────────────┴────────────────────┘ +)"}}}, + .categories{"JSON"}, + }); + + factory.registerFunction>(FunctionDocumentation{ + .description = R"( +Returns the list of all paths and their data types stored in each row in JSON column. +)", + .syntax = {"JSONAllPathsWithTypes(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONAllPathsWithTypes(json) FROM test; +)", + R"( +┌─json─────────────────────────────────┬─JSONAllPathsWithTypes(json)───────────────┠+│ {"a":"42"} │ {'a':'Int64'} │ +│ {"b":"Hello"} │ {'b':'String'} │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'a':'Array(Nullable(Int64))','c':'Date'} │ +└──────────────────────────────────────┴───────────────────────────────────────────┘ +)"}}}, + .categories{"JSON"}, + }); + + factory.registerFunction>(FunctionDocumentation{ + .description = R"( +Returns the list of dynamic paths that are stored as separate subcolumns in JSON column. +)", + .syntax = {"JSONDynamicPaths(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONDynamicPaths(json) FROM test; +)", + R"( +┌─json─────────────────────────────────┬─JSONDynamicPaths(json)─┠+│ {"a":"42"} │ ['a'] │ +│ {"b":"Hello"} │ [] │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ ['a'] │ +└──────────────────────────────────────┴────────────────────────┘ +)"}}}, + .categories{"JSON"}, + }); + + factory.registerFunction>(FunctionDocumentation{ + .description = R"( +Returns the list of dynamic paths that are stored as separate subcolumns and their types in each row in JSON column. +)", + .syntax = {"JSONDynamicPathsWithTypes(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONDynamicPathsWithTypes(json) FROM test; +)", + R"( +┌─json─────────────────────────────────┬─JSONDynamicPathsWithTypes(json)─┠+│ {"a":"42"} │ {'a':'Int64'} │ +│ {"b":"Hello"} │ {} │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'a':'Array(Nullable(Int64))'} │ +└──────────────────────────────────────┴─────────────────────────────────┘ +)"}}}, + .categories{"JSON"}, + }); + + factory.registerFunction>(FunctionDocumentation{ + .description = R"( +Returns the list of paths that are stored in shared data structure in JSON column. +)", + .syntax = {"JSONDynamicPaths(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONSharedDataPaths(json) FROM test; +)", + R"( +┌─json─────────────────────────────────┬─JSONSharedDataPaths(json)─┠+│ {"a":"42"} │ [] │ +│ {"b":"Hello"} │ ['b'] │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ ['c'] │ +└──────────────────────────────────────┴───────────────────────────┘ +)"}}}, + .categories{"JSON"}, + }); + + factory.registerFunction>(FunctionDocumentation{ + .description = R"( +Returns the list of paths that are stored in shared data structure and their types in each row in JSON column. +)", + .syntax = {"JSONDynamicPathsWithTypes(json)"}, + .arguments = {{"json", "JSON column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (json JSON(max_dynamic_paths=1)) ENGINE = Memory; +INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {"b" : "Hello"}}, {"json" : {"a" : [1, 2, 3], "c" : "2020-01-01"}} +SELECT json, JSONDynamicPathsWithTypes(json) FROM test; +)", + R"( +┌─json─────────────────────────────────┬─JSONDynamicPathsWithTypes(json)─┠+│ {"a":"42"} │ {'a':'Int64'} │ +│ {"b":"Hello"} │ {} │ +│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'a':'Array(Nullable(Int64))'} │ +└──────────────────────────────────────┴─────────────────────────────────┘ +)"}}}, + .categories{"JSON"}, + }); } } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 0e0b53c52d9..e0ac2ee6bed 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -282,6 +282,45 @@ bool ParserTableAsStringLiteralIdentifier::parseImpl(Pos & pos, ASTPtr & node, E return true; } +namespace +{ + +/// Parser of syntax sugar for reading JSON subcolumns of type Array(JSON): +/// json.a.b[][].c -> json.a.b.:Array(Array(JSON)).c +class ParserArrayOfJSONIdentifierAddition : public IParserBase +{ +public: + String getLastArrayOfJSONSubcolumnIdentifier() const + { + String subcolumn = ":`"; + for (size_t i = 0; i != last_array_level; ++i) + subcolumn += "Array("; + subcolumn += "JSON"; + for (size_t i = 0; i != last_array_level; ++i) + subcolumn += ")"; + return subcolumn + "`"; + } + +protected: + const char * getName() const override { return "ParserArrayOfJSONIdentifierDelimiter"; } + + bool parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected) override + { + last_array_level = 0; + ParserTokenSequence brackets_parser(std::vector{TokenType::OpeningSquareBracket, TokenType::ClosingSquareBracket}); + if (!brackets_parser.check(pos, expected)) + return false; + ++last_array_level; + while (brackets_parser.check(pos, expected)) + ++last_array_level; + return true; + } + +private: + size_t last_array_level; +}; + +} bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -290,6 +329,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex delimiter_parsers.emplace_back(std::make_unique(std::vector{TokenType::Dot, TokenType::Colon}), SpecialDelimiter::JSON_PATH_DYNAMIC_TYPE); delimiter_parsers.emplace_back(std::make_unique(std::vector{TokenType::Dot, TokenType::Caret}), SpecialDelimiter::JSON_PATH_PREFIX); delimiter_parsers.emplace_back(std::make_unique(TokenType::Dot), SpecialDelimiter::NONE); + ParserArrayOfJSONIdentifierAddition array_of_json_identifier_addition; std::vector parts; SpecialDelimiter last_special_delimiter = SpecialDelimiter::NONE; @@ -308,16 +348,23 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex break; } - is_first = false; - if (last_special_delimiter != SpecialDelimiter::NONE) + { parts.push_back(static_cast(last_special_delimiter) + backQuote(getIdentifierName(element))); + } else + { parts.push_back(getIdentifierName(element)); + /// Check if we have Array of JSON subcolumn additioon after identifier + /// and replace it with corresponding type subcolumn. + if (!is_first && array_of_json_identifier_addition.check(pos, expected)) + parts.push_back(array_of_json_identifier_addition.getLastArrayOfJSONSubcolumnIdentifier()); + } if (parts.back().empty()) params.push_back(element->as()->getParam()); + is_first = false; begin = pos; bool parsed_delimiter = false; for (const auto & [parser, special_delimiter] : delimiter_parsers) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 39608bc583e..e98920acc01 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -56,6 +56,7 @@ protected: * There is also special delimiters `.:` and `.^` for JSON type subcolumns. In case of special delimiter * the next identifier part after it will include special delimiter and be back quoted always: json.a.b.:UInt32 -> ['json', 'a', 'b', ':`UInt32`']. * It's needed to distinguish identifiers json.a.b.:UInt32 and json.a.b.`:UInt32`. + * There is also a special syntax sugar for reading JSON subcolumns of type Array(JSON): json.a.b[][].c -> json.a.b.:Array(Array(JSON)).c */ class ParserCompoundIdentifier : public IParserBase { diff --git a/tests/queries/0_stateless/00727_concat.reference b/tests/queries/0_stateless/00727_concat.reference index 329ad36ad3c..a93bf12b77a 100644 --- a/tests/queries/0_stateless/00727_concat.reference +++ b/tests/queries/0_stateless/00727_concat.reference @@ -34,6 +34,7 @@ With 2023-11-14 05:50:12.123 With hallo With [\'foo\',\'bar\'] With {"foo":"bar"} +With {"foo":"bar"} With (42,\'foo\') With {42:\'foo\'} With 122.233.64.201 diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index 76dae541261..65cd019cc13 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -2,6 +2,7 @@ -- no-fasttest: json type needs rapidjson library, geo types need s2 geometry SET allow_experimental_object_type = 1; +SET allow_experimental_json_type = 1; SET allow_suspicious_low_cardinality_types=1; SELECT '-- Const string + non-const arbitrary type'; @@ -40,6 +41,7 @@ SELECT concat('With ', materialize('2023-11-14 05:50:12.123' :: DateTime64(3, 'E SELECT concat('With ', materialize('hallo' :: Enum('hallo' = 1))); SELECT concat('With ', materialize(['foo', 'bar'] :: Array(String))); SELECT concat('With ', materialize('{"foo": "bar"}' :: JSON)); +SELECT concat('With ', materialize('{"foo": "bar"}' :: Object('json'))); SELECT concat('With ', materialize((42, 'foo') :: Tuple(Int32, String))); SELECT concat('With ', materialize(map(42, 'foo') :: Map(Int32, String))); SELECT concat('With ', materialize('122.233.64.201' :: IPv4)); diff --git a/tests/queries/0_stateless/01825_new_type_json_10.reference b/tests/queries/0_stateless/01825_new_type_json_10.reference new file mode 100644 index 00000000000..0dffa3c46f9 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_10.reference @@ -0,0 +1,14 @@ +('a.b','Int64') +('a.c','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') +('a.c','Array(Nullable(String))') +('e','Array(Nullable(Int64))') +('f','Int64') +('d','Int64') +{"o":{"a":{"b":"1","c":[{"d":"10","e":["31"]},{"d":"20","e":["63","127"]}]}}} +{"o":{"a":{"b":"2","c":[]}}} +{"o":{"a":{"b":"3","c":[{"e":["32"],"f":"20"},{"e":["64","128"],"f":"30"}]}}} +{"o":{"a":{"b":"4","c":[]}}} +1 [10,20] [[31],[63,127]] [NULL,NULL] +2 [] [] [] +3 [NULL,NULL] [[32],[64,128]] [20,30] +4 [] [] [] diff --git a/tests/queries/0_stateless/01825_new_type_json_10.sql b/tests/queries/0_stateless/01825_new_type_json_10.sql new file mode 100644 index 00000000000..0e599b0ac31 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_10.sql @@ -0,0 +1,16 @@ +-- Tags: no-fasttest + +SET allow_experimental_json_type = 1; + +DROP TABLE IF EXISTS t_json_10; +CREATE TABLE t_json_10 (o JSON) ENGINE = Memory; + +INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 1, "c": [{"d": 10, "e": [31]}, {"d": 20, "e": [63, 127]}]}} {"a": {"b": 2, "c": []}} +INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 3, "c": [{"f": 20, "e": [32]}, {"f": 30, "e": [64, 128]}]}} {"a": {"b": 4, "c": []}} + +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(o)) FROM t_json_10; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(o.a.c.:`Array(JSON)`))) FROM t_json_10; +SELECT o FROM t_json_10 ORDER BY o.a.b FORMAT JSONEachRow; +SELECT o.a.b, o.a.c.:`Array(JSON)`.d, o.a.c.:`Array(JSON)`.e, o.a.c.:`Array(JSON)`.f FROM t_json_10 ORDER BY o.a.b; + +DROP TABLE t_json_10; diff --git a/tests/queries/0_stateless/01825_new_type_json_11.reference b/tests/queries/0_stateless/01825_new_type_json_11.reference new file mode 100644 index 00000000000..aa3375a23cb --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_11.reference @@ -0,0 +1,13 @@ +('id','Int64') +('key_1','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') +('key_2','Int64') +('key_3','Array(JSON(max_dynamic_types=8, max_dynamic_paths=64))') +('key_4','Array(JSON(max_dynamic_types=4, max_dynamic_paths=16))') +('key_7','Int64') +('key_5','Int64') +{"obj":{"id":"1","key_1":[{"key_2":"100","key_3":[{"key_4":[{"key_5":"-2"}],"key_7":"257"}]},{"key_2":"65536"}]}} +{"obj":{"id":"2","key_1":[{"key_2":"101","key_3":[{"key_4":[{"key_5":"-2"}]}]},{"key_2":"102","key_3":[{"key_7":"257"}]},{"key_2":"65536"}]}} +{"obj.key_1.:`Array(JSON)`.key_3":[[{"key_4":[{"key_5":"-2"}],"key_7":"257"}],null]} +{"obj.key_1.:`Array(JSON)`.key_3":[[{"key_4":[{"key_5":"-2"}]}],[{"key_7":"257"}],null]} +[[[-2]],[]] [[257],[]] +[[[-2]],[[]],[]] [[NULL],[257],[]] diff --git a/tests/queries/0_stateless/01825_new_type_json_11.sh b/tests/queries/0_stateless/01825_new_type_json_11.sh new file mode 100755 index 00000000000..2eb6c1768fd --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_11.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_11" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_11 (obj JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_json_type 1 + +cat < notEmpty(x), outpoints)" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS btc" + +rm ${CLICKHOUSE_USER_FILES_UNIQUE}/btc_transactions.json diff --git a/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference b/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference new file mode 100644 index 00000000000..e8891cf6a56 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference @@ -0,0 +1 @@ +PushEvent some-repo {"actor":{"avatar_url":"https:\\/\\/avatars.githubusercontent.com\\/u\\/123213213?","display_login":"github-actions","gravatar_id":"","id":"123123123","login":"github-actions[bot]","url":"https:\\/\\/api.github.com\\/users\\/github-actions[bot]"},"created_at":"2022-01-04 07:00:00.000000000","repo":{"id":"1001001010101","name":"some-repo","url":"https:\\/\\/api.github.com\\/repos\\/some-repo"},"type":"PushEvent"} diff --git a/tests/queries/0_stateless/01825_new_type_json_ephemeral.sql b/tests/queries/0_stateless/01825_new_type_json_ephemeral.sql new file mode 100644 index 00000000000..4aaebfd326f --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_ephemeral.sql @@ -0,0 +1,18 @@ + +SET allow_experimental_json_type = 1; + +DROP TABLE IF EXISTS t_github_json; + +CREATE table t_github_json +( + event_type LowCardinality(String) DEFAULT JSONExtractString(message_raw, 'type'), + repo_name LowCardinality(String) DEFAULT JSONExtractString(message_raw, 'repo', 'name'), + message JSON DEFAULT empty(message_raw) ? '{}' : message_raw, + message_raw String EPHEMERAL +) ENGINE = MergeTree ORDER BY (event_type, repo_name); + +INSERT INTO t_github_json (message_raw) FORMAT JSONEachRow {"message_raw": "{\"type\":\"PushEvent\", \"created_at\": \"2022-01-04 07:00:00\", \"actor\":{\"avatar_url\":\"https://avatars.githubusercontent.com/u/123213213?\",\"display_login\":\"github-actions\",\"gravatar_id\":\"\",\"id\":123123123,\"login\":\"github-actions[bot]\",\"url\":\"https://api.github.com/users/github-actions[bot]\"},\"repo\":{\"id\":1001001010101,\"name\":\"some-repo\",\"url\":\"https://api.github.com/repos/some-repo\"}}"} + +SELECT * FROM t_github_json ORDER BY event_type, repo_name; + +DROP TABLE t_github_json; diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.reference b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh new file mode 100755 index 00000000000..d4a30901341 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_string" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_from_string" + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 + +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2 FORMAT JSONAsObject" +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_string FORMAT JSONAsString" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_from_string SELECT data FROM ghdata_2_string" + +${CLICKHOUSE_CLIENT} -q "SELECT \ + (SELECT groupUniqArrayMap(JSONAllPathsWithTypes(data)), sum(cityHash64(toString(data))) FROM ghdata_2_from_string) = \ + (SELECT groupUniqArrayMap(JSONAllPathsWithTypes(data)), sum(cityHash64(toString(data))) FROM ghdata_2)" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_string" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_from_string" diff --git a/tests/queries/0_stateless/01825_new_type_json_nbagames.reference b/tests/queries/0_stateless/01825_new_type_json_nbagames.reference new file mode 100644 index 00000000000..d39672c4d9d --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_nbagames.reference @@ -0,0 +1,65 @@ +1000 +('_id.$oid','String') +('date.$date','String') +('teams','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') +('abbreviation','String') +('city','String') +('home','Bool') +('name','String') +('players','Array(JSON(max_dynamic_types=8, max_dynamic_paths=64))') +('results.ast','Int64') +('results.blk','Int64') +('results.drb','Int64') +('results.fg','Int64') +('results.fg3','Int64') +('results.fg3_pct','DateTime64(9)') +('results.fg3a','Int64') +('results.fg_pct','DateTime64(9)') +('results.fga','Int64') +('results.ft','Int64') +('results.ft_pct','DateTime64(9)') +('results.fta','Int64') +('results.mp','Int64') +('results.orb','Int64') +('results.pf','Int64') +('results.pts','Int64') +('results.stl','Int64') +('results.tov','Int64') +('results.trb','Int64') +('score','Int64') +('won','Int64') +('results.fg3_pct','String') +Boston Celtics 70 +Los Angeles Lakers 64 +Milwaukee Bucks 61 +Philadelphia 76ers 57 +Atlanta Hawks 55 +('ast','Int64') +('blk','Int64') +('drb','Int64') +('fg','Int64') +('fg3','Int64') +('fg3_pct','String') +('fg3a','Int64') +('fg_pct','DateTime64(9)') +('fga','Int64') +('ft','Int64') +('ft_pct','DateTime64(9)') +('fta','Int64') +('mp','String') +('orb','Int64') +('pf','Int64') +('player','String') +('pts','Int64') +('stl','Int64') +('tov','Int64') +('trb','Int64') +('fg3_pct','DateTime64(9)') +('fg_pct','String') +('ft_pct','String') +Larry Bird 10 +Clyde Drexler 4 +Alvin Robertson 3 +Magic Johnson 3 +Charles Barkley 2 +1 diff --git a/tests/queries/0_stateless/01825_new_type_json_nbagames.sh b/tests/queries/0_stateless/01825_new_type_json_nbagames.sh new file mode 100755 index 00000000000..b305b16edbe --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_nbagames.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames_string" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames_from_string" + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE nbagames (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_json_type 1 + +cat $CUR_DIR/data_json/nbagames_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nbagames FORMAT JSONAsObject" + +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM nbagames WHERE NOT ignore(*)" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) from nbagames" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(data.teams[]))) from nbagames" + +${CLICKHOUSE_CLIENT} -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;" + +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(arrayJoin(data.teams[].players[])))) from nbagames" + +${CLICKHOUSE_CLIENT} -q \ +"SELECT player, sum(triple_double) AS triple_doubles FROM \ +( \ + SELECT \ + arrayJoin(arrayJoin(data.teams[].players[])) as players, \ + players.player.:String as player, \ + ((players.pts.:Int64 >= 10) + \ + (players.ast.:Int64 >= 10) + \ + (players.blk.:Int64 >= 10) + \ + (players.stl.:Int64 >= 10) + \ + (players.trb.:Int64 >= 10)) >= 3 AS triple_double \ + from nbagames \ +) \ +GROUP BY player ORDER BY triple_doubles DESC, player LIMIT 5" + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE nbagames_string (data String) ENGINE = MergeTree ORDER BY tuple()" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE nbagames_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_json_type 1 + +cat $CUR_DIR/data_json/nbagames_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO nbagames_string FORMAT JSONAsString" +${CLICKHOUSE_CLIENT} -q "INSERT INTO nbagames_from_string SELECT data FROM nbagames_string" + +${CLICKHOUSE_CLIENT} -q "SELECT \ + (SELECT groupUniqArrayMap(JSONAllPathsWithTypes(data)), sum(cityHash64(toString(data))) FROM nbagames_from_string) = \ + (SELECT groupUniqArrayMap(JSONAllPathsWithTypes(data)), sum(cityHash64(toString(data))) FROM nbagames)" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames_string" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS nbagames_from_string" diff --git a/tests/queries/0_stateless/01825_new_type_json_parallel_insert.reference b/tests/queries/0_stateless/01825_new_type_json_parallel_insert.reference new file mode 100644 index 00000000000..7cf3855d684 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_parallel_insert.reference @@ -0,0 +1 @@ +{'k1':['Int64'],'k2':['String']} 500000 diff --git a/tests/queries/0_stateless/01825_new_type_json_parallel_insert.sql b/tests/queries/0_stateless/01825_new_type_json_parallel_insert.sql new file mode 100644 index 00000000000..a8457ff4f15 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_parallel_insert.sql @@ -0,0 +1,10 @@ +-- Tags: long +DROP TABLE IF EXISTS t_json_parallel; + +SET allow_experimental_json_type = 1, max_insert_threads = 20, max_threads = 20, min_insert_block_size_rows = 65536; +CREATE TABLE t_json_parallel (data JSON) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_json_parallel SELECT materialize('{"k1":1, "k2": "some"}') FROM numbers_mt(500000); +SELECT groupUniqArrayMap(JSONAllPathsWithTypes(data)), count() FROM t_json_parallel; + +DROP TABLE t_json_parallel; diff --git a/tests/queries/0_stateless/01825_type_json_11.sh b/tests/queries/0_stateless/01825_type_json_11.sh index dbed15c8bb9..6109dff53a6 100755 --- a/tests/queries/0_stateless/01825_type_json_11.sh +++ b/tests/queries/0_stateless/01825_type_json_11.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_11" -$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_11 (obj JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 +$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_11 (obj Object('json')) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 cat <&1 | grep -o -m1 "Cannot parse object" $CLICKHOUSE_CLIENT -q "SELECT count() FROM t_json_async_insert" diff --git a/tests/queries/0_stateless/02421_type_json_empty_parts.sh b/tests/queries/0_stateless/02421_type_json_empty_parts.sh index b6cf5995bfa..2ecec524e25 100755 --- a/tests/queries/0_stateless/02421_type_json_empty_parts.sh +++ b/tests/queries/0_stateless/02421_type_json_empty_parts.sh @@ -11,7 +11,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" ${CLICKHOUSE_CLIENT} -q "SELECT 'Collapsing';" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, s Int8, data JSON) ENGINE = CollapsingMergeTree(s) ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, s Int8, data Object('json')) ENGINE = CollapsingMergeTree(s) ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, 1, '{\"k1\": \"aaa\"}') (1, -1, '{\"k2\": \"bbb\"}');" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" @@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" ${CLICKHOUSE_CLIENT} -q "SELECT 'DELETE all';" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, data JSON) ENGINE = MergeTree ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, data Object('json')) ENGINE = MergeTree ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, '{\"k1\": \"aaa\"}') (1, '{\"k2\": \"bbb\"}');" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" @@ -32,7 +32,7 @@ ${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" ${CLICKHOUSE_CLIENT} -q "SELECT 'TTL';" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, d Date, data JSON) ENGINE = MergeTree ORDER BY id TTL d WHERE id % 2 = 1 SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, d Date, data Object('json')) ENGINE = MergeTree ORDER BY id TTL d WHERE id % 2 = 1 SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, '2000-01-01', '{\"k1\": \"aaa\"}') (2, '2000-01-01', '{\"k2\": \"bbb\"}');" ${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE t_json_empty_parts FINAL;" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" diff --git a/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh b/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh index 0d0caa78ea3..99f243833f5 100755 --- a/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh +++ b/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh @@ -21,7 +21,7 @@ echo ' } }' > 02482_object_data.jsonl -$CLICKHOUSE_LOCAL --allow_experimental_object_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj JSON')" +$CLICKHOUSE_LOCAL --allow_experimental_object_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj Object('json')')" rm 02482_object_data.jsonl diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql index 5eb91ac7879..4996f63c5bd 100644 --- a/tests/queries/0_stateless/02513_validate_data_types.sql +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -1,9 +1,9 @@ -- Tags: no-fasttest set allow_experimental_object_type=0; -select CAST('{"x" : 1}', 'JSON'); -- {serverError ILLEGAL_COLUMN} +select CAST('{"x" : 1}', 'Object(''json'')'); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONAsObject); -- {serverError ILLEGAL_COLUMN} -desc file(nonexist.json, JSONEachRow, 'x JSON'); -- {serverError ILLEGAL_COLUMN} +desc file(nonexist.json, JSONEachRow, 'x Object(''json'')'); -- {serverError ILLEGAL_COLUMN} set allow_suspicious_low_cardinality_types=0; select CAST(1000000, 'LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} diff --git a/tests/queries/0_stateless/02553_type_json_attach_partition.sql b/tests/queries/0_stateless/02553_type_json_attach_partition.sql index 9225106f767..0a9d64451c1 100644 --- a/tests/queries/0_stateless/02553_type_json_attach_partition.sql +++ b/tests/queries/0_stateless/02553_type_json_attach_partition.sql @@ -2,7 +2,7 @@ SET allow_experimental_object_type = 1; DROP TABLE IF EXISTS t_json_attach_partition; -CREATE TABLE t_json_attach_partition(b UInt64, c JSON) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE t_json_attach_partition(b UInt64, c Object('json')) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": 1}}; ALTER TABLE t_json_attach_partition DETACH PARTITION tuple(); diff --git a/tests/queries/0_stateless/02553_type_object_analyzer.sql b/tests/queries/0_stateless/02553_type_object_analyzer.sql index 55482a02ed1..a8eeb57a6c6 100644 --- a/tests/queries/0_stateless/02553_type_object_analyzer.sql +++ b/tests/queries/0_stateless/02553_type_object_analyzer.sql @@ -3,7 +3,7 @@ SET allow_experimental_object_type = 1; SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS t_json_analyzer; -CREATE TABLE t_json_analyzer (a JSON) ENGINE = Memory; +CREATE TABLE t_json_analyzer (a Object('json')) ENGINE = Memory; INSERT INTO t_json_analyzer VALUES ('{"id": 2, "obj": {"k2": {"k3": "str", "k4": [{"k6": 55}]}, "some": 42}, "s": "bar"}'); SELECT any(a) AS data FROM t_json_analyzer FORMAT JSONEachRow; diff --git a/tests/queries/0_stateless/02717_pretty_json.sql b/tests/queries/0_stateless/02717_pretty_json.sql index 8a49eb50adf..1a5c090bcb2 100644 --- a/tests/queries/0_stateless/02717_pretty_json.sql +++ b/tests/queries/0_stateless/02717_pretty_json.sql @@ -1,3 +1,3 @@ set allow_experimental_object_type=1; -select 42 as num, [42, 42] as arr, [[[42, 42], [42, 42]], [[42, 42]]] as nested_arr, tuple(42, 42)::Tuple(a UInt32, b UInt32) as tuple, tuple(tuple(tuple(42, 42), 42), 42)::Tuple(a Tuple(b Tuple(c UInt32, d UInt32), e UInt32), f UInt32) as nested_tuple, map(42, 42, 24, 24) as map, map(42, map(42, map(42, 42))) as nested_map, [tuple(map(42, 42), [42, 42]), tuple(map(42, 42), [42, 42])]::Array(Tuple(Map(UInt32, UInt32), Array(UInt32))) as nested_types, '{"a" : {"b" : 1, "c" : 2}}'::JSON as json_object format PrettyNDJSON; +select 42 as num, [42, 42] as arr, [[[42, 42], [42, 42]], [[42, 42]]] as nested_arr, tuple(42, 42)::Tuple(a UInt32, b UInt32) as tuple, tuple(tuple(tuple(42, 42), 42), 42)::Tuple(a Tuple(b Tuple(c UInt32, d UInt32), e UInt32), f UInt32) as nested_tuple, map(42, 42, 24, 24) as map, map(42, map(42, map(42, 42))) as nested_map, [tuple(map(42, 42), [42, 42]), tuple(map(42, 42), [42, 42])]::Array(Tuple(Map(UInt32, UInt32), Array(UInt32))) as nested_types, '{"a" : {"b" : 1, "c" : 2}}'::Object('json') as json_object format PrettyNDJSON; diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql index 3bbcbb1a535..dadfa59bf87 100644 --- a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql +++ b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.sql @@ -11,7 +11,7 @@ DROP TABLE IF EXISTS tab; SET allow_suspicious_low_cardinality_types=1; -SET allow_experimental_object_type=1; +SET allow_experimental_json_type=1; CREATE TABLE tab ( diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect b/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect index 4798a6958c6..2079da9d34a 100755 --- a/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect +++ b/tests/queries/0_stateless/02775_show_columns_called_from_mysql.expect @@ -33,7 +33,7 @@ send -- "DROP TABLE IF EXISTS tab;\r" expect "Query OK, 0 rows affected" send -- "SET allow_suspicious_low_cardinality_types=1;\r" -send -- "SET allow_experimental_object_type=1;\r" +send -- "SET allow_experimental_json_type=1;\r" send -- " CREATE TABLE tab diff --git a/tests/queries/0_stateless/02870_per_column_settings.sql b/tests/queries/0_stateless/02870_per_column_settings.sql index d242ebe6c61..c3050222bc8 100644 --- a/tests/queries/0_stateless/02870_per_column_settings.sql +++ b/tests/queries/0_stateless/02870_per_column_settings.sql @@ -49,7 +49,7 @@ CREATE TABLE tab ( id UInt64, tup Tuple(UInt64, UInt64) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), - json JSON SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), + json Object('json') SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), ) ENGINE = MergeTree ORDER BY id diff --git a/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference b/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference index 3455adc8723..f100e8c48d4 100644 --- a/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference +++ b/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference @@ -34,6 +34,7 @@ The answer to all questions is 2023-11-14 05:50:12.123. The answer to all questions is hallo. The answer to all questions is [\'foo\',\'bar\']. The answer to all questions is {"foo":"bar"}. +The answer to all questions is {"foo":"bar"}. The answer to all questions is (42,\'foo\'). The answer to all questions is {42:\'foo\'}. The answer to all questions is 122.233.64.201. diff --git a/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql b/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql index ad1de2bec6d..dcc3964e4b0 100644 --- a/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql +++ b/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql @@ -3,6 +3,7 @@ -- no-fasttest: json type needs rapidjson library, geo types need s2 geometry SET allow_experimental_object_type = 1; +SET allow_experimental_json_type = 1; SET allow_suspicious_low_cardinality_types=1; SELECT '-- Const string + non-const arbitrary type'; @@ -40,6 +41,7 @@ SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2023-11 SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2023-11-14 05:50:12.123' :: DateTime64(3, 'Europe/Amsterdam'))); SELECT format('The {0} to all questions is {1}.', 'answer', materialize('hallo' :: Enum('hallo' = 1))); SELECT format('The {0} to all questions is {1}.', 'answer', materialize(['foo', 'bar'] :: Array(String))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('{"foo": "bar"}' :: Object('json'))); SELECT format('The {0} to all questions is {1}.', 'answer', materialize('{"foo": "bar"}' :: JSON)); SELECT format('The {0} to all questions is {1}.', 'answer', materialize((42, 'foo') :: Tuple(Int32, String))); SELECT format('The {0} to all questions is {1}.', 'answer', materialize(map(42, 'foo') :: Map(Int32, String))); diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference index 5555c918500..3e63763d544 100644 --- a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.reference @@ -7,7 +7,7 @@ Decimal 45 Decimal(10, 0) Decimal(M) 46 Decimal(4, 0) Decimal(M, D) 47.21 Decimal(4, 2) Double 48.11 Float64 -JSON {"foo":"bar"} Object(\'json\') +JSON {"foo":"bar"} JSON Real 49.22 Float32 Signed 50 Int64 Unsigned 52 UInt64 @@ -21,7 +21,7 @@ Decimal 45 Decimal(10, 0) Decimal(M) 46 Decimal(4, 0) Decimal(M, D) 47.21 Decimal(4, 2) Double 48.11 Float64 -JSON {"foo":"bar"} Object(\'json\') +JSON {"foo":"bar"} JSON Real 49.22 Float32 Signed 50 Int64 Unsigned 52 UInt64 diff --git a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql index 7b5735cdebc..8cccde4b0ab 100644 --- a/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql +++ b/tests/queries/0_stateless/02969_mysql_cast_type_aliases.sql @@ -1,7 +1,7 @@ -- See https://dev.mysql.com/doc/refman/8.0/en/cast-functions.html#function_cast -- Tests are in order of the type appearance in the docs -SET allow_experimental_object_type = 1; +SET allow_experimental_json_type = 1; SELECT '-- Uppercase tests'; -- Not supported as it is translated to FixedString without arguments diff --git a/tests/queries/0_stateless/03157_dynamic_type_json.reference b/tests/queries/0_stateless/03157_dynamic_type_json.reference index 38bca12bb95..14e851bdbc7 100644 --- a/tests/queries/0_stateless/03157_dynamic_type_json.reference +++ b/tests/queries/0_stateless/03157_dynamic_type_json.reference @@ -1,5 +1,5 @@ -1 (((((((((('deep_value')))))))))) -2 (((((((((('deep_array_value')))))))))) +1 {"level1":{"level2":{"level3":{"level4":{"level5":{"level6":{"level7":{"level8":{"level9":{"level10":"deep_value"}}}}}}}}}} +2 {"level1":{"level2":{"level3":{"level4":{"level5":{"level6":{"level7":{"level8":{"level9":{"level10":"deep_array_value"}}}}}}}}}} -(((((((((('deep_value')))))))))) Tuple(level1 Tuple(level2 Tuple(level3 Tuple(level4 Tuple(level5 Tuple(level6 Tuple(level7 Tuple(level8 Tuple(level9 Tuple(level10 String)))))))))) -(((((((((('deep_array_value')))))))))) Tuple(level1 Tuple(level2 Tuple(level3 Tuple(level4 Tuple(level5 Tuple(level6 Tuple(level7 Tuple(level8 Tuple(level9 Tuple(level10 String)))))))))) +{"level1":{"level2":{"level3":{"level4":{"level5":{"level6":{"level7":{"level8":{"level9":{"level10":"deep_value"}}}}}}}}}} JSON +{"level1":{"level2":{"level3":{"level4":{"level5":{"level6":{"level7":{"level8":{"level9":{"level10":"deep_array_value"}}}}}}}}}} JSON diff --git a/tests/queries/0_stateless/03157_dynamic_type_json.sql b/tests/queries/0_stateless/03157_dynamic_type_json.sql index cb1a5987104..91af7942718 100644 --- a/tests/queries/0_stateless/03157_dynamic_type_json.sql +++ b/tests/queries/0_stateless/03157_dynamic_type_json.sql @@ -1,7 +1,8 @@ SET allow_experimental_dynamic_type=1; -SET allow_experimental_object_type=1; +SET allow_experimental_json_type=1; SET allow_experimental_variant_type=1; +DROP TABLE IF EXISTS test_deep_nested_json; CREATE TABLE test_deep_nested_json (i UInt16, d JSON) ENGINE = Memory; INSERT INTO test_deep_nested_json VALUES (1, '{"level1": {"level2": {"level3": {"level4": {"level5": {"level6": {"level7": {"level8": {"level9": {"level10": "deep_value"}}}}}}}}}}'); @@ -11,3 +12,4 @@ SELECT * FROM test_deep_nested_json ORDER BY i; SELECT ''; SELECT d::Dynamic d1, dynamicType(d1) FROM test_deep_nested_json ORDER BY i; +DROP TABLE test_deep_nested_json; diff --git a/tests/queries/0_stateless/03158_dynamic_type_from_variant.sql b/tests/queries/0_stateless/03158_dynamic_type_from_variant.sql index 20a9e17a148..a18f985f217 100644 --- a/tests/queries/0_stateless/03158_dynamic_type_from_variant.sql +++ b/tests/queries/0_stateless/03158_dynamic_type_from_variant.sql @@ -1,5 +1,4 @@ SET allow_experimental_dynamic_type=1; -SET allow_experimental_object_type=1; SET allow_experimental_variant_type=1; CREATE TABLE test_variable (v Variant(String, UInt32, IPv6, Bool, DateTime64)) ENGINE = Memory; diff --git a/tests/queries/0_stateless/03159_dynamic_type_all_types.sql b/tests/queries/0_stateless/03159_dynamic_type_all_types.sql index d302205ca23..d6c293de1be 100644 --- a/tests/queries/0_stateless/03159_dynamic_type_all_types.sql +++ b/tests/queries/0_stateless/03159_dynamic_type_all_types.sql @@ -1,7 +1,6 @@ -- Tags: no-random-settings SET allow_experimental_dynamic_type=1; -SET allow_experimental_object_type=1; SET allow_experimental_variant_type=1; SET allow_suspicious_low_cardinality_types=1; diff --git a/tests/queries/0_stateless/03205_json_cast_from_string.reference b/tests/queries/0_stateless/03205_json_cast_from_string.reference new file mode 100644 index 00000000000..1c4d820b3b4 --- /dev/null +++ b/tests/queries/0_stateless/03205_json_cast_from_string.reference @@ -0,0 +1,18 @@ +{} +{"a":"42","b":"Hello"} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} +{"a":{"b":{"c":{"d":true},"e":"43"},"f":"44"},"g":"44"} +{"a":{"b":{"e":"43"},"f":"44"},"g":"44"} +{"a":{"b":{"e":"43"},"f":"44"},"g":"44"} +{"a":{"f":"44"},"g":"44"} +{"g":"44"} +{"a":{"f":"44"},"g":"44"} +{"g":"44"} +{} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64','a.b.e':'Int64'} {'a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64'} {'a.b.e':'Int64','a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'String','a.b.e':'String','a.f':'Int64','g':'Int64'} {'a.b.c.d':'String','a.b.e':'String'} {'a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'String','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'String'} {'a.b.e':'Int64','a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} diff --git a/tests/queries/0_stateless/03205_json_cast_from_string.sql b/tests/queries/0_stateless/03205_json_cast_from_string.sql new file mode 100644 index 00000000000..6ab88826c86 --- /dev/null +++ b/tests/queries/0_stateless/03205_json_cast_from_string.sql @@ -0,0 +1,22 @@ +-- Tags: no-fasttest +set allow_experimental_json_type=1; + +select materialize('{}')::JSON; +select materialize('{"a" : 42, "b" : "Hello"}')::JSON; +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON; +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(a.b.c.d Bool); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP a.b.c.d); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP a.b.c); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP a.b); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP a); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP REGEXP '.*a.*b'); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP REGEXP '.*a.*'); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(SKIP REGEXP '.*'); + +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 2) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 0) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 2, max_dynamic_types=1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 1, max_dynamic_types=1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 0, max_dynamic_types=1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); diff --git a/tests/queries/0_stateless/03205_json_syntax.reference b/tests/queries/0_stateless/03205_json_syntax.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03205_json_syntax.sql b/tests/queries/0_stateless/03205_json_syntax.sql new file mode 100644 index 00000000000..604bddf69d3 --- /dev/null +++ b/tests/queries/0_stateless/03205_json_syntax.sql @@ -0,0 +1,23 @@ +-- Tags: no-fasttest + +set allow_experimental_json_type=1; +drop table if exists test; +create table test (json JSON) engine=Memory; +replace table test (json JSON(max_dynamic_paths=10)) engine=Memory; +replace table test (json JSON(max_dynamic_types=10)) engine=Memory; +replace table test (json JSON(a UInt32)) engine=Memory; +replace table test (json JSON(aaaaa UInt32)) engine=Memory; +replace table test (json JSON(`a b c d` UInt32)) engine=Memory; +replace table test (json JSON(a.b.c UInt32)) engine=Memory; +replace table test (json JSON(aaaa.b.cccc UInt32)) engine=Memory; +replace table test (json JSON(`some path`.`path some` UInt32)) engine=Memory; +replace table test (json JSON(a.b.c Tuple(d UInt32, e UInt32))) engine=Memory; +replace table test (json JSON(SKIP a)) engine=Memory; +replace table test (json JSON(SKIP aaaa)) engine=Memory; +replace table test (json JSON(SKIP `a b c d`)) engine=Memory; +replace table test (json JSON(SKIP a.b.c)) engine=Memory; +replace table test (json JSON(SKIP aaaa.b.cccc)) engine=Memory; +replace table test (json JSON(SKIP `some path`.`path some`)) engine=Memory; +replace table test (json JSON(SKIP REGEXP '.*a.*')) engine=Memory; +replace table test (json JSON(max_dynamic_paths=10, max_dynamic_types=10, a.b.c UInt32, b.c.d String, SKIP g.d.a, SKIP o.g.a, SKIP REGEXP '.*a.*', SKIP REGEXP 'abc')) engine=Memory; +drop table test; diff --git a/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference b/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference new file mode 100644 index 00000000000..fa6fed341ba --- /dev/null +++ b/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference @@ -0,0 +1,195 @@ +JSON with no arguments +{"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {} +{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {} +{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','e':'String'} {} +{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} +{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} +1 2020-01-01 {"e":{"f":["s1","s2"]}} +2 [1,2,3] {"e":{"g":"43"}} +3 \N {} +4 \N {} +5 ['b1','b2'] {"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}} +JSON(a.b Tuple(c UInt32, d Array(Bool)), SKIP d.e, SKIP c, SKIP REGEXP '.*h.*') +{"a":{"b":{"c":1,"d":[false,true]}},"b":"2020-01-01"} +{"a":{"b":{"c":2,"d":[true,true]}},"b":["1","2","3"]} +{"a":{"b":{"c":3,"d":[true,true]}},"e":"Hello, World!"} +{"a":{"b":{"c":4,"d":[true,true]}}} +{"a":{"b":{"c":5,"d":[true,true]}},"b":["b1","b2"]} +{'a.b':'Tuple(c UInt32, d Array(Bool))','b':'Date'} {'b':'Date'} {} +{'a.b':'Tuple(c UInt32, d Array(Bool))','b':'Array(Nullable(Int64))'} {'b':'Array(Nullable(Int64))'} {} +{'a.b':'Tuple(c UInt32, d Array(Bool))','e':'String'} {'e':'String'} {} +{'a.b':'Tuple(c UInt32, d Array(Bool))'} {} {} +{'a.b':'Tuple(c UInt32, d Array(Bool))','b':'Array(Nullable(String))'} {'b':'Array(Nullable(String))'} {} +JSON(a.b.c UInt32, max_dynamic_paths=2) +{"a":{"b":{"c":1,"d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":2,"d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":3,"d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":4,"d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.d':'Array(Nullable(Int64))','b':'Date'} {'c':'Int64','d.e.f':'Array(Nullable(String))'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))'} {'d.e.g':'Int64'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.d':'Array(Nullable(Int64))'} {'e':'String'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.d':'Array(Nullable(Int64))'} {'c':'Int64'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))'} {'d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} +JSON(a.b.c UInt32, max_dynamic_paths=0) +{"a":{"b":{"c":1,"d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":2,"d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":3,"d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":4,"d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','e':'String'} {} {'a.b.d':'Array(Nullable(Int64))','e':'String'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} {'a.b.d':'Array(Nullable(Int64))','c':'Int64'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} +JSON(a.b.c UInt32, max_dynamic_types=1) +{"a":{"b":{"c":1,"d":"[0,1]"}},"b":"2020-01-01","c":"42","d":{"e":{"f":"[\\"s1\\",\\"s2\\"]"}}} +{"a":{"b":{"c":2,"d":"[2,3]"}},"b":"[1,2,3]","d":{"e":{"g":"43"}}} +{"a":{"b":{"c":3,"d":"[4,5]"}},"e":"Hello, World!"} +{"a":{"b":{"c":4,"d":"[6,7]"}},"c":"43"} +{"a":{"b":{"c":5,"d":"[8,9]"}},"b":"[\\"b1\\",\\"b2\\"]","d":{"e":{"f":"[\\"s3\\",\\"s4\\"]","g":"44","h":"2020-02-02 10:00:00"}}} +{'a.b.c':'UInt32','a.b.d':'String','b':'String','c':'String','d.e.f':'String'} {'a.b.d':'String','b':'String','c':'String','d.e.f':'String'} {} +{'a.b.c':'UInt32','a.b.d':'String','b':'String','d.e.g':'String'} {'a.b.d':'String','b':'String','d.e.g':'String'} {} +{'a.b.c':'UInt32','a.b.d':'String','e':'String'} {'a.b.d':'String','e':'String'} {} +{'a.b.c':'UInt32','a.b.d':'String','c':'String'} {'a.b.d':'String','c':'String'} {} +{'a.b.c':'UInt32','a.b.d':'String','b':'String','d.e.f':'String','d.e.g':'String','d.e.h':'String'} {'a.b.d':'String','b':'String','d.e.f':'String','d.e.g':'String','d.e.h':'String'} {} +Test small max_read_buffer_size +{"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +Test PrettyJSONEachRow +{ + "json": { + "a" : { + "b" : { + "c" : "1", + "d" : [ + "0", + "1" + ] + } + }, + "b" : "2020-01-01", + "c" : "42", + "d" : { + "e" : { + "f" : [ + "s1", + "s2" + ] + } + } + } +} +{ + "json": { + "a" : { + "b" : { + "c" : "2", + "d" : [ + "2", + "3" + ] + } + }, + "b" : [ + "1", + "2", + "3" + ], + "d" : { + "e" : { + "g" : "43" + } + } + } +} +{ + "json": { + "a" : { + "b" : { + "c" : "3", + "d" : [ + "4", + "5" + ] + } + }, + "e" : "Hello, World!" + } +} +{ + "json": { + "a" : { + "b" : { + "c" : "4", + "d" : [ + "6", + "7" + ] + } + }, + "c" : "43" + } +} +{ + "json": { + "a" : { + "b" : { + "c" : "5", + "d" : [ + "8", + "9" + ] + } + }, + "b" : [ + "b1", + "b2" + ], + "d" : { + "e" : { + "f" : [ + "s3", + "s4" + ], + "g" : "44", + "h" : "2020-02-02 10:00:00.000000000" + } + } + } +} +Test TSV +{"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +Test CSV +"{""a"":{""b"":{""c"":""1"",""d"":[""0"",""1""]}},""b"":""2020-01-01"",""c"":""42"",""d"":{""e"":{""f"":[""s1"",""s2""]}}}" +"{""a"":{""b"":{""c"":""2"",""d"":[""2"",""3""]}},""b"":[""1"",""2"",""3""],""d"":{""e"":{""g"":""43""}}}" +"{""a"":{""b"":{""c"":""3"",""d"":[""4"",""5""]}},""e"":""Hello, World!""}" +"{""a"":{""b"":{""c"":""4"",""d"":[""6"",""7""]}},""c"":""43""}" +"{""a"":{""b"":{""c"":""5"",""d"":[""8"",""9""]}},""b"":[""b1"",""b2""],""d"":{""e"":{""f"":[""s3"",""s4""],""g"":""44"",""h"":""2020-02-02 10:00:00.000000000""}}}" diff --git a/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh b/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh new file mode 100755 index 00000000000..44bef4f0e95 --- /dev/null +++ b/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.json + +echo '{"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}}} +{"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}}} +{"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "e" : "Hello, World!"} +{"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43} +{"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : ["b1", "b2"], "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44, "h" : "2020-02-02 10:00:00"}}}' > $DATA_FILE + +echo "JSON with no arguments" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json.a.b.c, json.b, json.^d from file($DATA_FILE, JSONAsObject)" + +echo "JSON(a.b Tuple(c UInt32, d Array(Bool)), SKIP d.e, SKIP c, SKIP REGEXP '.*h.*')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject, 'json JSON(a.b Tuple(c UInt32, d Array(Bool)), SKIP d.e, SKIP c, SKIP REGEXP \'.*h.*\')')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b Tuple(c UInt32, d Array(Bool)), SKIP d.e, SKIP c, SKIP REGEXP \'.*h.*\')')" + +echo "JSON(a.b.c UInt32, max_dynamic_paths=2)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_paths=2)')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_paths=2)')" + +echo "JSON(a.b.c UInt32, max_dynamic_paths=0)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_paths=0)')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_paths=0)')" + +echo "JSON(a.b.c UInt32, max_dynamic_types=1)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_types=1)')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_types=1)')" + +echo "Test small max_read_buffer_size" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --max_read_buffer_size=1 -q "select json from file($DATA_FILE, JSONAsObject)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --max_read_buffer_size=2 -q "select json from file($DATA_FILE, JSONAsObject)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --max_read_buffer_size=3 -q "select json from file($DATA_FILE, JSONAsObject)" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --max_read_buffer_size=4 -q "select json from file($DATA_FILE, JSONAsObject)" + +echo "Test PrettyJSONEachRow" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject) format PrettyJSONEachRow" + +echo "Test TSV" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, TSV, 'json JSON') format TSV" +echo "Test CSV" +echo '"{""a"" : {""b"" : {""c"" : 1, ""d"" : [0, 1]}}, ""b"" : ""2020-01-01"", ""c"" : 42, ""d"" : {""e"" : {""f"" : [""s1"", ""s2""]}}}" +"{""a"" : {""b"" : {""c"" : 2, ""d"" : [2, 3]}}, ""b"" : [1, 2, 3], ""c"" : null, ""d"" : {""e"" : {""g"" : 43}}}" +"{""a"" : {""b"" : {""c"" : 3, ""d"" : [4, 5]}}, ""e"" : ""Hello, World!""}" +"{""a"" : {""b"" : {""c"" : 4, ""d"" : [6, 7]}}, ""c"" : 43}" +"{""a"" : {""b"" : {""c"" : 5, ""d"" : [8, 9]}}, ""b"" : [""b1"", ""b2""], ""d"" : {""e"" : {""f"" : [""s3"", ""s4""], ""g"" : 44, ""h"" : ""2020-02-02 10:00:00""}}}"' > $DATA_FILE +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, CSV, 'json JSON') format CSV" + +rm $DATA_FILE diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1.reference new file mode 100644 index 00000000000..90a96b1bb2f --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1.reference @@ -0,0 +1,2080 @@ +Memory +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +MergeTree compact +No merges +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +MergeTree wide +No merges +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_1.sh new file mode 100755 index 00000000000..3c63b513ad6 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1.sh @@ -0,0 +1,125 @@ +#!/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;" + +echo "Memory" +$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;" + +echo "MergeTree compact" +$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;" + +echo "MergeTree wide" +$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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference new file mode 100644 index 00000000000..db7180cec75 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference @@ -0,0 +1,71 @@ +No merges +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._0','Int64') +('b.b._1','Int64') +('b.b._2','Int64') +('b.b._3','Int64') +('b.b._4','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +800000 +800000 +300000 +700000 +700000 +200000 +400000 +500000 +600000 +500000 +600000 +600000 +700000 +580000 +680000 +0 +0 +With merges +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._0','Int64') +('b.b._1','Int64') +('b.b._2','Int64') +('b.b._3','Int64') +('b.b._4','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +800000 +800000 +300000 +700000 +700000 +200000 +400000 +500000 +600000 +500000 +600000 +600000 +700000 +580000 +680000 +0 +0 diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh new file mode 100755 index 00000000000..128c525bf02 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh @@ -0,0 +1,142 @@ +#!/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(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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference new file mode 100644 index 00000000000..1bbd3926bdc --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference @@ -0,0 +1,35 @@ +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._0','Int64') +('b.b._1','Int64') +('b.b._2','Int64') +('b.b._3','Int64') +('b.b._4','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +800000 +800000 +300000 +700000 +700000 +200000 +400000 +500000 +600000 +500000 +600000 +600000 +700000 +580000 +680000 +0 +0 diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh new file mode 100755 index 00000000000..bf6360a89a4 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh @@ -0,0 +1,137 @@ +#!/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(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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference new file mode 100644 index 00000000000..db7180cec75 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference @@ -0,0 +1,71 @@ +No merges +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._0','Int64') +('b.b._1','Int64') +('b.b._2','Int64') +('b.b._3','Int64') +('b.b._4','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +800000 +800000 +300000 +700000 +700000 +200000 +400000 +500000 +600000 +500000 +600000 +600000 +700000 +580000 +680000 +0 +0 +With merges +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._0','Int64') +('b.b._1','Int64') +('b.b._2','Int64') +('b.b._3','Int64') +('b.b._4','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +800000 +800000 +300000 +700000 +700000 +200000 +400000 +500000 +600000 +500000 +600000 +600000 +700000 +580000 +680000 +0 +0 diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh new file mode 100755 index 00000000000..5e8af0c1f80 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh @@ -0,0 +1,142 @@ +#!/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(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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference new file mode 100644 index 00000000000..ec8d1407bdb --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference @@ -0,0 +1,560 @@ +Memory +insert +test +('a.a1','String') +('a.a2','String') +('a.a3','String') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +{ + "json": [{"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}], + "json.a.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +{ + "json": [{"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}], + "json.a.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +{ + "json": [{"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}], + "json.a.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +{ + "json": [{"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}], + "json.a.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +{ + "json": [{"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[]}}, {"a":{"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}, {"a":{"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}}, {"a":{"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}}, {"a":{"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}}, {"a":{"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}}, {"a":{"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}}], + "json.a.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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.b": [[], [], [], [], [], [{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.c.d.e": [[], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_0": [[], [], [], [], [], ["5"], ["6",null], ["7",null,null], ["8",null,null,null], ["9",null,null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_1": [[], [], [], [], [], [null], [null,"6"], [null,"7",null], [null,"8",null,null], [null,"9",null,null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_2": [[], [], [], [], [], [null], [null,null], [null,null,"7"], [null,null,"8",null], [null,null,"9",null,null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_3": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"8"], [null,null,null,"9",null], [], [], [], [], [], [], [], [], [], []], + "json.a.b.b.c.d_4": [[], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"9"], [], [], [], [], [], [], [], [], [], []], + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`": [[], [], [], [], [], [], [], [], [], [], [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_2": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,"12"], [null,null,"13",null], [null,null,"14",null,null], [null], [null,null], [null,null,"17"], [null,null,"18",null], [null,null,"19",null,null]], + "json.a.r.:`Array(JSON)`.b.c.d_3": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,"13"], [null,null,null,"14",null], [null], [null,null], [null,null,null], [null,null,null,"18"], [null,null,null,"19",null]], + "json.a.r.:`Array(JSON)`.b.c.d_4": [[], [], [], [], [], [], [], [], [], [], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"14"], [null], [null,null], [null,null,null], [null,null,null,null], [null,null,null,null,"19"]], + "json.^`a`": [{"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[]}, {"b":[{"b":{"c":{"d_0":"5"}},"c":{"d":{"e":["0"]}}}]}, {"b":[{"b":{"c":{"d_0":"6"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"6"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[{"b":{"c":{"d_0":"7"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"7"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"7"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[{"b":{"c":{"d_0":"8"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"8"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"8"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"8"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[{"b":{"c":{"d_0":"9"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"9"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"9"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"9"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"9"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}]}, {"b":[],"r":[{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}]}, {"a1":"15","a2":"15","a3":"15","a4":"15","a5":"15","a6":"15","a7":"15","a8":"15","b":[],"r":[{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}]}, {"a1":"16","a2":"16","a3":"16","a4":"16","a5":"16","a6":"16","a7":"16","a8":"16","b":[],"r":[{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}]}, {"a1":"17","a2":"17","a3":"17","a4":"17","a5":"17","a6":"17","a7":"17","a8":"17","b":[],"r":[{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}]}, {"a1":"18","a2":"18","a3":"18","a4":"18","a5":"18","a6":"18","a7":"18","a8":"18","b":[],"r":[{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}]}, {"a1":"19","a2":"19","a3":"19","a4":"19","a5":"19","a6":"19","a7":"19","a8":"19","b":[],"r":[{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]}], + "json.a.b.^`b`.c": [[], [], [], [], [], [{"d_0":"5"}], [{"d_0":"6"},{"d_1":"6"}], [{"d_0":"7"},{"d_1":"7"},{"d_2":"7"}], [{"d_0":"8"},{"d_1":"8"},{"d_2":"8"},{"d_3":"8"}], [{"d_0":"9"},{"d_1":"9"},{"d_2":"9"},{"d_3":"9"},{"d_4":"9"}], [], [], [], [], [], [], [], [], [], []], + "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)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "json.a.r.:`Array(JSON)`.b.c.d_0": [[], [], [], [], [], [], [], [], [], [], ["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]], + "json.a.r.:`Array(JSON)`.b.c.d_1": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.c.d.e.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]], [["0"]], [["0","1"],["0","1","2"]], [["0","1","2"],["0","1","2","3"],["0","1","2","3","4"]], [["0","1","2","3"],["0","1","2","3","4"],["0"],["0","1"]], [["0","1","2","3","4"],["0"],["0","1"],["0","1","2"],["0","1","2","3"]]], + "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]], + "json.a.r.:`Array(JSON)`.b.c.d_1.:`Int64`": [[], [], [], [], [], [], [], [], [], [], [null], [null,"11"], [null,"12",null], [null,"13",null,null], [null,"14",null,null,null], [null], [null,"16"], [null,"17",null], [null,"18",null,null], [null,"19",null,null,null]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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": [[], [], [], [], [], [], [], [], [], [], ["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]] +} +{ + "json.a.r": [null, null, null, null, null, null, null, null, null, null, [{"b":{"c":{"d_0":"10"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"11"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"11"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"12"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"12"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"12"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"13"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"13"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"13"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"13"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"14"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"14"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"14"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"14"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"14"}},"c":{"d":{"e":["0","1","2","3"]}}}], [{"b":{"c":{"d_0":"15"}},"c":{"d":{"e":["0"]}}}], [{"b":{"c":{"d_0":"16"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_1":"16"}},"c":{"d":{"e":["0","1","2"]}}}], [{"b":{"c":{"d_0":"17"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_1":"17"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_2":"17"}},"c":{"d":{"e":["0","1","2","3","4"]}}}], [{"b":{"c":{"d_0":"18"}},"c":{"d":{"e":["0","1","2","3"]}}},{"b":{"c":{"d_1":"18"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_2":"18"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_3":"18"}},"c":{"d":{"e":["0","1"]}}}], [{"b":{"c":{"d_0":"19"}},"c":{"d":{"e":["0","1","2","3","4"]}}},{"b":{"c":{"d_1":"19"}},"c":{"d":{"e":["0"]}}},{"b":{"c":{"d_2":"19"}},"c":{"d":{"e":["0","1"]}}},{"b":{"c":{"d_3":"19"}},"c":{"d":{"e":["0","1","2"]}}},{"b":{"c":{"d_4":"19"}},"c":{"d":{"e":["0","1","2","3"]}}}]], + "json.a.r.:`Array(JSON)`.^`b`": [[], [], [], [], [], [], [], [], [], [], [{"c":{"d_0":"10"}}], [{"c":{"d_0":"11"}},{"c":{"d_1":"11"}}], [{"c":{"d_0":"12"}},{"c":{"d_1":"12"}},{"c":{"d_2":"12"}}], [{"c":{"d_0":"13"}},{"c":{"d_1":"13"}},{"c":{"d_2":"13"}},{"c":{"d_3":"13"}}], [{"c":{"d_0":"14"}},{"c":{"d_1":"14"}},{"c":{"d_2":"14"}},{"c":{"d_3":"14"}},{"c":{"d_4":"14"}}], [{"c":{"d_0":"15"}}], [{"c":{"d_0":"16"}},{"c":{"d_1":"16"}}], [{"c":{"d_0":"17"}},{"c":{"d_1":"17"}},{"c":{"d_2":"17"}}], [{"c":{"d_0":"18"}},{"c":{"d_1":"18"}},{"c":{"d_2":"18"}},{"c":{"d_3":"18"}}], [{"c":{"d_0":"19"}},{"c":{"d_1":"19"}},{"c":{"d_2":"19"}},{"c":{"d_3":"19"}},{"c":{"d_4":"19"}}]], + "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]] +} diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh new file mode 100755 index 00000000000..9d8d02de7aa --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh @@ -0,0 +1,71 @@ +#!/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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference new file mode 100644 index 00000000000..9c092278ec7 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference @@ -0,0 +1,65 @@ +No merges +insert +test +('a.a1','String') +('a.a2','String') +('a.a3','String') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +20000 +20000 +0 +0 +20000 +20000 +0 +0 +With merges +test +('a.a1','String') +('a.a2','String') +('a.a3','String') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +20000 +20000 +0 +0 +20000 +20000 +0 +0 diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh new file mode 100755 index 00000000000..8fa1b809390 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh @@ -0,0 +1,73 @@ +#!/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(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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference new file mode 100644 index 00000000000..63d10b1315f --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference @@ -0,0 +1,32 @@ +insert +test +('a.a1','String') +('a.a2','String') +('a.a3','String') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +20000 +20000 +0 +0 +20000 +20000 +0 +0 diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh new file mode 100755 index 00000000000..f7c0fdad1a6 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh @@ -0,0 +1,68 @@ +#!/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(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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference new file mode 100644 index 00000000000..9c092278ec7 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference @@ -0,0 +1,65 @@ +No merges +insert +test +('a.a1','String') +('a.a2','String') +('a.a3','String') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +20000 +20000 +0 +0 +20000 +20000 +0 +0 +With merges +test +('a.a1','String') +('a.a2','String') +('a.a3','String') +('a.a4','String') +('a.a5','String') +('a.a6','String') +('a.a7','String') +('a.a8','String') +('a.b','Array(JSON)') +('a.r','Array(JSON(max_dynamic_types=16, max_dynamic_paths=2))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +('b.c.d_0','Int64') +('b.c.d_1','Int64') +('b.c.d_2','Int64') +('b.c.d_3','Int64') +('b.c.d_4','Int64') +('c.d.e','Array(Nullable(Int64))') +20000 +20000 +0 +0 +20000 +20000 +0 +0 diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh new file mode 100755 index 00000000000..3106702faf4 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh @@ -0,0 +1,73 @@ +#!/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(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;" diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference new file mode 100644 index 00000000000..203b59521f4 --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference @@ -0,0 +1,100 @@ +MergeTree compact +test +Dynamic paths +100000 a +90000 b +80000 c +70000 d +60000 e +Shared data paths +Dynamic paths +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +10000 g +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +10000 g +MergeTree wide +test +Dynamic paths +100000 a +90000 b +80000 c +70000 d +60000 e +Shared data paths +Dynamic paths +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +10000 g +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +10000 g diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sh b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sh new file mode 100755 index 00000000000..cb937eeb71a --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sh @@ -0,0 +1,70 @@ +#!/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" + +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;" + diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference new file mode 100644 index 00000000000..203b59521f4 --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference @@ -0,0 +1,100 @@ +MergeTree compact +test +Dynamic paths +100000 a +90000 b +80000 c +70000 d +60000 e +Shared data paths +Dynamic paths +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +10000 g +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +10000 g +MergeTree wide +test +Dynamic paths +100000 a +90000 b +80000 c +70000 d +60000 e +Shared data paths +Dynamic paths +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +80000 c +Shared data paths +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +10000 g +Shared data paths +80000 c +70000 d +60000 e +Dynamic paths +200000 f +100000 a +90000 b +Shared data paths +80000 c +70000 d +60000 e +10000 g diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.sh b/tests/queries/0_stateless/03209_json_type_vertical_merges.sh new file mode 100755 index 00000000000..df0bd5bbbdd --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.sh @@ -0,0 +1,70 @@ +#!/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" + +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;" + diff --git a/tests/queries/0_stateless/03210_json_type_alter_add_column.reference b/tests/queries/0_stateless/03210_json_type_alter_add_column.reference new file mode 100644 index 00000000000..907cde709f3 --- /dev/null +++ b/tests/queries/0_stateless/03210_json_type_alter_add_column.reference @@ -0,0 +1,75 @@ +Memory +initial insert +alter add column 1 +0 {} \N {} \N \N +1 {} \N {} \N \N +2 {} \N {} \N \N +insert after alter add column +3 a.b +3 b.c +3 c.d +0 {} \N {} \N \N +1 {} \N {} \N \N +2 {} \N {} \N \N +3 {"a":{"b":"3"}} 3 {"b":"3"} \N \N +4 {"a":{"b":"4"}} 4 {"b":"4"} \N \N +5 {"a":{"b":"5"}} 5 {"b":"5"} \N \N +6 {"b":{"c":"6"}} \N {} 6 \N +7 {"b":{"c":"7"}} \N {} 7 \N +8 {"b":{"c":"8"}} \N {} 8 \N +9 {"c":{"d":"9"}} \N {} \N 9 +10 {"c":{"d":"10"}} \N {} \N 10 +11 {"c":{"d":"11"}} \N {} \N 11 +12 {} \N {} \N \N +13 {} \N {} \N \N +14 {} \N {} \N \N +MergeTree compact +initial insert +alter add column 1 +0 {} \N {} \N \N +1 {} \N {} \N \N +2 {} \N {} \N \N +insert after alter add column +3 a.b +3 b.c +3 c.d +0 {} \N {} \N \N +1 {} \N {} \N \N +2 {} \N {} \N \N +3 {"a":{"b":"3"}} 3 {"b":"3"} \N \N +4 {"a":{"b":"4"}} 4 {"b":"4"} \N \N +5 {"a":{"b":"5"}} 5 {"b":"5"} \N \N +6 {"b":{"c":"6"}} \N {} 6 \N +7 {"b":{"c":"7"}} \N {} 7 \N +8 {"b":{"c":"8"}} \N {} 8 \N +9 {"c":{"d":"9"}} \N {} \N 9 +10 {"c":{"d":"10"}} \N {} \N 10 +11 {"c":{"d":"11"}} \N {} \N 11 +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 {} \N {} \N \N +2 {} \N {} \N \N +insert after alter add column +3 a.b +3 b.c +3 c.d +0 {} \N {} \N \N +1 {} \N {} \N \N +2 {} \N {} \N \N +3 {"a":{"b":"3"}} 3 {"b":"3"} \N \N +4 {"a":{"b":"4"}} 4 {"b":"4"} \N \N +5 {"a":{"b":"5"}} 5 {"b":"5"} \N \N +6 {"b":{"c":"6"}} \N {} 6 \N +7 {"b":{"c":"7"}} \N {} 7 \N +8 {"b":{"c":"8"}} \N {} 8 \N +9 {"c":{"d":"9"}} \N {} \N 9 +10 {"c":{"d":"10"}} \N {} \N 10 +11 {"c":{"d":"11"}} \N {} \N 11 +12 {} \N {} \N \N +13 {} \N {} \N \N +14 {} \N {} \N \N diff --git a/tests/queries/0_stateless/03210_json_type_alter_add_column.sh b/tests/queries/0_stateless/03210_json_type_alter_add_column.sh new file mode 100755 index 00000000000..dfa5de9f091 --- /dev/null +++ b/tests/queries/0_stateless/03210_json_type_alter_add_column.sh @@ -0,0 +1,47 @@ +#!/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;" + diff --git a/tests/queries/0_stateless/03211_nested_json_merges.reference b/tests/queries/0_stateless/03211_nested_json_merges.reference new file mode 100644 index 00000000000..221fcefba66 --- /dev/null +++ b/tests/queries/0_stateless/03211_nested_json_merges.reference @@ -0,0 +1,88 @@ +MergeTree compact + horizontal merge +test +Dynamic paths +300000 c +150000 d +Shared data paths +Dynamic paths +300000 c +Shared data paths +150000 d +Dynamic paths +600000 f +300000 c +150000 e +Shared data paths +150000 d +Dynamic paths +600000 f +Shared data paths +300000 c +150000 d +150000 e +MergeTree wide + horizontal merge +test +Dynamic paths +300000 c +150000 d +Shared data paths +Dynamic paths +300000 c +Shared data paths +150000 d +Dynamic paths +600000 f +300000 c +150000 e +Shared data paths +150000 d +Dynamic paths +600000 f +Shared data paths +300000 c +150000 d +150000 e +MergeTree compact + vertical merge +test +Dynamic paths +300000 c +150000 d +Shared data paths +Dynamic paths +300000 c +Shared data paths +150000 d +Dynamic paths +600000 f +300000 c +150000 e +Shared data paths +150000 d +Dynamic paths +600000 f +Shared data paths +300000 c +150000 d +150000 e +MergeTree wide + vertical merge +test +Dynamic paths +300000 c +150000 d +Shared data paths +Dynamic paths +300000 c +Shared data paths +150000 d +Dynamic paths +600000 f +300000 c +150000 e +Shared data paths +150000 d +Dynamic paths +600000 f +Shared data paths +300000 c +150000 d +150000 e diff --git a/tests/queries/0_stateless/03211_nested_json_merges.sh b/tests/queries/0_stateless/03211_nested_json_merges.sh new file mode 100755 index 00000000000..8e6be01f3fc --- /dev/null +++ b/tests/queries/0_stateless/03211_nested_json_merges.sh @@ -0,0 +1,65 @@ +#!/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" + + +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;" From 184405fc7aa60a3c2626475ac4ff9fdbc0e120ae Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Jul 2024 19:10:08 +0000 Subject: [PATCH 0335/1722] Fix build and style --- src/DataTypes/DataTypeObject.cpp | 2 +- src/DataTypes/DataTypeObject.h | 5 ----- src/DataTypes/DataTypeObjectDeprecated.cpp | 2 +- src/Functions/JSONPaths.cpp | 4 ++-- utils/check-style/aspell-ignore/en/aspell-dict.txt | 6 ++++++ 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index bae1d1935b5..c9378ab395c 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -478,7 +478,7 @@ static DataTypePtr createJSON(const ASTPtr & arguments) void registerDataTypeJSON(DataTypeFactory & factory) { if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) - factory.registerDataType("JSON", createJSON, DataTypeFactory::CaseInsensitive); + factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 2984078370b..5b76d96e0de 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -10,11 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - class DataTypeObject : public IDataType { public: diff --git a/src/DataTypes/DataTypeObjectDeprecated.cpp b/src/DataTypes/DataTypeObjectDeprecated.cpp index e3c1a458eda..a34fe8a628d 100644 --- a/src/DataTypes/DataTypeObjectDeprecated.cpp +++ b/src/DataTypes/DataTypeObjectDeprecated.cpp @@ -80,7 +80,7 @@ void registerDataTypeObjectDeprecated(DataTypeFactory & factory) if (Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) factory.registerSimpleDataType("JSON", [] { return std::make_shared("JSON", false); }, - DataTypeFactory::CaseInsensitive); + DataTypeFactory::Case::Insensitive); } } diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index 0bf6b9ffe12..31699ad1c9c 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -96,7 +96,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override { - if (data_types.size() != 1 ) + if (data_types.size() != 1) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single argument with type JSON", getName()); if (data_types[0]->getTypeId() != TypeIndex::Object) @@ -152,7 +152,7 @@ private: for (const auto & path : dynamic_paths) { /// Don't include path if it contains NULL, because we consider - /// it to be equivalent to the absense of this path in this row. + /// it to be equivalent to the absence of this path in this row. if (!dynamic_path_columns.at(path)->isNullAt(i)) data.insertData(path.data(), path.size()); } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 862f38976ce..723e10ff3ed 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -390,6 +390,8 @@ IsValid JBOD JOINed JOINs +JSONAllPaths +JSONAllPathsWithTypes JSONArrayLength JSONAsObject JSONAsString @@ -404,6 +406,8 @@ JSONCompactStrings JSONCompactStringsEachRow JSONCompactStringsEachRowWithNames JSONCompactStringsEachRowWithNamesAndTypes +JSONDynamicPaths +JSONDynamicPathsWithTypes JSONEachRow JSONEachRowWithProgress JSONExtract @@ -423,6 +427,8 @@ JSONObjectEachRow JSONStrings JSONStringsEachRow JSONStringsEachRowWithProgress +JSONSharedDataPaths +JSONSharedDataPathsWithTypes JSONType JSONs Jaeger From 6e7b5e74589350592a8ee163fc51352d12170db1 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Jul 2024 21:03:52 +0000 Subject: [PATCH 0336/1722] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 723e10ff3ed..ec09eb2e758 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -410,6 +410,7 @@ JSONDynamicPaths JSONDynamicPathsWithTypes JSONEachRow JSONEachRowWithProgress +JSONEmpty JSONExtract JSONExtractArrayRaw JSONExtractBool From 1ce13df07c1292051986e75dcb216488f7bbae35 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Jul 2024 21:47:45 +0000 Subject: [PATCH 0337/1722] Fix build --- src/Columns/ColumnObject.cpp | 6 ------ src/Columns/ColumnObjectDeprecated.cpp | 7 +++++++ src/Columns/ColumnObjectDeprecated.h | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index c4190a1b756..f9a3af601e9 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1197,10 +1197,4 @@ void ColumnObject::fillPathColumnFromSharedData(IColumn & path_column, StringRef } } -void ColumnObject::updateHashFast(SipHash & hash) const -{ - for (const auto & entry : subcolumns) - for (auto & part : entry->data.data) - part->updateHashFast(hash); -} } diff --git a/src/Columns/ColumnObjectDeprecated.cpp b/src/Columns/ColumnObjectDeprecated.cpp index 42f088503f7..6d0bfa0e05e 100644 --- a/src/Columns/ColumnObjectDeprecated.cpp +++ b/src/Columns/ColumnObjectDeprecated.cpp @@ -1101,4 +1101,11 @@ void ColumnObjectDeprecated::finalize() checkObjectHasNoAmbiguosPaths(getKeys()); } +void ColumnObjectDeprecated::updateHashFast(SipHash & hash) const +{ + for (const auto & entry : subcolumns) + for (auto & part : entry->data.data) + part->updateHashFast(hash); +} + } diff --git a/src/Columns/ColumnObjectDeprecated.h b/src/Columns/ColumnObjectDeprecated.h index bae32b82e8a..59614632972 100644 --- a/src/Columns/ColumnObjectDeprecated.h +++ b/src/Columns/ColumnObjectDeprecated.h @@ -251,7 +251,7 @@ public: const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } - void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } + void updateHashFast(SipHash &) const override; void expand(const Filter &, bool) override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } From a296717e14bf43714ab921f8f5f97105104f3055 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Jul 2024 22:57:42 +0000 Subject: [PATCH 0338/1722] Fix tests, add docs for setting type_json_skip_duplicated_paths --- docs/en/operations/settings/settings.md | 8 +++++++- src/Core/SettingsChangesHistory.cpp | 1 + .../Serializations/SerializationObject.cpp | 6 +++--- src/Formats/JSONExtractTree.cpp | 6 +++--- src/Formats/SchemaInferenceUtils.cpp | 9 ++++++++- .../02910_object-json-crash-add-column.sql | 16 ++++++++-------- 6 files changed, 30 insertions(+), 16 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9ddfe985cee..720cf1ce959 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5611,7 +5611,13 @@ Default value: `1GiB`. ## use_json_alias_for_old_object_type -When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/ob) type instead of the new [JSON](../../sql-reference/data-types/json.md) type. +When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/object-json.md) type instead of the new [JSON](../../sql-reference/data-types/json.md) type. This setting requires server restart to take effect when changed. Default value: `false`. + +## type_json_skip_duplicated_paths + +When enabled, ClickHouse will skip duplicated paths during parsing of [JSON](../../sql-reference/data-types/json.md) object. Only the value of the first occurrence of each path will be inserted. + +Default value: `false` diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 50a99c51dcb..47555ec290f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -83,6 +83,7 @@ static std::initializer_listsize() > prev_size) { if (!settings.json.type_json_skip_duplicated_paths) - throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of Object type: {}", path); + throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of JSON type: {}. You can enable setting type_json_skip_duplicated_paths to skip duplicated paths during insert", path); } else { @@ -613,7 +613,7 @@ void SerializationObject::deserializeBinary(IColumn & col, ReadBuffer & istr, co if (dynamic_it->second->size() > prev_size) { if (!settings.json.type_json_skip_duplicated_paths) - throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of Object type: {}", path); + throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of JSON type: {}. You can enable setting type_json_skip_duplicated_paths to skip duplicated paths during insert", path); } dynamic_serialization->deserializeBinary(*dynamic_it->second, istr, settings); @@ -648,7 +648,7 @@ void SerializationObject::deserializeBinary(IColumn & col, ReadBuffer & istr, co if (i != 0 && path == paths_and_values_for_shared_data[i - 1].first) { if (!settings.json.type_json_skip_duplicated_paths) - throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of Object type: {}", path); + throw Exception(ErrorCodes::INCORRECT_DATA, "Found duplicated path during binary deserialization of JSON type: {}. You can enable setting type_json_skip_duplicated_paths to skip duplicated paths during insert", path); } else { diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index e2c127b228d..680588c7825 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1629,7 +1629,7 @@ public: { if (!format_settings.json.type_json_skip_duplicated_paths) { - error = fmt::format("Duplicate path found during parsing JSON object: {}", path); + error = fmt::format("Duplicate path found during parsing JSON object: {}. You can enable setting type_json_skip_duplicated_paths to skip duplicated paths during insert", path); SerializationObject::restoreColumnObject(column_object, prev_size); return false; } @@ -1697,7 +1697,7 @@ private: { if (!format_settings.json.type_json_skip_duplicated_paths) { - error = fmt::format("Duplicate path found during parsing JSON object: {}", current_path); + error = fmt::format("Duplicate path found during parsing JSON object: {}. You can enable setting type_json_skip_duplicated_paths to skip duplicated paths during insert", current_path); return false; } } @@ -1715,7 +1715,7 @@ private: { if (!format_settings.json.type_json_skip_duplicated_paths) { - error = fmt::format("Duplicate path found during parsing JSON object: {}", current_path); + error = fmt::format("Duplicate path found during parsing JSON object: {}. You can enable setting type_json_skip_duplicated_paths to skip duplicated paths during insert", current_path); return false; } } diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index d245200eefa..441e7d77d24 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -780,11 +780,18 @@ namespace /// Check if it's just a number, and if so, don't try to infer DateTime from it, /// because we can interpret this number as a timestamp and it will lead to - /// inferring DateTime instead of simple Int64/Float64 in some cases. + /// inferring DateTime instead of simple Int64 in some cases. if (std::all_of(field.begin(), field.end(), isNumericASCII)) return false; ReadBufferFromString buf(field); + Float64 tmp_float; + /// Check if it's a float value, and if so, don't try to infer DateTime from it, + /// because it will lead to inferring DateTime instead of simple Float64 in some cases. + if (tryReadFloatText(tmp_float, buf) && buf.eof()) + return false; + + buf.seek(0, SEEK_SET); /// Return position to the beginning DateTime64 tmp; switch (settings.date_time_input_format) { diff --git a/tests/queries/0_stateless/02910_object-json-crash-add-column.sql b/tests/queries/0_stateless/02910_object-json-crash-add-column.sql index b2d64be1676..0573ac928f8 100644 --- a/tests/queries/0_stateless/02910_object-json-crash-add-column.sql +++ b/tests/queries/0_stateless/02910_object-json-crash-add-column.sql @@ -9,10 +9,10 @@ ORDER BY i; INSERT INTO test02910 (i, jString) SELECT 1, '{"a":"123"}'; -ALTER TABLE test02910 ADD COLUMN j2 Tuple(JSON) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } -ALTER TABLE test02910 ADD COLUMN j2 Tuple(Float64, JSON); -- { serverError SUPPORT_IS_DISABLED } -ALTER TABLE test02910 ADD COLUMN j2 Tuple(Array(Tuple(JSON))) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } -ALTER TABLE test02910 ADD COLUMN j2 JSON default jString; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910 ADD COLUMN j2 Tuple(Object('json')) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910 ADD COLUMN j2 Tuple(Float64, Object('json')); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910 ADD COLUMN j2 Tuple(Array(Tuple(Object('json')))) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910 ADD COLUMN j2 Object('json') default jString; -- { serverError SUPPORT_IS_DISABLED } -- If we would allow adding a column with dynamic subcolumns the subsequent select would crash the server. -- SELECT * FROM test02910; @@ -37,10 +37,10 @@ INSERT INTO test02910_second SELECT number, number, '2023-10-28 11:11:11.11111', INSERT INTO test02910_second SELECT number, number, '2023-10-28 11:11:11.11111', ['c', 'd'] FROM numbers(10); INSERT INTO test02910_second SELECT number, number, '2023-10-28 11:11:11.11111', [] FROM numbers(10); -ALTER TABLE test02910_second ADD COLUMN `tags_json` Tuple(JSON) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } -ALTER TABLE test02910_second ADD COLUMN `tags_json` Tuple(Float64, JSON); -- { serverError SUPPORT_IS_DISABLED } -ALTER TABLE test02910_second ADD COLUMN `tags_json` Tuple(Array(Tuple(JSON))) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } -ALTER TABLE test02910_second ADD COLUMN `tags_json` JSON; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910_second ADD COLUMN `tags_json` Tuple(Object('json')) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910_second ADD COLUMN `tags_json` Tuple(Float64, Object('json')); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910_second ADD COLUMN `tags_json` Tuple(Array(Tuple(Object('json')))) DEFAULT jString; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE test02910_second ADD COLUMN `tags_json` Object('json'); -- { serverError SUPPORT_IS_DISABLED } -- If we would allow adding a column with dynamic subcolumns the subsequent select would crash the server. -- SELECT * FROM test02910; From 3b45916470affeb555ae20a4d557ea9de5075f50 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 05:46:53 +0200 Subject: [PATCH 0339/1722] What if we tighten limits for functional tests? --- tests/config/install.sh | 1 + tests/config/users.d/limits.xml | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/config/users.d/limits.xml diff --git a/tests/config/install.sh b/tests/config/install.sh index 1b0edc5fc16..265b9248f4a 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -93,6 +93,7 @@ ln -sf $SRC_PATH/users.d/prefetch_settings.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/nonconst_timezone.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/allow_introspection_functions.yaml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/replicated_ddl_entry.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/limits.xml $DEST_SERVER_PATH/users.d/ if [[ -n "$USE_OLD_ANALYZER" ]] && [[ "$USE_OLD_ANALYZER" -eq 1 ]]; then ln -sf $SRC_PATH/users.d/analyzer.xml $DEST_SERVER_PATH/users.d/ diff --git a/tests/config/users.d/limits.xml b/tests/config/users.d/limits.xml new file mode 100644 index 00000000000..f44c73241ab --- /dev/null +++ b/tests/config/users.d/limits.xml @@ -0,0 +1,8 @@ + + + + 5G + 20000000 + + + From 8c264230e30bf97f6bac999401cd594b31e09977 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 07:20:33 +0200 Subject: [PATCH 0340/1722] Loosen the limit --- docker/test/stateful/run.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 857385f4715..72a8f31ab71 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -118,8 +118,8 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] clickhouse-client --query "CREATE TABLE test.hits AS datasets.hits_v1" clickhouse-client --query "CREATE TABLE test.visits AS datasets.visits_v1" - clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" - clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" clickhouse-client --query "DROP TABLE datasets.hits_v1" clickhouse-client --query "DROP TABLE datasets.visits_v1" @@ -191,16 +191,16 @@ else ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" - clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0" clickhouse-client --query "DROP TABLE datasets.visits_v1 SYNC" clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC" else clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" fi - clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0" + clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0" fi clickhouse-client --query "SHOW TABLES FROM test" From c59949d057bba311f040f7b6386b1f6b481a23dd Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 22 Jul 2024 09:50:47 +0000 Subject: [PATCH 0341/1722] Add different iceberg tables --- .../DataLakes/registerDataLakeStorages.cpp | 60 +++++++++++++++++-- src/TableFunctions/ITableFunctionDataLake.h | 27 ++++++++- .../registerDataLakeTableFunctions.cpp | 27 +++++---- 3 files changed, 96 insertions(+), 18 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index 0fa6402e892..e75ab3201b8 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -2,10 +2,12 @@ #if USE_AWS_S3 -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include namespace DB @@ -22,6 +24,54 @@ void registerStorageIceberg(StorageFactory & factory) auto configuration = std::make_shared(); StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergS3", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergAzure", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); + + factory.registerStorage( + "IcebergLocal", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); @@ -29,7 +79,7 @@ void registerStorageIceberg(StorageFactory & factory) { .supports_settings = false, .supports_schema_inference = true, - .source_access_type = AccessType::S3, + .source_access_type = AccessType::FILE, }); } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index fe6e5b3e593..db8287f97bf 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -76,6 +76,21 @@ struct TableFunctionIcebergName static constexpr auto name = "iceberg"; }; +struct TableFunctionIcebergS3Name +{ + static constexpr auto name = "icebergS3"; +}; + +struct TableFunctionIcebergAzureName +{ + static constexpr auto name = "icebergAzure"; +}; + +struct TableFunctionIcebergLocalName +{ + static constexpr auto name = "icebergLocal"; +}; + struct TableFunctionDeltaLakeName { static constexpr auto name = "deltaLake"; @@ -86,14 +101,20 @@ struct TableFunctionHudiName static constexpr auto name = "hudi"; }; -#if USE_AWS_S3 #if USE_AVRO +# if USE_AWS_S3 using TableFunctionIceberg = ITableFunctionDataLake; +using TableFunctionIcebergS3 = ITableFunctionDataLake; +# endif +# if USE_AZURE_BLOB_STORAGE +using TableFunctionIcebergAzure = ITableFunctionDataLake; +# endif +using TableFunctionIcebergLocal = ITableFunctionDataLake; #endif -#if USE_PARQUET +#if USE_AWS_S3 +# if USE_PARQUET using TableFunctionDeltaLake = ITableFunctionDataLake; #endif using TableFunctionHudi = ITableFunctionDataLake; #endif - } diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 15a6668f434..40561a1e075 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -4,24 +4,31 @@ namespace DB { -#if USE_AWS_S3 #if USE_AVRO void registerTableFunctionIceberg(TableFunctionFactory & factory) { +# if USE_AWS_S3 factory.registerFunction( - { - .documentation = - { - .description=R"(The table function can be used to read the Iceberg table stored on object store.)", + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"} - }, - .allow_readonly = false - }); + .categories{"DataLake"}}, + .allow_readonly = false}); + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", + .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); + +# endif +# if USE_AZURE_BLOB_STORAGE +# endif } #endif -#if USE_PARQUET +#if USE_AWS_S3 +# if USE_PARQUET void registerTableFunctionDeltaLake(TableFunctionFactory & factory) { factory.registerFunction( From 99026efcdcc890acc4540eea3e7c94a33d3f99fc Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 10:51:15 +0000 Subject: [PATCH 0342/1722] Improve JSON parsing by different type inference logic --- src/Formats/JSONExtractTree.cpp | 98 ++++++++++++++++++++++++--------- src/Formats/JSONExtractTree.h | 5 +- src/Functions/FunctionsJSON.cpp | 2 +- 3 files changed, 77 insertions(+), 28 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 680588c7825..a8980e785de 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -126,7 +126,7 @@ void jsonElementToString(const typename JSONParser::Element & element, WriteBuff template bool tryGetNumericValueFromJSONElement( - NumberType & value, const typename JSONParser::Element & element, bool convert_bool_to_integer, String & error) + NumberType & value, const typename JSONParser::Element & element, bool convert_bool_to_integer, bool allow_type_conversion, String & error) { switch (element.type()) { @@ -138,7 +138,7 @@ bool tryGetNumericValueFromJSONElement( /// But it will be more convenient for user to perform conversion. value = static_cast(element.getDouble()); } - else if (!accurate::convertNumeric(element.getDouble(), value)) + else if (!allow_type_conversion || !accurate::convertNumeric(element.getDouble(), value)) { error = fmt::format("cannot convert double value {} to {}", element.getDouble(), TypeName); return false; @@ -161,7 +161,7 @@ bool tryGetNumericValueFromJSONElement( case ElementType::BOOL: if constexpr (is_integer) { - if (convert_bool_to_integer) + if (convert_bool_to_integer && allow_type_conversion) { value = static_cast(element.getBool()); break; @@ -169,7 +169,11 @@ bool tryGetNumericValueFromJSONElement( } error = fmt::format("cannot convert bool value to {}", TypeName); return false; - case ElementType::STRING: { + case ElementType::STRING: + { + if (!allow_type_conversion) + return false; + auto rb = ReadBufferFromMemory{element.getString()}; if constexpr (std::is_floating_point_v) { @@ -244,8 +248,16 @@ public: return false; } + if (is_bool_type && !insert_settings.allow_type_conversion) + { + if (!element.isBool()) + return false; + assert_cast &>(column).insertValue(element.getBool()); + return true; + } + NumberType value; - if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || is_bool_type, error)) + if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || is_bool_type, insert_settings.allow_type_conversion, error)) { if (error.empty()) error = fmt::format("cannot read {} value from JSON element: {}", TypeName, jsonElementToString(element, format_settings)); @@ -292,8 +304,17 @@ public: return false; } + if (this->is_bool_type && !insert_settings.allow_type_conversion) + { + if (!element.isBool()) + return false; + UInt8 value = element.getBool(); + assert_cast(column).insertData(reinterpret_cast(&value), sizeof(value)); + return true; + } + NumberType value; - if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || this->is_bool_type, error)) + if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || this->is_bool_type, insert_settings.allow_type_conversion, error)) { if (error.empty()) error = fmt::format("cannot read {} value from JSON element: {}", TypeName, jsonElementToString(element, format_settings)); @@ -319,7 +340,7 @@ public: bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, - const JSONExtractInsertSettings &, + const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { @@ -336,6 +357,9 @@ public: if (!element.isString()) { + if (!insert_settings.allow_type_conversion) + return false; + auto & col_str = assert_cast(column); auto & chars = col_str.getChars(); WriteBufferFromVector buf(chars, AppendModeTag()); @@ -363,7 +387,7 @@ public: bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, - const JSONExtractInsertSettings &, + const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { @@ -381,6 +405,9 @@ public: if (!element.isString()) { + if (!insert_settings.allow_type_conversion) + return false; + auto value = jsonElementToString(element, format_settings); assert_cast(column).insertData(value.data(), value.size()); } @@ -405,7 +432,7 @@ public: bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, - const JSONExtractInsertSettings &, + const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { @@ -422,7 +449,11 @@ public: } if (!element.isString()) + { + if (!insert_settings.allow_type_conversion) + return false; return checkValueSizeAndInsert(column, jsonElementToString(element, format_settings), error); + } return checkValueSizeAndInsert(column, element.getString(), error); } @@ -453,7 +484,7 @@ public: bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, - const JSONExtractInsertSettings &, + const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { @@ -469,7 +500,11 @@ public: } if (!element.isString()) + { + if (!insert_settings.allow_type_conversion) + return false; return checkValueSizeAndInsert(column, jsonElementToString(element, format_settings), error); + } return checkValueSizeAndInsert(column, element.getString(), error); } @@ -633,7 +668,7 @@ public: bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, - const JSONExtractInsertSettings &, + const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { @@ -652,7 +687,7 @@ public: return false; } } - else if (element.isUInt64()) + else if (element.isUInt64() && insert_settings.allow_type_conversion) { value = element.getUInt64(); } @@ -715,7 +750,8 @@ public: case ElementType::INT64: value = convertToDecimal, DataTypeDecimal>(element.getInt64(), scale); break; - case ElementType::STRING: { + case ElementType::STRING: + { auto rb = ReadBufferFromMemory{element.getString()}; if (!SerializationDecimal::tryReadText(value, rb, DecimalUtils::max_precision, scale)) { @@ -724,7 +760,8 @@ public: } break; } - case ElementType::NULL_VALUE: { + case ElementType::NULL_VALUE: + { if (!format_settings.null_as_default) { error = "cannot convert null to Decimal value"; @@ -759,7 +796,7 @@ public: bool insertResultToColumn( IColumn & column, const typename JSONParser::Element & element, - const JSONExtractInsertSettings &, + const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { @@ -780,6 +817,9 @@ public: } else { + if (!insert_settings.allow_type_conversion) + return false; + switch (element.type()) { case ElementType::DOUBLE: @@ -1373,7 +1413,20 @@ public: auto & variant_column = column_dynamic.getVariantColumn(); auto & variant_info = column_dynamic.getVariantInfo(); - /// First, infer ClickHouse type for this element and add it as a new variant. + + /// First, try to insert element into current variants but with no types conversion. + /// We want to avoid inferring the type on each row, so if we can insert this element into + /// any existing variant with no types conversion (like Integer -> String, Double -> Integer, etc) + /// we will do it and won't try to infer the type. + auto it = json_extract_nodes_cache.find(variant_info.variant_name); + if (it == json_extract_nodes_cache.end()) + it = json_extract_nodes_cache.emplace(variant_info.variant_name, buildJSONExtractTree(variant_info.variant_type, "Dynamic inference")).first; + auto insert_settings_with_no_type_conversion = insert_settings; + insert_settings_with_no_type_conversion.allow_type_conversion = false; + if (it->second->insertResultToColumn(variant_column, element, insert_settings_with_no_type_conversion, format_settings, error)) + return true; + + /// We couldn't insert element into current variants, infer ClickHouse type for this element and add it as a new variant. auto element_type = removeNullable(elementToDataType(element, format_settings)); if (!checkIfTypeIsComplete(element_type)) { @@ -1387,7 +1440,7 @@ public: auto element_type_name = element_type->getName(); if (column_dynamic.addNewVariant(element_type, element_type_name)) { - auto it = json_extract_nodes_cache.find(element_type_name); + it = json_extract_nodes_cache.find(element_type_name); if (it == json_extract_nodes_cache.end()) it = json_extract_nodes_cache.emplace(element_type_name, buildJSONExtractTree(element_type, "Dynamic inference")).first; auto global_discriminator = variant_info.variant_name_to_discriminator.at(element_type_name); @@ -1399,14 +1452,7 @@ public: return true; } - /// We couldn't add new variant. Try to insert element into current variants. - auto it = json_extract_nodes_cache.find(variant_info.variant_name); - if (it == json_extract_nodes_cache.end()) - it = json_extract_nodes_cache.emplace(variant_info.variant_name, buildJSONExtractTree(variant_info.variant_type, "Dynamic inference")).first; - if (it->second->insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) - return true; - - /// We couldn't insert element into any existing variant, add String variant and read value as String. + /// We couldn't add a new variant, add String variant and read value as String. column_dynamic.addStringVariant(); auto string_global_discriminator = variant_info.variant_name_to_discriminator.at("String"); auto & string_column = variant_column.getVariantByGlobalDiscriminator(string_global_discriminator); @@ -1962,7 +2008,7 @@ template std::unique_ptr> buildJSONExtractTr #if USE_RAPIDJSON template void jsonElementToString(const RapidJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); -template bool tryGetNumericValueFromJSONElement(Float64 & value, const RapidJSONParser::Element & element, bool convert_bool_to_integer, String & error); +template bool tryGetNumericValueFromJSONElement(Float64 & value, const RapidJSONParser::Element & element, bool convert_bool_to_integer, bool allow_type_conversion, String & error); #else template void jsonElementToString(const DummyJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); diff --git a/src/Formats/JSONExtractTree.h b/src/Formats/JSONExtractTree.h index b5e82506548..89f2d191dfb 100644 --- a/src/Formats/JSONExtractTree.h +++ b/src/Formats/JSONExtractTree.h @@ -17,6 +17,9 @@ struct JSONExtractInsertSettings /// For example, if we have [1, "hello", 2] and type Array(UInt32), /// we will insert [1, 0, 2] in the column. Used in all JSONExtract functions. bool insert_default_on_invalid_elements_in_complex_types = false; + /// If false, JSON value will be inserted into column only if type of the value is + /// the same as column type (no conversions like Integer -> String, Integer -> Float, etc). + bool allow_type_conversion = true; }; template @@ -36,6 +39,6 @@ template void jsonElementToString(const typename JSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); template -bool tryGetNumericValueFromJSONElement(NumberType & value, const typename JSONParser::Element & element, bool convert_bool_to_integer, String & error); +bool tryGetNumericValueFromJSONElement(NumberType & value, const typename JSONParser::Element & element, bool convert_bool_to_integer, bool allow_type_conversion, String & error); } diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 47040545677..2681f278958 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -739,7 +739,7 @@ public: { NumberType value; - tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, error); + tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, true, error); auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); return true; From 2626880f6e0ed23e043d1f5756c6ab9786993426 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 14:28:07 +0000 Subject: [PATCH 0343/1722] Fix special builds and tests --- src/Columns/ColumnObject.cpp | 28 ++-- src/Columns/ColumnObject.h | 2 +- src/DataTypes/DataTypeObject.cpp | 37 ++++-- src/DataTypes/DataTypeObject.h | 2 +- .../Serializations/SerializationJSON.h | 2 +- .../Serializations/SerializationObject.cpp | 2 +- .../Serializations/SerializationObject.h | 2 +- .../Serializations/SerializationSubObject.h | 1 - src/Formats/JSONExtractTree.cpp | 65 ++------- src/Formats/SchemaInferenceUtils.cpp | 123 +++++------------- src/Functions/JSONPaths.cpp | 1 + .../InterpreterShowColumnsQuery.cpp | 1 + src/Parsers/ASTObjectTypeArgument.cpp | 18 +-- .../MergeTree/MergeTreeReaderWide.cpp | 6 + src/Storages/MergeTree/checkDataPart.cpp | 3 + .../01825_new_type_json_10.reference | 3 +- .../0_stateless/01825_new_type_json_10.sql | 4 +- .../0_stateless/01825_new_type_json_11.sh | 8 +- .../0_stateless/01825_new_type_json_12.sh | 8 +- .../0_stateless/01825_new_type_json_13.sh | 4 +- .../0_stateless/01825_new_type_json_6.sh | 6 +- .../0_stateless/01825_new_type_json_btc.sh | 6 +- .../01825_new_type_json_ghdata.reference | 12 ++ .../0_stateless/01825_new_type_json_ghdata.sh | 26 ++++ ...1825_new_type_json_ghdata_insert_select.sh | 4 +- .../01825_new_type_json_nbagames.reference | 14 +- .../01825_new_type_json_nbagames.sh | 6 +- .../0_stateless/01825_type_json_10.reference | 9 -- .../0_stateless/01825_type_json_10.sql | 2 +- .../0_stateless/01825_type_json_ghdata.sh | 2 +- .../0_stateless/01825_type_json_nbagames.sh | 2 +- .../02421_type_json_async_insert.sh | 2 +- ...02482_json_nested_arrays_with_same_keys.sh | 2 +- ...w_columns_called_from_clickhouse.reference | 2 +- 34 files changed, 178 insertions(+), 237 deletions(-) create mode 100644 tests/queries/0_stateless/01825_new_type_json_ghdata.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_ghdata.sh diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index f9a3af601e9..2fb5831188d 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -18,13 +18,13 @@ namespace ErrorCodes namespace { -static const FormatSettings & getFormatSettings() +const FormatSettings & getFormatSettings() { static const FormatSettings settings; return settings; } -static const std::shared_ptr & getDynamicSerialization() +const std::shared_ptr & getDynamicSerialization() { static const std::shared_ptr dynamic_serialization = std::make_shared(); return dynamic_serialization; @@ -394,7 +394,7 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) const auto & src_object_column = assert_cast(src); /// First, insert typed paths, they must be the same for both columns. - for (auto & [path, column] : src_object_column.typed_paths) + for (const auto & [path, column] : src_object_column.typed_paths) typed_paths[path]->insertFrom(*column, n); /// Second, insert dynamic paths and extend them if needed. @@ -428,7 +428,7 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l const auto & src_object_column = assert_cast(src); /// First, insert typed paths, they must be the same for both columns. - for (auto & [path, column] : src_object_column.typed_paths) + for (const auto & [path, column] : src_object_column.typed_paths) typed_paths[path]->insertRangeFrom(*column, start, length); /// Second, insert dynamic paths and extend them if needed. @@ -898,9 +898,9 @@ void ColumnObject::ensureOwnership() size_t ColumnObject::byteSize() const { size_t size = 0; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) size += column->byteSize(); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) size += column->byteSize(); size += shared_data->byteSize(); return size; @@ -909,9 +909,9 @@ size_t ColumnObject::byteSize() const size_t ColumnObject::byteSizeAt(size_t n) const { size_t size = 0; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) size += column->byteSizeAt(n); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) size += column->byteSizeAt(n); size += shared_data->byteSizeAt(n); return size; @@ -920,9 +920,9 @@ size_t ColumnObject::byteSizeAt(size_t n) const size_t ColumnObject::allocatedBytes() const { size_t size = 0; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) size += column->allocatedBytes(); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) size += column->allocatedBytes(); size += shared_data->allocatedBytes(); return size; @@ -1040,9 +1040,9 @@ void ColumnObject::finalize() bool ColumnObject::isFinalized() const { bool finalized = true; - for (auto & [_, column] : typed_paths) + for (const auto & [_, column] : typed_paths) finalized &= column->isFinalized(); - for (auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths) finalized &= column->isFinalized(); finalized &= shared_data->isFinalized(); return finalized; @@ -1144,8 +1144,8 @@ size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const Column Iterator() = delete; Iterator(const ColumnString * data_, size_t index_) : data(data_), index(index_) {} - Iterator(const Iterator & rhs) : data(rhs.data), index(rhs.index) {} - Iterator & operator=(const Iterator & rhs) { data = rhs.data; index = rhs.index; return *this; } + Iterator(const Iterator & rhs) = default; + Iterator & operator=(const Iterator & rhs) = default; inline Iterator& operator+=(difference_type rhs) { index += rhs; return *this;} inline StringRef operator*() const {return data->getDataAt(index);} diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index b130c7f4468..41739782266 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -188,7 +188,7 @@ public: static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); private: - void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector & dynamic_paths_to_shared_data, size_t start, size_t length); + void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector & src_dynamic_paths_for_shared_data, size_t start, size_t length); void serializePathAndValueIntoArena(Arena & arena, const char *& begin, StringRef path, StringRef value, StringRef & res) const; /// Map path -> column for paths with explicitly specified types. diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index c9378ab395c..a6c5c201f36 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -196,10 +196,29 @@ MutableColumnPtr DataTypeObject::createColumn() const namespace { +/// It is possible to have nested JSON object inside Dynamic. For example when we have an array of JSON objects. +/// During type inference in parsing in case of creating nested JSON objects, we reduce max_dynamic_paths/max_dynamic_types by factors +/// NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR/NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR. +/// So the type name will actually be JSON(max_dynamic_paths=N, max_dynamic_types=M). But we want the user to be able to query it +/// using json.array.:`Array(JSON)`.some.path without specifying max_dynamic_paths/max_dynamic_types. +/// To support it, we do a trick - we replace JSON name in subcolumn to JSON(max_dynamic_paths=N, max_dynamic_types=M), because we know +/// the exact values of max_dynamic_paths/max_dynamic_types for it. +void replaceJSONTypeNameIfNeeded(String & type_name, size_t max_dynamic_paths, size_t max_dynamic_types) +{ + auto pos = type_name.find("JSON"); + while (pos != String::npos) + { + /// Replace only if we don't already have parameters in JSON type declaration. + if (pos + 4 == type_name.size() || type_name[pos + 4] != '(') + type_name.replace(pos, 4, fmt::format("JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); + pos = type_name.find("JSON", pos + 4); + } +} + /// JSON subcolumn name with Dynamic type subcolumn looks like this: /// "json.some.path.:`Type_name`.some.subcolumn". /// We back quoted type name during identifier parsing so we can distinguish type subcolumn and path element ":TypeName". -std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subcolumn_name) +std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subcolumn_name, size_t max_dynamic_paths, size_t max_dynamic_types) { /// Try to find dynamic type subcolumn in a form .:`Type`. auto pos = subcolumn_name.find(".:`"); @@ -212,6 +231,8 @@ std::pair splitPathAndDynamicTypeSubcolumn(std::string_view subc if (!tryReadBackQuotedString(dynamic_subcolumn, buf)) return {String(subcolumn_name), ""}; + replaceJSONTypeNameIfNeeded(dynamic_subcolumn, max_dynamic_paths, max_dynamic_types); + /// If there is more data in the buffer - it's subcolumn of a type, append it to the type name. if (!buf.eof()) dynamic_subcolumn += String(buf.position(), buf.available()); @@ -333,7 +354,7 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu } /// Split requested subcolumn to the JSON path and Dynamic type subcolumn. - auto [path, path_subcolumn] = splitPathAndDynamicTypeSubcolumn(subcolumn_name); + auto [path, path_subcolumn] = splitPathAndDynamicTypeSubcolumn(subcolumn_name, max_dynamic_paths, max_dynamic_types); std::unique_ptr res; if (auto it = typed_paths.find(path); it != typed_paths.end()) { @@ -373,18 +394,6 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu /// Get subcolumn for Dynamic type if needed. if (!path_subcolumn.empty()) { - /// It is possible to have nested JSON object inside Dynamic. For example when we have an array of JSON objects. - /// During parsing in case of creating nested JSON objects, we reduce max_dynamic_paths/max_dynamic_types by NESTED_OBJECT_REDUCE_FACTOR factor. - /// So the type name will actually be JSON(max_dynamic_paths=N, max_dynamic_types=M). But we want the user to be able to query it - /// using json.array.:`Array(JSON)`.some.path without specifying max_dynamic_paths/max_dynamic_types. - /// To support it, we do a trick - we replace JSON name in subcolumn to JSON(max_dynamic_paths=N, max_dynamic_types=M), because we know - /// the exact values of max_dynamic_paths/max_dynamic_types for it. - auto pos = path_subcolumn.find("JSON"); - /// We want to replace JSON keyword only in the first subcolumn part before the first dot. - auto first_dot_pos = path_subcolumn.find('.'); - if (pos != path_subcolumn.npos && (first_dot_pos == path_subcolumn.npos || pos < first_dot_pos)) - path_subcolumn.replace(pos, 4, fmt::format("JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); - res = res->type->getSubcolumnData(path_subcolumn, *res, throw_if_null); if (!res) return nullptr; diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 5b76d96e0de..6eea777ed26 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -23,7 +23,7 @@ public: static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR = 4; static constexpr size_t NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR = 2; - DataTypeObject( + explicit DataTypeObject( const SchemaFormat & schema_format_, const std::unordered_map & typed_paths_ = {}, const std::unordered_set & paths_to_skip_ = {}, diff --git a/src/DataTypes/Serializations/SerializationJSON.h b/src/DataTypes/Serializations/SerializationJSON.h index aee1413bdd3..934c94527f3 100644 --- a/src/DataTypes/Serializations/SerializationJSON.h +++ b/src/DataTypes/Serializations/SerializationJSON.h @@ -31,7 +31,7 @@ public: void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - virtual void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; + void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 8a9860fd592..7c8c23e8a29 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -48,7 +48,7 @@ bool SerializationObject::shouldSkipPath(const String & path) const if (paths_to_skip.contains(path)) return true; - auto it = std::lower_bound(sorted_typed_paths.begin(), sorted_typed_paths.end(), path); + auto it = std::lower_bound(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end(), path); if (it != sorted_paths_to_skip.end() && it != sorted_paths_to_skip.begin() && path.starts_with(*std::prev(it))) return true; diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index b279fab2603..faf15aa3260 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -101,7 +101,7 @@ private: { String path; - TypedPathSubcolumnCreator(const String & path_) : path(path_) {} + explicit TypedPathSubcolumnCreator(const String & path_) : path(path_) {} DataTypePtr create(const DataTypePtr & prev) const override { return prev; } ColumnPtr create(const ColumnPtr & prev) const override { return prev; } diff --git a/src/DataTypes/Serializations/SerializationSubObject.h b/src/DataTypes/Serializations/SerializationSubObject.h index 8a5582536e7..10973b48957 100644 --- a/src/DataTypes/Serializations/SerializationSubObject.h +++ b/src/DataTypes/Serializations/SerializationSubObject.h @@ -67,7 +67,6 @@ private: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Text/binary serialization is not implemented for object sub-object subcolumn"); } -private: String path_prefix; std::unordered_map typed_paths_serializations; SerializationPtr dynamic_serialization; diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index a8980e785de..640ab1b0acd 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1389,7 +1389,7 @@ template class DynamicNode : public JSONExtractTreeNode { public: - DynamicNode( + explicit DynamicNode( size_t max_dynamic_paths_for_object_ = DataTypeObject::DEFAULT_MAX_SEPARATELY_STORED_PATHS, size_t max_dynamic_types_for_object_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES) : max_dynamic_paths_for_object(max_dynamic_paths_for_object_), max_dynamic_types_for_object(max_dynamic_types_for_object_) @@ -1412,7 +1412,7 @@ public: } auto & variant_column = column_dynamic.getVariantColumn(); - auto & variant_info = column_dynamic.getVariantInfo(); + const auto & variant_info = column_dynamic.getVariantInfo(); /// First, try to insert element into current variants but with no types conversion. /// We want to avoid inferring the type on each row, so if we can insert this element into @@ -1486,20 +1486,20 @@ private: switch (element.type()) { case ElementType::NULL_VALUE: - return getNullType(); + return std::make_shared(std::make_shared()); case ElementType::BOOL: - return getBoolType(); + return DataTypeFactory::instance().get("Bool"); case ElementType::INT64: { - auto type = getInt64Type(); + auto type = std::make_shared(); if (element.getInt64() < 0) json_inference_info.negative_integers.insert(type.get()); return type; } case ElementType::UINT64: - return getUInt64Type(); + return std::make_shared(); case ElementType::DOUBLE: - return getFloat64Type(); + return std::make_shared(); case ElementType::STRING: { auto data = element.getString(); @@ -1516,7 +1516,7 @@ private: } } - return getStringType(); + return std::make_shared(); } case ElementType::ARRAY: { @@ -1527,7 +1527,7 @@ private: types.push_back(elementToDataTypeImpl(value, format_settings, json_inference_info)); if (types.empty()) - return getEmptyArrayType(); + return std::make_shared(std::make_shared()); if (checkIfTypesAreEqual(types)) return std::make_shared(types.back()); @@ -1561,51 +1561,6 @@ private: } } - /// During schema inference we create shared_ptr to the some data types quite a lot. - /// Single creating of such shared_ptr is not expensive, but when it happens on each - /// column on each row, it can be noticeable. - const DataTypePtr & getBoolType() const - { - static const DataTypePtr bool_type = DataTypeFactory::instance().get("Bool"); - return bool_type; - } - - const DataTypePtr & getStringType() const - { - static const DataTypePtr string_type = std::make_shared(); - return string_type; - } - - const DataTypePtr & getInt64Type() const - { - static const DataTypePtr int64_type = std::make_shared(); - return int64_type; - } - - const DataTypePtr & getUInt64Type() const - { - static const DataTypePtr uint64_type = std::make_shared(); - return uint64_type; - } - - const DataTypePtr & getFloat64Type() const - { - static const DataTypePtr float64_type = std::make_shared(); - return float64_type; - } - - const DataTypePtr & getNullType() const - { - static const DataTypePtr null_type = std::make_shared(std::make_shared()); - return null_type; - } - - const DataTypePtr & getEmptyArrayType() const - { - static const DataTypePtr empty_array_type = std::make_shared(std::make_shared()); - return empty_array_type; - } - size_t max_dynamic_paths_for_object; size_t max_dynamic_types_for_object; @@ -1772,7 +1727,7 @@ private: } } /// Try to add a new dynamic path. - else if (auto dynamic_column = column_object.tryToAddNewDynamicPath(current_path)) + else if (auto * dynamic_column = column_object.tryToAddNewDynamicPath(current_path)) { if (!dynamic_node->insertResultToColumn(*dynamic_column, element, insert_settings, format_settings, error)) { diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 441e7d77d24..28d60bf078b 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -36,63 +36,6 @@ namespace ErrorCodes namespace { - /// During schema inference we create shared_ptr to the some data types quite a lot. - /// Single creating of such shared_ptr is not expensive, but when it happens on each - /// column on each row, it can be noticeable. - const DataTypePtr & getBoolType() - { - static const DataTypePtr bool_type = DataTypeFactory::instance().get("Bool"); - return bool_type; - } - - const DataTypePtr & getStringType() - { - static const DataTypePtr string_type = std::make_shared(); - return string_type; - } - - const DataTypePtr & getInt64Type() - { - static const DataTypePtr int64_type = std::make_shared(); - return int64_type; - } - - const DataTypePtr & getUInt64Type() - { - static const DataTypePtr uint64_type = std::make_shared(); - return uint64_type; - } - - const DataTypePtr & getFloat64Type() - { - static const DataTypePtr float64_type = std::make_shared(); - return float64_type; - } - - const DataTypePtr & getDateType() - { - static const DataTypePtr date_type = std::make_shared(); - return date_type; - } - - const DataTypePtr & getDateTime64Type() - { - static const DataTypePtr date_type = std::make_shared(9); - return date_type; - } - - const DataTypePtr & getNullType() - { - static const DataTypePtr null_type = std::make_shared(std::make_shared()); - return null_type; - } - - const DataTypePtr & getEmptyArrayType() - { - static const DataTypePtr empty_array_type = std::make_shared(std::make_shared()); - return empty_array_type; - } - /// Special data type that represents JSON object as a set of paths and their types. /// It supports merging two JSON objects and creating Named Tuple from itself. /// It's used only for schema inference of Named Tuples from JSON objects. @@ -265,7 +208,7 @@ namespace if (leaf_type && !isNothing(removeNullable(leaf_type)) && !nodes.empty()) { if (use_string_type_for_ambiguous_paths) - return getStringType(); + return std::make_shared(); throw Exception( ErrorCodes::INCORRECT_DATA, @@ -331,7 +274,7 @@ namespace bool is_negative = json_info && json_info->negative_integers.contains(type.get()); have_negative_integers |= is_negative; if (!is_negative) - type = getUInt64Type(); + type = std::make_shared(); } } @@ -352,7 +295,7 @@ namespace WhichDataType which(type); if (which.isInt64() || which.isUInt64()) { - auto new_type = getFloat64Type(); + const auto & new_type = std::make_shared(); if (json_info && json_info->numbers_parsed_from_json_strings.erase(type.get())) json_info->numbers_parsed_from_json_strings.insert(new_type.get()); type = new_type; @@ -376,7 +319,7 @@ namespace for (auto & type : data_types) { if (isDate(type) || isDateTime64(type)) - type = getStringType(); + type = std::make_shared(); } type_indexes.erase(TypeIndex::Date); @@ -390,7 +333,7 @@ namespace for (auto & type : data_types) { if (isDate(type)) - type = getDateTime64Type(); + type = std::make_shared(9); } type_indexes.erase(TypeIndex::Date); @@ -412,7 +355,7 @@ namespace if (isNumber(type) && (settings.json.read_numbers_as_strings || !json_info || json_info->numbers_parsed_from_json_strings.contains(type.get()))) - type = getStringType(); + type = std::make_shared(); } updateTypeIndexes(data_types, type_indexes); @@ -435,11 +378,11 @@ namespace if (isBool(type)) { if (have_signed_integers) - type = getInt64Type(); + type = std::make_shared(); else if (have_unsigned_integers) - type = getUInt64Type(); + type = std::make_shared(); else - type = getFloat64Type(); + type = std::make_shared(); } } @@ -456,7 +399,7 @@ namespace for (auto & type : data_types) { if (isBool(type)) - type = getStringType(); + type = std::make_shared(); } type_indexes.erase(TypeIndex::UInt8); @@ -606,7 +549,7 @@ namespace for (auto & type : data_types) { if (isMap(type)) - type = getStringType(); + type = std::make_shared(); } type_indexes.erase(TypeIndex::Map); @@ -856,7 +799,7 @@ namespace /// Empty array has type Array(Nothing) if (nested_types.empty()) - return getEmptyArrayType(); + return std::make_shared(std::make_shared()); if (checkIfTypesAreEqual(nested_types)) return std::make_shared(std::move(nested_types.back())); @@ -969,13 +912,13 @@ namespace /// NOTE: it may break parsing of tryReadFloat() != tryReadIntText() + parsing of '.'/'e' /// But, for now it is true if (tryReadFloat(tmp_float, buf, settings, has_fractional) && has_fractional) - return getFloat64Type(); + return std::make_shared(); Int64 tmp_int; buf.position() = number_start; if (tryReadIntText(tmp_int, buf)) { - auto type = getInt64Type(); + auto type = std::make_shared(); if (json_info && tmp_int < 0) json_info->negative_integers.insert(type.get()); return type; @@ -985,7 +928,7 @@ namespace UInt64 tmp_uint; buf.position() = number_start; if (tryReadIntText(tmp_uint, buf)) - return getUInt64Type(); + return std::make_shared(); return nullptr; } @@ -997,13 +940,13 @@ namespace PeekableReadBufferCheckpoint checkpoint(peekable_buf); if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional) && has_fractional) - return getFloat64Type(); + return std::make_shared(); peekable_buf.rollbackToCheckpoint(/* drop= */ false); Int64 tmp_int; if (tryReadIntText(tmp_int, peekable_buf)) { - auto type = getInt64Type(); + auto type = std::make_shared(); if (json_info && tmp_int < 0) json_info->negative_integers.insert(type.get()); return type; @@ -1013,11 +956,11 @@ namespace /// In case of Int64 overflow we can try to infer UInt64. UInt64 tmp_uint; if (tryReadIntText(tmp_uint, peekable_buf)) - return getUInt64Type(); + return std::make_shared(); } else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { - return getFloat64Type(); + return std::make_shared(); } /// This is not a number. @@ -1034,7 +977,7 @@ namespace Int64 tmp_int; if (tryReadIntText(tmp_int, buf) && buf.eof()) { - auto type = getInt64Type(); + auto type = std::make_shared(); if (json_inference_info && tmp_int < 0) json_inference_info->negative_integers.insert(type.get()); return type; @@ -1046,7 +989,7 @@ namespace /// In case of Int64 overflow, try to infer UInt64 UInt64 tmp_uint; if (tryReadIntText(tmp_uint, buf) && buf.eof()) - return getUInt64Type(); + return std::make_shared(); } /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. @@ -1055,7 +998,7 @@ namespace Float64 tmp; bool has_fractional; if (tryReadFloat(tmp, buf, settings, has_fractional) && buf.eof()) - return getFloat64Type(); + return std::make_shared(); return nullptr; } @@ -1079,7 +1022,7 @@ namespace if constexpr (is_json) { if (json_info->is_object_key) - return getStringType(); + return std::make_shared(); } if (auto type = tryInferDateOrDateTimeFromString(field, settings)) @@ -1097,7 +1040,7 @@ namespace } } - return getStringType(); + return std::make_shared(); } bool tryReadJSONObject(ReadBuffer & buf, const FormatSettings & settings, DataTypeJSONPaths::Paths & paths, const std::vector & path, JSONInferenceInfo * json_info, size_t depth) @@ -1254,7 +1197,7 @@ namespace return std::make_shared("json", true); if (settings.json.read_objects_as_strings) - return getStringType(); + return std::make_shared(); transformInferredTypesIfNeededImpl(value_types, settings, json_info); if (!checkIfTypesAreEqual(value_types)) @@ -1320,15 +1263,15 @@ namespace /// Bool if (checkStringCaseInsensitive("true", buf) || checkStringCaseInsensitive("false", buf)) - return getBoolType(); + return DataTypeFactory::instance().get("Bool"); /// Null or NaN if (checkCharCaseInsensitive('n', buf)) { if (checkStringCaseInsensitive("ull", buf)) - return getNullType(); + return std::make_shared(std::make_shared()); else if (checkStringCaseInsensitive("an", buf)) - return getFloat64Type(); + return std::make_shared(); } /// Number @@ -1385,7 +1328,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F if (!remain_nothing_types && isNothing(data_type) && settings.json.infer_incomplete_types_as_strings) { - data_type = getStringType(); + data_type = std::make_shared(); return; } @@ -1402,7 +1345,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F /// If all objects were empty, use type String, so these JSON objects will be read as Strings. if (json_paths->empty() && settings.json.infer_incomplete_types_as_strings) { - data_type = getStringType(); + data_type = std::make_shared(); return; } @@ -1424,7 +1367,7 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F auto key_type = map_type->getKeyType(); /// If all inferred Maps are empty, use type String, so these JSON objects will be read as Strings. if (isNothing(key_type) && settings.json.infer_incomplete_types_as_strings) - key_type = getStringType(); + key_type = std::make_shared(); auto value_type = map_type->getValueType(); @@ -1501,10 +1444,10 @@ DataTypePtr tryInferJSONNumberFromString(std::string_view field, const FormatSet DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const FormatSettings & settings) { if (settings.try_infer_dates && tryInferDate(field)) - return getDateType(); + return std::make_shared(); if (settings.try_infer_datetimes && tryInferDateTime(field, settings)) - return getDateTime64Type(); + return std::make_shared(9); return nullptr; } diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index 31699ad1c9c..4f29846319d 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -165,6 +165,7 @@ private: std::vector sorted_dynamic_and_typed_paths; const auto & typed_path_columns = column_object.getTypedPaths(); const auto & dynamic_path_columns = column_object.getDynamicPaths(); + sorted_dynamic_and_typed_paths.reserve(typed_path_columns.size() + dynamic_path_columns.size()); for (const auto & [path, _] : typed_path_columns) sorted_dynamic_and_typed_paths.push_back(path); for (const auto & [path, _] : dynamic_path_columns) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index d8fff4e6026..b39ce25ab57 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -68,6 +68,7 @@ WITH map( 'Map', 'JSON', 'Tuple', 'JSON', 'Object', 'JSON', + 'JSON', 'JSON', 'String', '{}', 'FixedString', '{}') AS native_to_mysql_mapping, )", diff --git a/src/Parsers/ASTObjectTypeArgument.cpp b/src/Parsers/ASTObjectTypeArgument.cpp index aa5d23d7881..975f0389505 100644 --- a/src/Parsers/ASTObjectTypeArgument.cpp +++ b/src/Parsers/ASTObjectTypeArgument.cpp @@ -35,27 +35,27 @@ ASTPtr ASTObjectTypeArgument::clone() const return res; } -void ASTObjectTypeArgument::formatImpl(const FormatSettings & parameters, FormatState & state, FormatStateStacked frame) const +void ASTObjectTypeArgument::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { if (path_with_type) { - path_with_type->formatImpl(parameters, state, frame); + path_with_type->formatImpl(settings, state, frame); } else if (parameter) { - parameter->formatImpl(parameters, state, frame); + parameter->formatImpl(settings, state, frame); } else if (skip_path) { - std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); - parameters.ostr << indent_str << "SKIP" << ' '; - skip_path->formatImpl(parameters, state, frame); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << indent_str << "SKIP" << ' '; + skip_path->formatImpl(settings, state, frame); } else if (skip_path_regexp) { - std::string indent_str = parameters.one_line ? "" : std::string(4 * frame.indent, ' '); - parameters.ostr << indent_str << "SKIP REGEXP" << ' '; - skip_path_regexp->formatImpl(parameters, state, frame); + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << indent_str << "SKIP REGEXP" << ' '; + skip_path_regexp->formatImpl(settings, state, frame); } } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index c017d7db786..2c876fade74 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -213,6 +213,9 @@ void MergeTreeReaderWide::addStreams( ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); /** If data file is missing then we will not try to open it. @@ -348,6 +351,9 @@ void MergeTreeReaderWide::prefetchForColumn( deserializePrefix(serialization, name_and_type, current_task_last_mark, cache, deserialize_states_cache); auto callback = [&](const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); if (stream_name && !prefetched_streams.contains(*stream_name)) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 774fd95ebc6..dcdcb7f5800 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -211,6 +211,9 @@ static IMergeTreeDataPart::Checksums checkDataPart( { get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { + if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + return; + auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column, substream_path, ".bin", data_part_storage); if (!stream_name) diff --git a/tests/queries/0_stateless/01825_new_type_json_10.reference b/tests/queries/0_stateless/01825_new_type_json_10.reference index 0dffa3c46f9..d70c8210914 100644 --- a/tests/queries/0_stateless/01825_new_type_json_10.reference +++ b/tests/queries/0_stateless/01825_new_type_json_10.reference @@ -1,9 +1,8 @@ ('a.b','Int64') ('a.c','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') -('a.c','Array(Nullable(String))') +('d','Int64') ('e','Array(Nullable(Int64))') ('f','Int64') -('d','Int64') {"o":{"a":{"b":"1","c":[{"d":"10","e":["31"]},{"d":"20","e":["63","127"]}]}}} {"o":{"a":{"b":"2","c":[]}}} {"o":{"a":{"b":"3","c":[{"e":["32"],"f":"20"},{"e":["64","128"],"f":"30"}]}}} diff --git a/tests/queries/0_stateless/01825_new_type_json_10.sql b/tests/queries/0_stateless/01825_new_type_json_10.sql index 0e599b0ac31..a313adb4757 100644 --- a/tests/queries/0_stateless/01825_new_type_json_10.sql +++ b/tests/queries/0_stateless/01825_new_type_json_10.sql @@ -8,8 +8,8 @@ CREATE TABLE t_json_10 (o JSON) ENGINE = Memory; INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 1, "c": [{"d": 10, "e": [31]}, {"d": 20, "e": [63, 127]}]}} {"a": {"b": 2, "c": []}} INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 3, "c": [{"f": 20, "e": [32]}, {"f": 30, "e": [64, 128]}]}} {"a": {"b": 4, "c": []}} -SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(o)) FROM t_json_10; -SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(o.a.c.:`Array(JSON)`))) FROM t_json_10; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(o)) as path FROM t_json_10 order by path; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(o.a.c.:`Array(JSON)`))) as path FROM t_json_10 order by path; SELECT o FROM t_json_10 ORDER BY o.a.b FORMAT JSONEachRow; SELECT o.a.b, o.a.c.:`Array(JSON)`.d, o.a.c.:`Array(JSON)`.e, o.a.c.:`Array(JSON)`.f FROM t_json_10 ORDER BY o.a.b; diff --git a/tests/queries/0_stateless/01825_new_type_json_11.sh b/tests/queries/0_stateless/01825_new_type_json_11.sh index 2eb6c1768fd..f448b7433ab 100755 --- a/tests/queries/0_stateless/01825_new_type_json_11.sh +++ b/tests/queries/0_stateless/01825_new_type_json_11.sh @@ -53,10 +53,10 @@ cat <&1 | grep -o -m1 "Cannot parse object" $CLICKHOUSE_CLIENT -q "SELECT count() FROM t_json_async_insert" diff --git a/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh b/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh index 99f243833f5..e0648f4df6e 100755 --- a/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh +++ b/tests/queries/0_stateless/02482_json_nested_arrays_with_same_keys.sh @@ -21,7 +21,7 @@ echo ' } }' > 02482_object_data.jsonl -$CLICKHOUSE_LOCAL --allow_experimental_object_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj Object('json')')" +$CLICKHOUSE_LOCAL --allow_experimental_object_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj Object(''json'')')" rm 02482_object_data.jsonl diff --git a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference index de0f151db7d..cb905d63ca5 100644 --- a/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference +++ b/tests/queries/0_stateless/02775_show_columns_called_from_clickhouse.reference @@ -44,7 +44,7 @@ nested.col1 Array(String) NO \N nested.col2 Array(UInt32) NO \N nfs Nullable(FixedString(3)) YES \N ns Nullable(String) YES \N -o Object(\'json\') NO \N +o JSON NO \N p Point NO \N pg Polygon NO \N r Ring NO \N From f7b7a22c18a39994ae4e212f8069134b7bd595a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 14:28:21 +0000 Subject: [PATCH 0344/1722] Update docs --- docs/en/sql-reference/data-types/json.md | 1 + docs/en/sql-reference/data-types/object-json.md | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 1001ad63999..494bfba3173 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -2,6 +2,7 @@ slug: /en/sql-reference/data-types/json sidebar_position: 63 sidebar_label: JSON +keywords: [json, data type] --- # JSON diff --git a/docs/en/sql-reference/data-types/object-json.md b/docs/en/sql-reference/data-types/object-json.md index c6fa6f0e882..9609f30aac1 100644 --- a/docs/en/sql-reference/data-types/object-json.md +++ b/docs/en/sql-reference/data-types/object-json.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/data-types/object-data-type +slug: /en/sql-reference/data-types/object-json sidebar_position: 26 sidebar_label: Object Data Type keywords: [object, data type] From d13bbdf5e1e0f5763045d301dbfc9fddd6013f14 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 15:49:46 +0000 Subject: [PATCH 0345/1722] Fix style --- src/Formats/JSONExtractTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 640ab1b0acd..e8a78a40749 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -314,7 +314,7 @@ public: } NumberType value; - if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || this->is_bool_type, insert_settings.allow_type_conversion, error)) + if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || this->is_bool_type, insert_settings.allow_type_conversion, error)) { if (error.empty()) error = fmt::format("cannot read {} value from JSON element: {}", TypeName, jsonElementToString(element, format_settings)); From a8e34f07a8090892ea98c58937bfb4a4ed4a6661 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 16:26:40 +0000 Subject: [PATCH 0346/1722] Fix debug build --- src/Columns/ColumnObjectDeprecated.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnObjectDeprecated.cpp b/src/Columns/ColumnObjectDeprecated.cpp index 6d0bfa0e05e..d3f23dc6b57 100644 --- a/src/Columns/ColumnObjectDeprecated.cpp +++ b/src/Columns/ColumnObjectDeprecated.cpp @@ -763,7 +763,7 @@ void ColumnObjectDeprecated::get(size_t n, Field & res) const } } -#if !defined(ABORT_ON_LOGICAL_ERROR) +#if !defined(DEBUG_OR_SANITIZER_BUILD) void ColumnObjectDeprecated::insertFrom(const IColumn & src, size_t n) #else void ColumnObjectDeprecated::doInsertFrom(const IColumn & src, size_t n) @@ -772,7 +772,7 @@ void ColumnObjectDeprecated::doInsertFrom(const IColumn & src, size_t n) insert(src[n]); } -#if !defined(ABORT_ON_LOGICAL_ERROR) +#if !defined(DEBUG_OR_SANITIZER_BUILD) void ColumnObjectDeprecated::insertRangeFrom(const IColumn & src, size_t start, size_t length) #else void ColumnObjectDeprecated::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) From 79b5fdc90a21094fd9978ab1174acd733df3cf0c Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 17:09:31 +0000 Subject: [PATCH 0347/1722] More tests --- .../01825_new_type_json_7.reference | 5 +++ .../0_stateless/01825_new_type_json_7.sh | 30 +++++++++++++++ .../01825_new_type_json_8.reference | 12 ++++++ .../0_stateless/01825_new_type_json_8.sh | 38 +++++++++++++++++++ 4 files changed, 85 insertions(+) create mode 100644 tests/queries/0_stateless/01825_new_type_json_7.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_7.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_8.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_8.sh diff --git a/tests/queries/0_stateless/01825_new_type_json_7.reference b/tests/queries/0_stateless/01825_new_type_json_7.reference new file mode 100644 index 00000000000..8e26e9b2c44 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_7.reference @@ -0,0 +1,5 @@ +('categories','Array(Nullable(String))') +('key','String') +v1 \N +v2 ['foo','bar'] +v3 \N diff --git a/tests/queries/0_stateless/01825_new_type_json_7.sh b/tests/queries/0_stateless/01825_new_type_json_7.sh new file mode 100755 index 00000000000..36483175df6 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_7.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_7;" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_7 (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_json_type 1 + +cat < Date: Mon, 22 Jul 2024 19:26:44 +0000 Subject: [PATCH 0348/1722] Fix failing tests, add even more tests --- docs/en/sql-reference/data-types/json.md | 553 +---- docs/en/sql-reference/data-types/newjson.md | 516 ++++ .../sql-reference/data-types/object-json.md | 83 - src/Columns/ColumnObject.cpp | 14 + src/Columns/ColumnObject.h | 6 +- src/Interpreters/convertFieldToType.cpp | 7 + .../01825_new_type_json_12.reference | 7 +- .../01825_new_type_json_3.reference.j2 | 4 +- .../01825_new_type_json_6.reference | 2 +- .../01825_new_type_json_9.reference | 3 + .../0_stateless/01825_new_type_json_9.sql | 16 + .../01825_new_type_json_bools.reference | 1 + .../0_stateless/01825_new_type_json_bools.sql | 10 + .../01825_new_type_json_btc.reference | 2 +- .../01825_new_type_json_distributed.reference | 4 + .../01825_new_type_json_distributed.sql | 18 + .../01825_new_type_json_in_array.reference | 28 + .../01825_new_type_json_in_array.sql | 34 + ...825_new_type_json_in_other_types.reference | 17 + .../01825_new_type_json_in_other_types.sh | 90 + ...1825_new_type_json_insert_select.reference | 27 + .../01825_new_type_json_insert_select.sql | 67 + ...1825_new_type_json_missed_values.reference | 4 + .../01825_new_type_json_missed_values.sql | 19 + ...825_new_type_json_multiple_files.reference | 22 + .../01825_new_type_json_multiple_files.sh | 43 + .../01825_new_type_json_mutations.reference | 7 + .../01825_new_type_json_mutations.sql | 21 + .../01825_new_type_json_order_by.reference | 6 + .../01825_new_type_json_order_by.sql | 6 + .../01825_new_type_json_partitions.reference | 2 + .../01825_new_type_json_partitions.sql | 14 + .../0_stateless/01825_type_json_10.reference | 9 + .../queries/0_stateless/01825_type_json_5.sql | 4 +- ...02421_new_type_json_async_insert.reference | 5 + .../02421_new_type_json_async_insert.sh | 21 + .../02421_new_type_json_empty_parts.reference | 16 + .../02421_new_type_json_empty_parts.sh | 48 + ...son_nested_arrays_with_same_keys.reference | 1 + ...2_new_json_nested_arrays_with_same_keys.sh | 27 + ...3_new_type_json_attach_partition.reference | 2 + .../02553_new_type_json_attach_partition.sql | 14 + .../03207_json_read_subcolumns_1.reference | 2080 ----------------- ..._subcolumns_1_compact_merge_tree.reference | 831 +++++++ ...n_read_subcolumns_1_compact_merge_tree.sh} | 18 - ...07_json_read_subcolumns_1_memory.reference | 413 ++++ .../03207_json_read_subcolumns_1_memory.sql | 87 + ...ead_subcolumns_1_wide_merge_tree.reference | 831 +++++++ ..._json_read_subcolumns_1_wide_merge_tree.sh | 107 + ...on_read_subcolumns_2_compact_merge_tree.sh | 2 +- .../03207_json_read_subcolumns_2_memory.sh | 2 +- ..._json_read_subcolumns_2_wide_merge_tree.sh | 2 +- ...on_read_subcolumns_2_compact_merge_tree.sh | 2 +- ..._array_of_json_read_subcolumns_2_memory.sh | 2 +- ..._json_read_subcolumns_2_wide_merge_tree.sh | 2 +- 55 files changed, 3484 insertions(+), 2695 deletions(-) create mode 100644 docs/en/sql-reference/data-types/newjson.md delete mode 100644 docs/en/sql-reference/data-types/object-json.md create mode 100644 tests/queries/0_stateless/01825_new_type_json_9.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_9.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_bools.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_bools.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_distributed.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_distributed.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_in_array.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_in_array.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_in_other_types.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_in_other_types.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_insert_select.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_insert_select.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_missed_values.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_missed_values.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_multiple_files.reference create mode 100755 tests/queries/0_stateless/01825_new_type_json_multiple_files.sh create mode 100644 tests/queries/0_stateless/01825_new_type_json_mutations.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_mutations.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_order_by.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_order_by.sql create mode 100644 tests/queries/0_stateless/01825_new_type_json_partitions.reference create mode 100644 tests/queries/0_stateless/01825_new_type_json_partitions.sql create mode 100644 tests/queries/0_stateless/02421_new_type_json_async_insert.reference create mode 100755 tests/queries/0_stateless/02421_new_type_json_async_insert.sh create mode 100644 tests/queries/0_stateless/02421_new_type_json_empty_parts.reference create mode 100755 tests/queries/0_stateless/02421_new_type_json_empty_parts.sh create mode 100644 tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.reference create mode 100755 tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.sh create mode 100644 tests/queries/0_stateless/02553_new_type_json_attach_partition.reference create mode 100644 tests/queries/0_stateless/02553_new_type_json_attach_partition.sql delete mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1.reference create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference rename tests/queries/0_stateless/{03207_json_read_subcolumns_1.sh => 03207_json_read_subcolumns_1_compact_merge_tree.sh} (95%) create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference create mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference create mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 494bfba3173..c6fa6f0e882 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -1,516 +1,83 @@ --- -slug: /en/sql-reference/data-types/json -sidebar_position: 63 -sidebar_label: JSON -keywords: [json, data type] +slug: /en/sql-reference/data-types/object-data-type +sidebar_position: 26 +sidebar_label: Object Data Type +keywords: [object, data type] --- -# JSON +# Object Data Type (deprecated) + +**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864). + +
Stores JavaScript Object Notation (JSON) documents in a single column. -:::note -This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. -If you want to use JSON type, set `allow_experimental_json_type = 1`. -::: +`JSON` can be used as an alias to `Object('json')` when setting [use_json_alias_for_old_object_type](../../operations/settings/settings.md#usejsonaliasforoldobjecttype) is enabled. -To declare a column of `JSON` type, use the following syntax: +## Example -``` sql - JSON(max_dynamic_paths=N, max_dynamic_types=M, some.path TypeName, SKIP path.to.skip, SKIP REGEXP 'paths_regexp') -``` -Where: -- `max_dynamic_paths` is an optional parameter indicating how many paths can be stored separately as subcolumns across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all other paths will be stored together in a single structure. Default value of `max_dynamic_paths` is `1024`. -- `max_dynamic_types` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a single path column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_dynamic_types` is `32`. -- `some.path TypeName` is an optional type hint for particular path in the JSON. Such paths will be always stored as subcolumns with specified type. -- `SKIP path.to.skip` is an optional hint for particular path that should be skipped during JSON parsing. Such paths will never be stored in the JSON column. If specified path is a nested JSON object, the whole nested object will be skipped. -- `SKIP REGEXP 'path_regexp'` is an optional hint with a regular expression that is used to skip paths during JSON parsing. All paths that match this regular expression will never be stored in the JSON column. +**Example 1** -## Creating JSON - -Using `JSON` type in table column definition: +Creating a table with a `JSON` column and inserting data into it: ```sql -CREATE TABLE test (json JSON) ENGINE = Memory; -INSERT INTO test VALUES ('{"a" : {"b" : 42}, "c" : [1, 2, 3]}'), ('{"f" : "Hello, World!"}'), ('{"a" : {"b" : 43, "e" : 10}, "c" : [4, 5, 6]}'); -SELECT json FROM test; +CREATE TABLE json +( + o JSON +) +ENGINE = Memory +``` + +```sql +INSERT INTO json VALUES ('{"a": 1, "b": { "c": 2, "d": [1, 2, 3] }}') +``` + +```sql +SELECT o.a, o.b.c, o.b.d[3] FROM json ``` ```text -┌─json────────────────────────────────────────┠-│ {"a":{"b":"42"},"c":["1","2","3"]} │ -│ {"f":"Hello, World!"} │ -│ {"a":{"b":"43","e":"10"},"c":["4","5","6"]} │ -└─────────────────────────────────────────────┘ +┌─o.a─┬─o.b.c─┬─arrayElement(o.b.d, 3)─┠+│ 1 │ 2 │ 3 │ +└─────┴───────┴────────────────────────┘ +``` + +**Example 2** + +To be able to create an ordered `MergeTree` family table, the sorting key has to be extracted into its column. For example, to insert a file of compressed HTTP access logs in JSON format: + +```sql +CREATE TABLE logs +( + timestamp DateTime, + message JSON +) +ENGINE = MergeTree +ORDER BY timestamp ``` ```sql -CREATE TABLE test (json JSON(a.b UInt32, SKIP a.e)) ENGINE = Memory; -INSERT INTO test VALUES ('{"a" : {"b" : 42}, "c" : [1, 2, 3]}'), ('{"f" : "Hello, World!"}'), ('{"a" : {"b" : 43, "e" : 10}, "c" : [4, 5, 6]}'); -SELECT json FROM test; +INSERT INTO logs +SELECT parseDateTimeBestEffort(JSONExtractString(json, 'timestamp')), json +FROM file('access.json.gz', JSONAsString) +``` + +## Displaying JSON columns + +When displaying a `JSON` column, ClickHouse only shows the field values by default (because internally, it is represented as a tuple). You can also display the field names by setting `output_format_json_named_tuples_as_objects = 1`: + +```sql +SET output_format_json_named_tuples_as_objects = 1 + +SELECT * FROM json FORMAT JSONEachRow ``` ```text -┌─json──────────────────────────────┠-│ {"a":{"b":42},"c":[1,2,3]} │ -│ {"a":{"b":0},"f":"Hello, World!"} │ -│ {"a":{"b":43},"c":[4,5,6]} │ -└───────────────────────────────────┘ +{"o":{"a":1,"b":{"c":2,"d":[1,2,3]}}} ``` -Using CAST from 'String': +## Related Content -```sql -SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json; -``` - -```text -┌─json───────────────────────────────────────────┠-│ {"a":{"b":42},"c":[1,2,3],"d":"Hello, World!"} │ -└────────────────────────────────────────────────┘ -``` - -CAST from named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. - -## Reading JSON paths as subcolumns - -JSON type supports reading every path as a separate subcolumn. If type of the requested path was not specified in the JSON type declaration, the subcolumn of the path will always have type [Dynamic](/docs/en/sql-reference/data-types/dynamic.md). - -For example: - -```sql -CREATE TABLE test (json JSON(a.b UInt32, SKIP a.e)) ENGINE = Memory; -INSERT INTO test VALUES ('{"a" : {"b" : 42, "g" : 42.42}, "c" : [1, 2, 3], "d" : "2020-01-01"}'), ('{"f" : "Hello, World!", "d" : "2020-01-02"}'), ('{"a" : {"b" : 43, "e" : 10, "g" : 43.43}, "c" : [4, 5, 6]}'); -SELECT json FROM test; -``` - -```text -┌─json──────────────────────────────────────────────────┠-│ {"a":{"b":42,"g":42.42},"c":[1,2,3],"d":"2020-01-01"} │ -│ {"a":{"b":0},"d":"2020-01-02","f":"Hello, World!"} │ -│ {"a":{"b":43,"g":43.43},"c":[4,5,6]} │ -└───────────────────────────────────────────────────────┘ -``` - -```sql -SELECT json.a.b, json.a.g, json.c, json.d FROM test; -``` - -```text -┌─json.a.b─┬─json.a.g─┬─json.c──┬─json.d─────┠-│ 42 │ 42.42 │ [1,2,3] │ 2020-01-01 │ -│ 0 │ á´ºáµá´¸á´¸ │ á´ºáµá´¸á´¸ │ 2020-01-02 │ -│ 43 │ 43.43 │ [4,5,6] │ á´ºáµá´¸á´¸ │ -└──────────┴──────────┴─────────┴────────────┘ -``` - -If the requested path wasn't found in the data, it will be filled with `NULL` values: - -```sql -SELECT json.non.existing.path FROM test; -``` - -```text -┌─json.non.existing.path─┠-│ á´ºáµá´¸á´¸ │ -│ á´ºáµá´¸á´¸ │ -│ á´ºáµá´¸á´¸ │ -└────────────────────────┘ -``` - -Let's check the data types of returned subcolumns: -```sql -SELECT toTypeName(json.a.b), toTypeName(json.a.g), toTypeName(json.c), toTypeName(json.d) FROM test; -``` - -```text -┌─toTypeName(json.a.b)─┬─toTypeName(json.a.g)─┬─toTypeName(json.c)─┬─toTypeName(json.d)─┠-│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ -│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ -│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ -└──────────────────────┴──────────────────────┴────────────────────┴────────────────────┘ -``` - -As we can see, for `a.b` the type is `UInt32` as we specified in the JSON type declaration, and for all other subcolumns the type is `Dynamic`. - -It is also possible to read subcolumns of a `Dynamic` type using special syntax `json.some.path.:TypeName`: - -```sql -select json.a.g.:Float64, dynamicType(json.a.g), json.d.:Date, dynamicType(json.d) FROM test; -``` - -```text -┌─json.a.g.:`Float64`─┬─dynamicType(json.a.g)─┬─json.d.:`Date`─┬─dynamicType(json.d)─┠-│ 42.42 │ Float64 │ 2020-01-01 │ Date │ -│ á´ºáµá´¸á´¸ │ None │ 2020-01-02 │ Date │ -│ 43.43 │ Float64 │ á´ºáµá´¸á´¸ │ None │ -└─────────────────────┴───────────────────────┴────────────────┴─────────────────────┘ -``` - -`Dynamic` subcolumns can be casted to any data type. In this case the exception will be thrown if internal type inside `Dynamic` cannot be casted to the requested type: - -```sql -select json.a.g::UInt64 as uint FROM test; -``` - -```text -┌─uint─┠-│ 42 │ -│ 0 │ -│ 43 │ -└──────┘ -``` - -```sql -select json.a.g::UUID as float FROM test; -``` - -```text -Received exception: -Code: 48. DB::Exception: Conversion between numeric types and UUID is not supported. Probably the passed UUID is unquoted: while executing 'FUNCTION CAST(__table1.json.a.g :: 2, 'UUID'_String :: 1) -> CAST(__table1.json.a.g, 'UUID'_String) UUID : 0'. (NOT_IMPLEMENTED) -``` - -## Reading JSON sub-objects as subcolumns - -JSON type supports reading nested objects as subcolumns with type `JSON` using special syntax `json.^some.path`: - -```sql -CREATE TABLE test (json JSON) ENGINE = Memory; -INSERT INTO test VALUES ('{"a" : {"b" : {"c" : 42, "g" : 42.42}}, "c" : [1, 2, 3], "d" : {"e" : {"f" : {"g" : "Hello, World", "h" : [1, 2, 3]}}}}'), ('{"f" : "Hello, World!", "d" : {"e" : {"f" : {"h" : [4, 5, 6]}}}}'), ('{"a" : {"b" : {"c" : 43, "e" : 10, "g" : 43.43}}, "c" : [4, 5, 6]}'); -SELECT json FROM test; -``` - -```text -┌─json────────────────────────────────────────────────────────────────────────────────────────┠-│ {"a":{"b":{"c":42,"g":42.42}},"c":[1,2,3],"d":{"e":{"f":{"g":"Hello, World","h":[1,2,3]}}}} │ -│ {"d":{"e":{"f":{"h":[4,5,6]}}},"f":"Hello, World!"} │ -│ {"a":{"b":{"c":43,"e":10,"g":43.43}},"c":[4,5,6]} │ -└─────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -```sql -SELECT json.^a.b, json.^d.e.f FROM test; -``` - -```text -┌─json.^`a`.b───────────────┬─json.^`d`.e.f────────────────────┠-│ {"c":42,"g":42.42} │ {"g":"Hello, World","h":[1,2,3]} │ -│ {} │ {"h":[4,5,6]} │ -│ {"c":43,"e":10,"g":43.43} │ {} │ -└───────────────────────────┴──────────────────────────────────┘ -``` - -:::note -Reading sub-objects as subcolumns may be inefficient, as this may require almost full scan of the JSON data. -::: - -## Types inference for paths - -During JSON parsing ClickHouse tries to detect the most appropriate data type for each JSON path. It works similar to [automatic schema inference from input data](/docs/en/interfaces/schema-inference.md) and controlled by the same settings: - -- [input_format_try_infer_integers](/docs/en/interfaces/schema-inference.md#inputformattryinferintegers) -- [input_format_try_infer_dates](/docs/en/interfaces/schema-inference.md#inputformattryinferdates) -- [input_format_try_infer_datetimes](/docs/en/interfaces/schema-inference.md#inputformattryinferdatetimes) -- [schema_inference_make_columns_nullable](/docs/en/interfaces/schema-inference.md#schemainferencemakecolumnsnullable) -- [input_format_json_try_infer_numbers_from_strings](/docs/en/interfaces/schema-inference.md#inputformatjsontryinfernumbersfromstrings) -- [input_format_json_infer_incomplete_types_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsoninferincompletetypesasstrings) -- [input_format_json_read_numbers_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadnumbersasstrings) -- [input_format_json_read_bools_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadboolsasstrings) -- [input_format_json_read_bools_as_numbers](/docs/en/interfaces/schema-inference.md#inputformatjsonreadboolsasnumbers) -- [input_format_json_read_arrays_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadarraysasstrings) - -Let's see some examples: - -```sql -SELECT JSONAllPathsWithTypes('{"a" : "2020-01-01", "b" : "2020-01-01 10:00:00"}'::JSON) AS paths_with_types settings input_format_try_infer_dates=1, input_format_try_infer_datetimes=1; -``` - -```text -┌─paths_with_types─────────────────┠-│ {'a':'Date','b':'DateTime64(9)'} │ -└──────────────────────────────────┘ -``` - -```sql -SELECT JSONAllPathsWithTypes('{"a" : "2020-01-01", "b" : "2020-01-01 10:00:00"}'::JSON) AS paths_with_types settings input_format_try_infer_dates=0, input_format_try_infer_datetimes=0; -``` - -```text -┌─paths_with_types────────────┠-│ {'a':'String','b':'String'} │ -└─────────────────────────────┘ -``` - -```sql -SELECT JSONAllPathsWithTypes('{"a" : [1, 2, 3]}'::JSON) AS paths_with_types settings schema_inference_make_columns_nullable=1; -``` - -```text -┌─paths_with_types───────────────┠-│ {'a':'Array(Nullable(Int64))'} │ -└────────────────────────────────┘ -``` - -```sql -SELECT JSONAllPathsWithTypes('{"a" : [1, 2, 3]}'::JSON) AS paths_with_types settings schema_inference_make_columns_nullable=0; -``` - -```text -┌─paths_with_types─────┠-│ {'a':'Array(Int64)'} │ -└──────────────────────┘ -``` - -## Handling arrays of JSON objects - -JSON paths that contains an array of objects are parsed as type `Array(JSON)` and inserted into `Dynamic` column for this path. To read an array of objects you can extract it from `Dynamic` column as a subcolumn: - -```sql -CREATE TABLE test (json JSON) ENGINE = Memory; -INSERT INTO test VALUES -('{"a" : {"b" : [{"c" : 42, "d" : "Hello", "f" : [[{"g" : 42.42}]], "k" : {"j" : 1000}}, {"c" : 43}, {"e" : [1, 2, 3], "d" : "My", "f" : [[{"g" : 43.43, "h" : "2020-01-01"}]], "k" : {"j" : 2000}}]}}'), -('{"a" : {"b" : [1, 2, 3]}}'), -('{"a" : {"b" : [{"c" : 44, "f" : [[{"h" : "2020-01-02"}]]}, {"e" : [4, 5, 6], "d" : "World", "f" : [[{"g" : 44.44}]], "k" : {"j" : 3000}}]}}'); -SELECT json FROM test; -``` - -```text3 -┌─json────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┠-│ {"a":{"b":[{"c":"42","d":"Hello","f":[[{"g":42.42}]],"k":{"j":"1000"}},{"c":"43"},{"d":"My","e":["1","2","3"],"f":[[{"g":43.43,"h":"2020-01-01"}]],"k":{"j":"2000"}}]}} │ -│ {"a":{"b":["1","2","3"]}} │ -│ {"a":{"b":[{"c":"44","f":[[{"h":"2020-01-02"}]]},{"d":"World","e":["4","5","6"],"f":[[{"g":44.44}]],"k":{"j":"3000"}}]}} │ -└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -```sql -SELECT json.a.b, dynamicType(json.a.b) FROM test; -``` - -```text -┌─json.a.b──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─dynamicType(json.a.b)────────────────────────────────────┠-│ ['{"c":"42","d":"Hello","f":[[{"g":42.42}]],"k":{"j":"1000"}}','{"c":"43"}','{"d":"My","e":["1","2","3"],"f":[[{"g":43.43,"h":"2020-01-01"}]],"k":{"j":"2000"}}'] │ Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)) │ -│ [1,2,3] │ Array(Nullable(Int64)) │ -│ ['{"c":"44","f":[[{"h":"2020-01-02"}]]}','{"d":"World","e":["4","5","6"],"f":[[{"g":44.44}]],"k":{"j":"3000"}}'] │ Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)) │ -└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────────────────────────────────────────────────────────┘ -``` - -As you can notice, the `max_dynamic_types/max_dynamic_paths` parameters of the nested `JSON` type were reduced compared to the default values. It's needed to avoid number of subcolumns to grow uncontrolled on nested arrays of JSON objects. - -Let's try to read subcolumns from this nested `JSON` column: - -```sql -SELECT json.a.b.:`Array(JSON)`.c, json.a.b.:`Array(JSON)`.f, json.a.b.:`Array(JSON)`.d FROM test; -``` - -```text -┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f───────────────────────────────────┬─json.a.b.:`Array(JSON)`.d─┠-│ [42,43,NULL] │ [[['{"g":42.42}']],NULL,[['{"g":43.43,"h":"2020-01-01"}']]] │ ['Hello',NULL,'My'] │ -│ [] │ [] │ [] │ -│ [44,NULL] │ [[['{"h":"2020-01-02"}']],[['{"g":44.44}']]] │ [NULL,'World'] │ -└───────────────────────────┴─────────────────────────────────────────────────────────────┴───────────────────────────┘ -``` - -We can avoid writing `Array(JSON)` subcolumn name using special syntax: - -```sql -SELECT json.a.b[].c, json.a.b[].f, json.a.b[].d FROM test; -``` - -```text -┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f───────────────────────────────────┬─json.a.b.:`Array(JSON)`.d─┠-│ [42,43,NULL] │ [[['{"g":42.42}']],NULL,[['{"g":43.43,"h":"2020-01-01"}']]] │ ['Hello',NULL,'My'] │ -│ [] │ [] │ [] │ -│ [44,NULL] │ [[['{"h":"2020-01-02"}']],[['{"g":44.44}']]] │ [NULL,'World'] │ -└───────────────────────────┴─────────────────────────────────────────────────────────────┴───────────────────────────┘ -``` - -The number of `[]` after path indicates the array level. `json.path[][]` will be transformed to `json.path.:Array(Array(JSON))` - -Let's check the paths and types inside our `Array(JSON)`: - -```sql -SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b[]))) FROM test; -``` - -```text -┌─arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b.:`Array(JSON)`)))──┠-│ ('c','Int64') │ -│ ('d','String') │ -│ ('f','Array(Array(JSON(max_dynamic_types=8, max_dynamic_paths=64)))') │ -│ ('k.j','Int64') │ -│ ('e','Array(Nullable(Int64))') │ -└───────────────────────────────────────────────────────────────────────┘ -``` - -Let's read subcolumns from `Array(JSON)` column: - -```sql -SELECT json.a.b[].c.:Int64, json.a.b[].f[][].g.:Float64, json.a.b[].f[][].h.:Date FROM test; -``` - -```text -┌─json.a.b.:`Array(JSON)`.c.:`Int64`─┬─json.a.b.:`Array(JSON)`.f.:`Array(Array(JSON))`.g.:`Float64`─┬─json.a.b.:`Array(JSON)`.f.:`Array(Array(JSON))`.h.:`Date`─┠-│ [42,43,NULL] │ [[[42.42]],[],[[43.43]]] │ [[[NULL]],[],[['2020-01-01']]] │ -│ [] │ [] │ [] │ -│ [44,NULL] │ [[[NULL]],[[44.44]]] │ [[['2020-01-02']],[[NULL]]] │ -└────────────────────────────────────┴──────────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ -``` - -We can also read sub-object subcolumns from nested `JSON` column: - -```sql -SELECT json.a.b[].^k FROM test -``` - -```text -┌─json.a.b.:`Array(JSON)`.^`k`─────────┠-│ ['{"j":"1000"}','{}','{"j":"2000"}'] │ -│ [] │ -│ ['{}','{"j":"3000"}'] │ -└──────────────────────────────────────┘ -``` - -## Reading JSON type from the data - -All text formats (JSONEachRow, TSV, CSV, CustomSeparated, Values, etc) supports reading `JSON` type. - -Examples: - -```sql -SELECT json FROM format(JSONEachRow, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP d.e, SKIP REGEXP \'b.*\')', ' -{"json" : {"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}}} -{"json" : {"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}}} -{"json" : {"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"}} -{"json" : {"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43}} -{"json" : {"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : {"c" : 11, "j" : [1, 2, 3]}, "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44}, "h" : "2020-02-02 10:00:00"}}} -') -``` - -```text -┌─json──────────────────────────────────────────────────────────┠-│ {"a":{"b":{"c":1}},"c":"42","d":{"i":["1","2","3"]}} │ -│ {"a":{"b":{"c":2}},"d":{"i":["4","5","6"]}} │ -│ {"a":{"b":{"c":3}},"e":"Hello, World!"} │ -│ {"a":{"b":{"c":4}},"c":"43"} │ -│ {"a":{"b":{"c":5}},"d":{"h":"2020-02-02 10:00:00.000000000"}} │ -└───────────────────────────────────────────────────────────────┘ -``` - -For text formats like CSV/TSV/etc `JSON` is parsed from a string containing JSON object - -```sql -SELECT json FROM format(TSV, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP REGEXP \'b.*\')', -'{"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}} -{"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}} -{"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"} -{"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43} -{"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : {"c" : 11, "j" : [1, 2, 3]}, "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44}, "h" : "2020-02-02 10:00:00"}}') -``` - -```text -┌─json──────────────────────────────────────────────────────────┠-│ {"a":{"b":{"c":1}},"c":"42","d":{"i":["1","2","3"]}} │ -│ {"a":{"b":{"c":2}},"d":{"i":["4","5","6"]}} │ -│ {"a":{"b":{"c":3}},"e":"Hello, World!"} │ -│ {"a":{"b":{"c":4}},"c":"43"} │ -│ {"a":{"b":{"c":5}},"d":{"h":"2020-02-02 10:00:00.000000000"}} │ -└───────────────────────────────────────────────────────────────┘ -``` - -## Reaching the limit of dynamic paths inside JSON - -`JSON` data type can store only limited number of paths as separate subcolumns inside. By default, this limit is 1024, but you can change it in type declaration using parameter `max_dynamic_paths`. -When the limit is reached, all new paths inserted to `JSON` column will be stored in a single shared data structure. It's still possible to read such paths as subcolumns, but it will require reading the whole -shared data structure to extract the values of this path. This limit is needed to avoid the enormous number of different subcolumns that can make the table unusable. - -Let's see what happens when the limit is reached in different scenarios. - -### Reaching the limit during data parsing - -During parsing of `JSON` object from the data, when the limit is reached for current block of data, all new paths will be stored in a shared data structure. We can check it using introspection functions `JSONDynamicPaths, JSONSharedDataPaths`: - -```sql -SELECT json, JSONDynamicPaths(json), JSONSharedDataPaths(json) FROM format(JSONEachRow, 'json JSON(max_dynamic_paths=3)', ' -{"json" : {"a" : {"b" : 42}, "c" : [1, 2, 3]}} -{"json" : {"a" : {"b" : 43}, "d" : "2020-01-01"}} -{"json" : {"a" : {"b" : 44}, "c" : [4, 5, 6]}} -{"json" : {"a" : {"b" : 43}, "d" : "2020-01-02", "e" : "Hello", "f" : {"g" : 42.42}}} -{"json" : {"a" : {"b" : 43}, "c" : [7, 8, 9], "f" : {"g" : 43.43}, "h" : "World"}} -') -``` - -```text -┌─json───────────────────────────────────────────────────────────┬─JSONDynamicPaths(json)─┬─JSONSharedDataPaths(json)─┠-│ {"a":{"b":"42"},"c":["1","2","3"]} │ ['a.b','c','d'] │ [] │ -│ {"a":{"b":"43"},"d":"2020-01-01"} │ ['a.b','c','d'] │ [] │ -│ {"a":{"b":"44"},"c":["4","5","6"]} │ ['a.b','c','d'] │ [] │ -│ {"a":{"b":"43"},"d":"2020-01-02","e":"Hello","f":{"g":42.42}} │ ['a.b','c','d'] │ ['e','f.g'] │ -│ {"a":{"b":"43"},"c":["7","8","9"],"f":{"g":43.43},"h":"World"} │ ['a.b','c','d'] │ ['f.g','h'] │ -└────────────────────────────────────────────────────────────────┴────────────────────────┴───────────────────────────┘ -``` - -As we can see, after inserting paths `e` and `f.g` the limit was reached and we inserted them into shared data structure. - -### During merges of data parts in MergeTree table engines - -During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths won't be able to store all paths from source parts as subcolumns. -In this case ClickHouse chooses what paths will remain as subcolumns after merge and what types will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contains -the largest number of non-null values and move the rarest paths to the shared data structure, but it depends on the implementation. - -Let's see an example of such merge. First, let's create a table with `JSON` column, set the limit of dynamic paths to `3` and insert values with `5` different paths: - -```sql -CREATE TABLE test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree ORDER BY id; -SYSTEM STOP MERGES test; -INSERT INTO test SELECT number, formatRow('JSONEachRow', number as a) FROM numbers(5); -INSERT INTO test SELECT number, formatRow('JSONEachRow', number as b) FROM numbers(4); -INSERT INTO test SELECT number, formatRow('JSONEachRow', number as c) FROM numbers(3); -INSERT INTO test SELECT number, formatRow('JSONEachRow', number as d) FROM numbers(2); -INSERT INTO test SELECT number, formatRow('JSONEachRow', number as e) FROM numbers(1); -``` - -Each insert will create a separate data pert with `JSON` column containing single path: -```sql -SELECT count(), JSONDynamicPaths(json) AS dynamic_paths, JSONSharedDataPaths(json) AS shared_data_paths, _part FROM test GROUP BY _part, dynamic_paths, shared_data_paths ORDER BY _part ASC -``` - -```text -┌─count()─┬─dynamic_paths─┬─shared_data_paths─┬─_part─────┠-│ 5 │ ['a'] │ [] │ all_1_1_0 │ -│ 4 │ ['b'] │ [] │ all_2_2_0 │ -│ 3 │ ['c'] │ [] │ all_3_3_0 │ -│ 2 │ ['d'] │ [] │ all_4_4_0 │ -│ 1 │ ['e'] │ [] │ all_5_5_0 │ -└─────────┴───────────────┴───────────────────┴───────────┘ - -``` - -Now, let's merge all parts into one and see what will happen: - -```sql -SYSTEM START MERGES test; -OPTIMIZE TABLE test FINAL; -SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; -``` - -```text -┌─count()─┬─dynamic_paths─┬─shared_data_paths─┬─_part─────┠-│ 1 │ ['a','b','c'] │ ['e'] │ all_1_5_2 │ -│ 2 │ ['a','b','c'] │ ['d'] │ all_1_5_2 │ -│ 12 │ ['a','b','c'] │ [] │ all_1_5_2 │ -└─────────┴───────────────┴───────────────────┴───────────┘ -``` - -As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and moved paths `e` and `d` to shared data structure. - -## Introspection functions - -There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes). - -## Tips for better usage of the JSON type - -Before creating `JSON` column and loading data into it, consider the following tips: - -- Investigate your data and specify as many path hints with types as you can. It will make the storage and the reading much more efficient. -- Think about what paths you will need and what paths you will never need. Specify paths that you won't need in the SKIP section and SKIP REGEXP if needed. It will improve the storage. -- Don't set `max_dynamic_paths` parameter to very high values, it can make the storage and reading less efficient. +- [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) +- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) \ No newline at end of file diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md new file mode 100644 index 00000000000..494bfba3173 --- /dev/null +++ b/docs/en/sql-reference/data-types/newjson.md @@ -0,0 +1,516 @@ +--- +slug: /en/sql-reference/data-types/json +sidebar_position: 63 +sidebar_label: JSON +keywords: [json, data type] +--- + +# JSON + +Stores JavaScript Object Notation (JSON) documents in a single column. + +:::note +This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. +If you want to use JSON type, set `allow_experimental_json_type = 1`. +::: + +To declare a column of `JSON` type, use the following syntax: + +``` sql + JSON(max_dynamic_paths=N, max_dynamic_types=M, some.path TypeName, SKIP path.to.skip, SKIP REGEXP 'paths_regexp') +``` +Where: +- `max_dynamic_paths` is an optional parameter indicating how many paths can be stored separately as subcolumns across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all other paths will be stored together in a single structure. Default value of `max_dynamic_paths` is `1024`. +- `max_dynamic_types` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a single path column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_dynamic_types` is `32`. +- `some.path TypeName` is an optional type hint for particular path in the JSON. Such paths will be always stored as subcolumns with specified type. +- `SKIP path.to.skip` is an optional hint for particular path that should be skipped during JSON parsing. Such paths will never be stored in the JSON column. If specified path is a nested JSON object, the whole nested object will be skipped. +- `SKIP REGEXP 'path_regexp'` is an optional hint with a regular expression that is used to skip paths during JSON parsing. All paths that match this regular expression will never be stored in the JSON column. + +## Creating JSON + +Using `JSON` type in table column definition: + +```sql +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : 42}, "c" : [1, 2, 3]}'), ('{"f" : "Hello, World!"}'), ('{"a" : {"b" : 43, "e" : 10}, "c" : [4, 5, 6]}'); +SELECT json FROM test; +``` + +```text +┌─json────────────────────────────────────────┠+│ {"a":{"b":"42"},"c":["1","2","3"]} │ +│ {"f":"Hello, World!"} │ +│ {"a":{"b":"43","e":"10"},"c":["4","5","6"]} │ +└─────────────────────────────────────────────┘ +``` + +```sql +CREATE TABLE test (json JSON(a.b UInt32, SKIP a.e)) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : 42}, "c" : [1, 2, 3]}'), ('{"f" : "Hello, World!"}'), ('{"a" : {"b" : 43, "e" : 10}, "c" : [4, 5, 6]}'); +SELECT json FROM test; +``` + +```text +┌─json──────────────────────────────┠+│ {"a":{"b":42},"c":[1,2,3]} │ +│ {"a":{"b":0},"f":"Hello, World!"} │ +│ {"a":{"b":43},"c":[4,5,6]} │ +└───────────────────────────────────┘ +``` + +Using CAST from 'String': + +```sql +SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json; +``` + +```text +┌─json───────────────────────────────────────────┠+│ {"a":{"b":42},"c":[1,2,3],"d":"Hello, World!"} │ +└────────────────────────────────────────────────┘ +``` + +CAST from named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. + +## Reading JSON paths as subcolumns + +JSON type supports reading every path as a separate subcolumn. If type of the requested path was not specified in the JSON type declaration, the subcolumn of the path will always have type [Dynamic](/docs/en/sql-reference/data-types/dynamic.md). + +For example: + +```sql +CREATE TABLE test (json JSON(a.b UInt32, SKIP a.e)) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : 42, "g" : 42.42}, "c" : [1, 2, 3], "d" : "2020-01-01"}'), ('{"f" : "Hello, World!", "d" : "2020-01-02"}'), ('{"a" : {"b" : 43, "e" : 10, "g" : 43.43}, "c" : [4, 5, 6]}'); +SELECT json FROM test; +``` + +```text +┌─json──────────────────────────────────────────────────┠+│ {"a":{"b":42,"g":42.42},"c":[1,2,3],"d":"2020-01-01"} │ +│ {"a":{"b":0},"d":"2020-01-02","f":"Hello, World!"} │ +│ {"a":{"b":43,"g":43.43},"c":[4,5,6]} │ +└───────────────────────────────────────────────────────┘ +``` + +```sql +SELECT json.a.b, json.a.g, json.c, json.d FROM test; +``` + +```text +┌─json.a.b─┬─json.a.g─┬─json.c──┬─json.d─────┠+│ 42 │ 42.42 │ [1,2,3] │ 2020-01-01 │ +│ 0 │ á´ºáµá´¸á´¸ │ á´ºáµá´¸á´¸ │ 2020-01-02 │ +│ 43 │ 43.43 │ [4,5,6] │ á´ºáµá´¸á´¸ │ +└──────────┴──────────┴─────────┴────────────┘ +``` + +If the requested path wasn't found in the data, it will be filled with `NULL` values: + +```sql +SELECT json.non.existing.path FROM test; +``` + +```text +┌─json.non.existing.path─┠+│ á´ºáµá´¸á´¸ │ +│ á´ºáµá´¸á´¸ │ +│ á´ºáµá´¸á´¸ │ +└────────────────────────┘ +``` + +Let's check the data types of returned subcolumns: +```sql +SELECT toTypeName(json.a.b), toTypeName(json.a.g), toTypeName(json.c), toTypeName(json.d) FROM test; +``` + +```text +┌─toTypeName(json.a.b)─┬─toTypeName(json.a.g)─┬─toTypeName(json.c)─┬─toTypeName(json.d)─┠+│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ +│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ +│ UInt32 │ Dynamic │ Dynamic │ Dynamic │ +└──────────────────────┴──────────────────────┴────────────────────┴────────────────────┘ +``` + +As we can see, for `a.b` the type is `UInt32` as we specified in the JSON type declaration, and for all other subcolumns the type is `Dynamic`. + +It is also possible to read subcolumns of a `Dynamic` type using special syntax `json.some.path.:TypeName`: + +```sql +select json.a.g.:Float64, dynamicType(json.a.g), json.d.:Date, dynamicType(json.d) FROM test; +``` + +```text +┌─json.a.g.:`Float64`─┬─dynamicType(json.a.g)─┬─json.d.:`Date`─┬─dynamicType(json.d)─┠+│ 42.42 │ Float64 │ 2020-01-01 │ Date │ +│ á´ºáµá´¸á´¸ │ None │ 2020-01-02 │ Date │ +│ 43.43 │ Float64 │ á´ºáµá´¸á´¸ │ None │ +└─────────────────────┴───────────────────────┴────────────────┴─────────────────────┘ +``` + +`Dynamic` subcolumns can be casted to any data type. In this case the exception will be thrown if internal type inside `Dynamic` cannot be casted to the requested type: + +```sql +select json.a.g::UInt64 as uint FROM test; +``` + +```text +┌─uint─┠+│ 42 │ +│ 0 │ +│ 43 │ +└──────┘ +``` + +```sql +select json.a.g::UUID as float FROM test; +``` + +```text +Received exception: +Code: 48. DB::Exception: Conversion between numeric types and UUID is not supported. Probably the passed UUID is unquoted: while executing 'FUNCTION CAST(__table1.json.a.g :: 2, 'UUID'_String :: 1) -> CAST(__table1.json.a.g, 'UUID'_String) UUID : 0'. (NOT_IMPLEMENTED) +``` + +## Reading JSON sub-objects as subcolumns + +JSON type supports reading nested objects as subcolumns with type `JSON` using special syntax `json.^some.path`: + +```sql +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test VALUES ('{"a" : {"b" : {"c" : 42, "g" : 42.42}}, "c" : [1, 2, 3], "d" : {"e" : {"f" : {"g" : "Hello, World", "h" : [1, 2, 3]}}}}'), ('{"f" : "Hello, World!", "d" : {"e" : {"f" : {"h" : [4, 5, 6]}}}}'), ('{"a" : {"b" : {"c" : 43, "e" : 10, "g" : 43.43}}, "c" : [4, 5, 6]}'); +SELECT json FROM test; +``` + +```text +┌─json────────────────────────────────────────────────────────────────────────────────────────┠+│ {"a":{"b":{"c":42,"g":42.42}},"c":[1,2,3],"d":{"e":{"f":{"g":"Hello, World","h":[1,2,3]}}}} │ +│ {"d":{"e":{"f":{"h":[4,5,6]}}},"f":"Hello, World!"} │ +│ {"a":{"b":{"c":43,"e":10,"g":43.43}},"c":[4,5,6]} │ +└─────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +```sql +SELECT json.^a.b, json.^d.e.f FROM test; +``` + +```text +┌─json.^`a`.b───────────────┬─json.^`d`.e.f────────────────────┠+│ {"c":42,"g":42.42} │ {"g":"Hello, World","h":[1,2,3]} │ +│ {} │ {"h":[4,5,6]} │ +│ {"c":43,"e":10,"g":43.43} │ {} │ +└───────────────────────────┴──────────────────────────────────┘ +``` + +:::note +Reading sub-objects as subcolumns may be inefficient, as this may require almost full scan of the JSON data. +::: + +## Types inference for paths + +During JSON parsing ClickHouse tries to detect the most appropriate data type for each JSON path. It works similar to [automatic schema inference from input data](/docs/en/interfaces/schema-inference.md) and controlled by the same settings: + +- [input_format_try_infer_integers](/docs/en/interfaces/schema-inference.md#inputformattryinferintegers) +- [input_format_try_infer_dates](/docs/en/interfaces/schema-inference.md#inputformattryinferdates) +- [input_format_try_infer_datetimes](/docs/en/interfaces/schema-inference.md#inputformattryinferdatetimes) +- [schema_inference_make_columns_nullable](/docs/en/interfaces/schema-inference.md#schemainferencemakecolumnsnullable) +- [input_format_json_try_infer_numbers_from_strings](/docs/en/interfaces/schema-inference.md#inputformatjsontryinfernumbersfromstrings) +- [input_format_json_infer_incomplete_types_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsoninferincompletetypesasstrings) +- [input_format_json_read_numbers_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadnumbersasstrings) +- [input_format_json_read_bools_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadboolsasstrings) +- [input_format_json_read_bools_as_numbers](/docs/en/interfaces/schema-inference.md#inputformatjsonreadboolsasnumbers) +- [input_format_json_read_arrays_as_strings](/docs/en/interfaces/schema-inference.md#inputformatjsonreadarraysasstrings) + +Let's see some examples: + +```sql +SELECT JSONAllPathsWithTypes('{"a" : "2020-01-01", "b" : "2020-01-01 10:00:00"}'::JSON) AS paths_with_types settings input_format_try_infer_dates=1, input_format_try_infer_datetimes=1; +``` + +```text +┌─paths_with_types─────────────────┠+│ {'a':'Date','b':'DateTime64(9)'} │ +└──────────────────────────────────┘ +``` + +```sql +SELECT JSONAllPathsWithTypes('{"a" : "2020-01-01", "b" : "2020-01-01 10:00:00"}'::JSON) AS paths_with_types settings input_format_try_infer_dates=0, input_format_try_infer_datetimes=0; +``` + +```text +┌─paths_with_types────────────┠+│ {'a':'String','b':'String'} │ +└─────────────────────────────┘ +``` + +```sql +SELECT JSONAllPathsWithTypes('{"a" : [1, 2, 3]}'::JSON) AS paths_with_types settings schema_inference_make_columns_nullable=1; +``` + +```text +┌─paths_with_types───────────────┠+│ {'a':'Array(Nullable(Int64))'} │ +└────────────────────────────────┘ +``` + +```sql +SELECT JSONAllPathsWithTypes('{"a" : [1, 2, 3]}'::JSON) AS paths_with_types settings schema_inference_make_columns_nullable=0; +``` + +```text +┌─paths_with_types─────┠+│ {'a':'Array(Int64)'} │ +└──────────────────────┘ +``` + +## Handling arrays of JSON objects + +JSON paths that contains an array of objects are parsed as type `Array(JSON)` and inserted into `Dynamic` column for this path. To read an array of objects you can extract it from `Dynamic` column as a subcolumn: + +```sql +CREATE TABLE test (json JSON) ENGINE = Memory; +INSERT INTO test VALUES +('{"a" : {"b" : [{"c" : 42, "d" : "Hello", "f" : [[{"g" : 42.42}]], "k" : {"j" : 1000}}, {"c" : 43}, {"e" : [1, 2, 3], "d" : "My", "f" : [[{"g" : 43.43, "h" : "2020-01-01"}]], "k" : {"j" : 2000}}]}}'), +('{"a" : {"b" : [1, 2, 3]}}'), +('{"a" : {"b" : [{"c" : 44, "f" : [[{"h" : "2020-01-02"}]]}, {"e" : [4, 5, 6], "d" : "World", "f" : [[{"g" : 44.44}]], "k" : {"j" : 3000}}]}}'); +SELECT json FROM test; +``` + +```text3 +┌─json────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┠+│ {"a":{"b":[{"c":"42","d":"Hello","f":[[{"g":42.42}]],"k":{"j":"1000"}},{"c":"43"},{"d":"My","e":["1","2","3"],"f":[[{"g":43.43,"h":"2020-01-01"}]],"k":{"j":"2000"}}]}} │ +│ {"a":{"b":["1","2","3"]}} │ +│ {"a":{"b":[{"c":"44","f":[[{"h":"2020-01-02"}]]},{"d":"World","e":["4","5","6"],"f":[[{"g":44.44}]],"k":{"j":"3000"}}]}} │ +└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +```sql +SELECT json.a.b, dynamicType(json.a.b) FROM test; +``` + +```text +┌─json.a.b──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─dynamicType(json.a.b)────────────────────────────────────┠+│ ['{"c":"42","d":"Hello","f":[[{"g":42.42}]],"k":{"j":"1000"}}','{"c":"43"}','{"d":"My","e":["1","2","3"],"f":[[{"g":43.43,"h":"2020-01-01"}]],"k":{"j":"2000"}}'] │ Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)) │ +│ [1,2,3] │ Array(Nullable(Int64)) │ +│ ['{"c":"44","f":[[{"h":"2020-01-02"}]]}','{"d":"World","e":["4","5","6"],"f":[[{"g":44.44}]],"k":{"j":"3000"}}'] │ Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)) │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────────────────────────────────────────────────────────┘ +``` + +As you can notice, the `max_dynamic_types/max_dynamic_paths` parameters of the nested `JSON` type were reduced compared to the default values. It's needed to avoid number of subcolumns to grow uncontrolled on nested arrays of JSON objects. + +Let's try to read subcolumns from this nested `JSON` column: + +```sql +SELECT json.a.b.:`Array(JSON)`.c, json.a.b.:`Array(JSON)`.f, json.a.b.:`Array(JSON)`.d FROM test; +``` + +```text +┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f───────────────────────────────────┬─json.a.b.:`Array(JSON)`.d─┠+│ [42,43,NULL] │ [[['{"g":42.42}']],NULL,[['{"g":43.43,"h":"2020-01-01"}']]] │ ['Hello',NULL,'My'] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [[['{"h":"2020-01-02"}']],[['{"g":44.44}']]] │ [NULL,'World'] │ +└───────────────────────────┴─────────────────────────────────────────────────────────────┴───────────────────────────┘ +``` + +We can avoid writing `Array(JSON)` subcolumn name using special syntax: + +```sql +SELECT json.a.b[].c, json.a.b[].f, json.a.b[].d FROM test; +``` + +```text +┌─json.a.b.:`Array(JSON)`.c─┬─json.a.b.:`Array(JSON)`.f───────────────────────────────────┬─json.a.b.:`Array(JSON)`.d─┠+│ [42,43,NULL] │ [[['{"g":42.42}']],NULL,[['{"g":43.43,"h":"2020-01-01"}']]] │ ['Hello',NULL,'My'] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [[['{"h":"2020-01-02"}']],[['{"g":44.44}']]] │ [NULL,'World'] │ +└───────────────────────────┴─────────────────────────────────────────────────────────────┴───────────────────────────┘ +``` + +The number of `[]` after path indicates the array level. `json.path[][]` will be transformed to `json.path.:Array(Array(JSON))` + +Let's check the paths and types inside our `Array(JSON)`: + +```sql +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b[]))) FROM test; +``` + +```text +┌─arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b.:`Array(JSON)`)))──┠+│ ('c','Int64') │ +│ ('d','String') │ +│ ('f','Array(Array(JSON(max_dynamic_types=8, max_dynamic_paths=64)))') │ +│ ('k.j','Int64') │ +│ ('e','Array(Nullable(Int64))') │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +Let's read subcolumns from `Array(JSON)` column: + +```sql +SELECT json.a.b[].c.:Int64, json.a.b[].f[][].g.:Float64, json.a.b[].f[][].h.:Date FROM test; +``` + +```text +┌─json.a.b.:`Array(JSON)`.c.:`Int64`─┬─json.a.b.:`Array(JSON)`.f.:`Array(Array(JSON))`.g.:`Float64`─┬─json.a.b.:`Array(JSON)`.f.:`Array(Array(JSON))`.h.:`Date`─┠+│ [42,43,NULL] │ [[[42.42]],[],[[43.43]]] │ [[[NULL]],[],[['2020-01-01']]] │ +│ [] │ [] │ [] │ +│ [44,NULL] │ [[[NULL]],[[44.44]]] │ [[['2020-01-02']],[[NULL]]] │ +└────────────────────────────────────┴──────────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ +``` + +We can also read sub-object subcolumns from nested `JSON` column: + +```sql +SELECT json.a.b[].^k FROM test +``` + +```text +┌─json.a.b.:`Array(JSON)`.^`k`─────────┠+│ ['{"j":"1000"}','{}','{"j":"2000"}'] │ +│ [] │ +│ ['{}','{"j":"3000"}'] │ +└──────────────────────────────────────┘ +``` + +## Reading JSON type from the data + +All text formats (JSONEachRow, TSV, CSV, CustomSeparated, Values, etc) supports reading `JSON` type. + +Examples: + +```sql +SELECT json FROM format(JSONEachRow, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP d.e, SKIP REGEXP \'b.*\')', ' +{"json" : {"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}}} +{"json" : {"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}}} +{"json" : {"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"}} +{"json" : {"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43}} +{"json" : {"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : {"c" : 11, "j" : [1, 2, 3]}, "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44}, "h" : "2020-02-02 10:00:00"}}} +') +``` + +```text +┌─json──────────────────────────────────────────────────────────┠+│ {"a":{"b":{"c":1}},"c":"42","d":{"i":["1","2","3"]}} │ +│ {"a":{"b":{"c":2}},"d":{"i":["4","5","6"]}} │ +│ {"a":{"b":{"c":3}},"e":"Hello, World!"} │ +│ {"a":{"b":{"c":4}},"c":"43"} │ +│ {"a":{"b":{"c":5}},"d":{"h":"2020-02-02 10:00:00.000000000"}} │ +└───────────────────────────────────────────────────────────────┘ +``` + +For text formats like CSV/TSV/etc `JSON` is parsed from a string containing JSON object + +```sql +SELECT json FROM format(TSV, 'json JSON(a.b.c UInt32, SKIP a.b.d, SKIP REGEXP \'b.*\')', +'{"a" : {"b" : {"c" : 1, "d" : [0, 1]}}, "b" : "2020-01-01", "c" : 42, "d" : {"e" : {"f" : ["s1", "s2"]}, "i" : [1, 2, 3]}} +{"a" : {"b" : {"c" : 2, "d" : [2, 3]}}, "b" : [1, 2, 3], "c" : null, "d" : {"e" : {"g" : 43}, "i" : [4, 5, 6]}} +{"a" : {"b" : {"c" : 3, "d" : [4, 5]}}, "b" : {"c" : 10}, "e" : "Hello, World!"} +{"a" : {"b" : {"c" : 4, "d" : [6, 7]}}, "c" : 43} +{"a" : {"b" : {"c" : 5, "d" : [8, 9]}}, "b" : {"c" : 11, "j" : [1, 2, 3]}, "d" : {"e" : {"f" : ["s3", "s4"], "g" : 44}, "h" : "2020-02-02 10:00:00"}}') +``` + +```text +┌─json──────────────────────────────────────────────────────────┠+│ {"a":{"b":{"c":1}},"c":"42","d":{"i":["1","2","3"]}} │ +│ {"a":{"b":{"c":2}},"d":{"i":["4","5","6"]}} │ +│ {"a":{"b":{"c":3}},"e":"Hello, World!"} │ +│ {"a":{"b":{"c":4}},"c":"43"} │ +│ {"a":{"b":{"c":5}},"d":{"h":"2020-02-02 10:00:00.000000000"}} │ +└───────────────────────────────────────────────────────────────┘ +``` + +## Reaching the limit of dynamic paths inside JSON + +`JSON` data type can store only limited number of paths as separate subcolumns inside. By default, this limit is 1024, but you can change it in type declaration using parameter `max_dynamic_paths`. +When the limit is reached, all new paths inserted to `JSON` column will be stored in a single shared data structure. It's still possible to read such paths as subcolumns, but it will require reading the whole +shared data structure to extract the values of this path. This limit is needed to avoid the enormous number of different subcolumns that can make the table unusable. + +Let's see what happens when the limit is reached in different scenarios. + +### Reaching the limit during data parsing + +During parsing of `JSON` object from the data, when the limit is reached for current block of data, all new paths will be stored in a shared data structure. We can check it using introspection functions `JSONDynamicPaths, JSONSharedDataPaths`: + +```sql +SELECT json, JSONDynamicPaths(json), JSONSharedDataPaths(json) FROM format(JSONEachRow, 'json JSON(max_dynamic_paths=3)', ' +{"json" : {"a" : {"b" : 42}, "c" : [1, 2, 3]}} +{"json" : {"a" : {"b" : 43}, "d" : "2020-01-01"}} +{"json" : {"a" : {"b" : 44}, "c" : [4, 5, 6]}} +{"json" : {"a" : {"b" : 43}, "d" : "2020-01-02", "e" : "Hello", "f" : {"g" : 42.42}}} +{"json" : {"a" : {"b" : 43}, "c" : [7, 8, 9], "f" : {"g" : 43.43}, "h" : "World"}} +') +``` + +```text +┌─json───────────────────────────────────────────────────────────┬─JSONDynamicPaths(json)─┬─JSONSharedDataPaths(json)─┠+│ {"a":{"b":"42"},"c":["1","2","3"]} │ ['a.b','c','d'] │ [] │ +│ {"a":{"b":"43"},"d":"2020-01-01"} │ ['a.b','c','d'] │ [] │ +│ {"a":{"b":"44"},"c":["4","5","6"]} │ ['a.b','c','d'] │ [] │ +│ {"a":{"b":"43"},"d":"2020-01-02","e":"Hello","f":{"g":42.42}} │ ['a.b','c','d'] │ ['e','f.g'] │ +│ {"a":{"b":"43"},"c":["7","8","9"],"f":{"g":43.43},"h":"World"} │ ['a.b','c','d'] │ ['f.g','h'] │ +└────────────────────────────────────────────────────────────────┴────────────────────────┴───────────────────────────┘ +``` + +As we can see, after inserting paths `e` and `f.g` the limit was reached and we inserted them into shared data structure. + +### During merges of data parts in MergeTree table engines + +During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths won't be able to store all paths from source parts as subcolumns. +In this case ClickHouse chooses what paths will remain as subcolumns after merge and what types will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contains +the largest number of non-null values and move the rarest paths to the shared data structure, but it depends on the implementation. + +Let's see an example of such merge. First, let's create a table with `JSON` column, set the limit of dynamic paths to `3` and insert values with `5` different paths: + +```sql +CREATE TABLE test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree ORDER BY id; +SYSTEM STOP MERGES test; +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as a) FROM numbers(5); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as b) FROM numbers(4); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as c) FROM numbers(3); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as d) FROM numbers(2); +INSERT INTO test SELECT number, formatRow('JSONEachRow', number as e) FROM numbers(1); +``` + +Each insert will create a separate data pert with `JSON` column containing single path: +```sql +SELECT count(), JSONDynamicPaths(json) AS dynamic_paths, JSONSharedDataPaths(json) AS shared_data_paths, _part FROM test GROUP BY _part, dynamic_paths, shared_data_paths ORDER BY _part ASC +``` + +```text +┌─count()─┬─dynamic_paths─┬─shared_data_paths─┬─_part─────┠+│ 5 │ ['a'] │ [] │ all_1_1_0 │ +│ 4 │ ['b'] │ [] │ all_2_2_0 │ +│ 3 │ ['c'] │ [] │ all_3_3_0 │ +│ 2 │ ['d'] │ [] │ all_4_4_0 │ +│ 1 │ ['e'] │ [] │ all_5_5_0 │ +└─────────┴───────────────┴───────────────────┴───────────┘ + +``` + +Now, let's merge all parts into one and see what will happen: + +```sql +SYSTEM START MERGES test; +OPTIMIZE TABLE test FINAL; +SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; +``` + +```text +┌─count()─┬─dynamic_paths─┬─shared_data_paths─┬─_part─────┠+│ 1 │ ['a','b','c'] │ ['e'] │ all_1_5_2 │ +│ 2 │ ['a','b','c'] │ ['d'] │ all_1_5_2 │ +│ 12 │ ['a','b','c'] │ [] │ all_1_5_2 │ +└─────────┴───────────────┴───────────────────┴───────────┘ +``` + +As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and moved paths `e` and `d` to shared data structure. + +## Introspection functions + +There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes). + +## Tips for better usage of the JSON type + +Before creating `JSON` column and loading data into it, consider the following tips: + +- Investigate your data and specify as many path hints with types as you can. It will make the storage and the reading much more efficient. +- Think about what paths you will need and what paths you will never need. Specify paths that you won't need in the SKIP section and SKIP REGEXP if needed. It will improve the storage. +- Don't set `max_dynamic_paths` parameter to very high values, it can make the storage and reading less efficient. diff --git a/docs/en/sql-reference/data-types/object-json.md b/docs/en/sql-reference/data-types/object-json.md deleted file mode 100644 index 9609f30aac1..00000000000 --- a/docs/en/sql-reference/data-types/object-json.md +++ /dev/null @@ -1,83 +0,0 @@ ---- -slug: /en/sql-reference/data-types/object-json -sidebar_position: 26 -sidebar_label: Object Data Type -keywords: [object, data type] ---- - -# Object Data Type (deprecated) - -**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864). - -
- -Stores JavaScript Object Notation (JSON) documents in a single column. - -`JSON` can be used as an alias to `Object('json')` when setting [use_json_alias_for_old_object_type](../../operations/settings/settings.md#usejsonaliasforoldobjecttype) is enabled. - -## Example - -**Example 1** - -Creating a table with a `JSON` column and inserting data into it: - -```sql -CREATE TABLE json -( - o JSON -) -ENGINE = Memory -``` - -```sql -INSERT INTO json VALUES ('{"a": 1, "b": { "c": 2, "d": [1, 2, 3] }}') -``` - -```sql -SELECT o.a, o.b.c, o.b.d[3] FROM json -``` - -```text -┌─o.a─┬─o.b.c─┬─arrayElement(o.b.d, 3)─┠-│ 1 │ 2 │ 3 │ -└─────┴───────┴────────────────────────┘ -``` - -**Example 2** - -To be able to create an ordered `MergeTree` family table, the sorting key has to be extracted into its column. For example, to insert a file of compressed HTTP access logs in JSON format: - -```sql -CREATE TABLE logs -( - timestamp DateTime, - message JSON -) -ENGINE = MergeTree -ORDER BY timestamp -``` - -```sql -INSERT INTO logs -SELECT parseDateTimeBestEffort(JSONExtractString(json, 'timestamp')), json -FROM file('access.json.gz', JSONAsString) -``` - -## Displaying JSON columns - -When displaying a `JSON` column, ClickHouse only shows the field values by default (because internally, it is represented as a tuple). You can also display the field names by setting `output_format_json_named_tuples_as_objects = 1`: - -```sql -SET output_format_json_named_tuples_as_objects = 1 - -SELECT * FROM json FORMAT JSONEachRow -``` - -```text -{"o":{"a":1,"b":{"c":2,"d":[1,2,3]}}} -``` - -## Related Content - -- [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) -- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) \ No newline at end of file diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index aa2010579af..65aacf2b539 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1050,6 +1050,20 @@ bool ColumnObject::isFinalized() const return finalized; } +void ColumnObject::getExtremes(DB::Field & min, DB::Field & max) const +{ + if (size() == 0) + { + min = Object(); + max = Object(); + } + else + { + get(0, min); + get(0, max); + } +} + void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & source_columns) { if (!empty()) diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 61e78e727a0..0b949cbcb3a 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -113,11 +113,7 @@ public: #else int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } #endif - void getExtremes(Field & min, Field & max) const override - { - min = Object(); - max = Object(); - } + void getExtremes(Field & min, Field & max) const override; void reserve(size_t n) override; void ensureOwnership() override; diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 665f1a6ecd2..61d34b49964 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -519,6 +519,13 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID /// We can insert any field to Dynamic column. return src; } + else if (isObject(type)) + { + if (src.getType() == Field::Types::Object) + return src; /// Already in needed type. + + /// TODO: add conversion from Map/Tuple to Object. + } /// Conversion from string by parsing. if (src.getType() == Field::Types::String) diff --git a/tests/queries/0_stateless/01825_new_type_json_12.reference b/tests/queries/0_stateless/01825_new_type_json_12.reference index 3fd2abcac50..36f014f0178 100644 --- a/tests/queries/0_stateless/01825_new_type_json_12.reference +++ b/tests/queries/0_stateless/01825_new_type_json_12.reference @@ -3,12 +3,11 @@ ('key_1','Array(JSON(max_dynamic_types=8, max_dynamic_paths=64))') ('key_3','Array(JSON(max_dynamic_types=4, max_dynamic_paths=16))') ('key_4','Int64') +('key_4','String') ('key_5','Float64') ('key_6','Float64') -('key_7','Int64') ('key_6','String') -('key_5','Int64') ('key_7','Float64') -('key_4','String') -{"obj":{"id":"1","key_0":[{"key_1":[{"key_3":[{"key_4":"1048576","key_5":0.0001048576,"key_6":25.5,"key_7":"1025"},{"key_6":"","key_7":"2"}]}]},{},{"key_1":[{"key_3":[{"key_5":"-1","key_6":"aqbjfiruu","key_7":-922337203685477600},{"key_4":"","key_6":"","key_7":"65537"}]},{"key_3":[{"key_4":"ghdqyeiom","key_5":"1048575","key_7":21474836.48}]}]}]}} +('key_7','Int64') +{"obj":{"id":"1","key_0":[{"key_1":[{"key_3":[{"key_4":"1048576","key_5":0.0001048576,"key_6":25.5,"key_7":"1025"},{"key_6":"","key_7":"2"}]}]},{},{"key_1":[{"key_3":[{"key_5":-1,"key_6":"aqbjfiruu","key_7":-922337203685477600},{"key_4":"","key_6":"","key_7":"65537"}]},{"key_3":[{"key_4":"ghdqyeiom","key_5":1048575,"key_7":21474836.48}]}]}]}} [[[1048576,NULL]],[],[[NULL,''],['ghdqyeiom']]] [[[0.0001048576,NULL]],[],[[-1,NULL],[1048575]]] [[[25.5,'']],[],[['aqbjfiruu',''],[NULL]]] [[[1025,2]],[],[[-922337203685477600,65537],[21474836.48]]] diff --git a/tests/queries/0_stateless/01825_new_type_json_3.reference.j2 b/tests/queries/0_stateless/01825_new_type_json_3.reference.j2 index 62c00713c45..8ff87d1132b 100644 --- a/tests/queries/0_stateless/01825_new_type_json_3.reference.j2 +++ b/tests/queries/0_stateless/01825_new_type_json_3.reference.j2 @@ -25,7 +25,7 @@ data JSON ======== 1 {"k1":{"k2":"1","k3":"foo"}} {'k1.k2':'Int64','k1.k3':'String'} 2 {"k4":["1","2","3"]} {'k4':'Array(Nullable(Int64))'} -3 {"k1":{"k2":"10"},"k4":[]} {'k1.k2':'Int64','k4':'Array(Nullable(String))'} +3 {"k1":{"k2":"10"},"k4":[]} {'k1.k2':'Int64','k4':'Array(Nullable(Int64))'} 1 1 foo \N 2 \N \N [1,2,3] 3 10 \N [] @@ -53,7 +53,7 @@ data JSON ======== 1 {"k1":{"k2":"1","k3":"foo"}} {'k1.k2':'Int64','k1.k3':'String'} 2 {"k4":["1","2","3"]} {'k4':'Array(Nullable(Int64))'} -3 {"k1":{"k2":"10"},"k4":[]} {'k1.k2':'Int64','k4':'Array(Nullable(String))'} +3 {"k1":{"k2":"10"},"k4":[]} {'k1.k2':'Int64','k4':'Array(Nullable(Int64))'} 1 1 foo \N 2 \N \N [1,2,3] 3 10 \N [] diff --git a/tests/queries/0_stateless/01825_new_type_json_6.reference b/tests/queries/0_stateless/01825_new_type_json_6.reference index 0227093eab4..dacf9ec4a23 100644 --- a/tests/queries/0_stateless/01825_new_type_json_6.reference +++ b/tests/queries/0_stateless/01825_new_type_json_6.reference @@ -1,9 +1,9 @@ ('key','String') ('out','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') +('outputs','Array(JSON(max_dynamic_types=8, max_dynamic_paths=64))') ('outputs','Array(Nullable(String))') ('type','Int64') ('value','Int64') -('outputs','Array(JSON(max_dynamic_types=8, max_dynamic_paths=64))') ('index','Int64') ('n','Int64') v1 [0,0] [1,2] [[],[1960131]] [[],[0]] diff --git a/tests/queries/0_stateless/01825_new_type_json_9.reference b/tests/queries/0_stateless/01825_new_type_json_9.reference new file mode 100644 index 00000000000..9ad0b55b9c5 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_9.reference @@ -0,0 +1,3 @@ +('foo','Int64') +('k1','Int64') +('k2','Int64') diff --git a/tests/queries/0_stateless/01825_new_type_json_9.sql b/tests/queries/0_stateless/01825_new_type_json_9.sql new file mode 100644 index 00000000000..626016a0a81 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_9.sql @@ -0,0 +1,16 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json; + +SET allow_experimental_json_type = 1; + +CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}}; +INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}}; + +OPTIMIZE TABLE t_json FINAL; + +SELECT distinct arrayJoin(JSONAllPathsWithTypes(obj)) as path from t_json order by path; + +DROP TABLE IF EXISTS t_json; diff --git a/tests/queries/0_stateless/01825_new_type_json_bools.reference b/tests/queries/0_stateless/01825_new_type_json_bools.reference new file mode 100644 index 00000000000..ebbfd28c321 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_bools.reference @@ -0,0 +1 @@ +{"k1":true,"k2":false} {'k1':'Bool','k2':'Bool'} diff --git a/tests/queries/0_stateless/01825_new_type_json_bools.sql b/tests/queries/0_stateless/01825_new_type_json_bools.sql new file mode 100644 index 00000000000..f8775d0f00a --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_bools.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json_bools; +SET allow_experimental_json_type = 1; + +CREATE TABLE t_json_bools (data JSON) ENGINE = Memory; +INSERT INTO t_json_bools VALUES ('{"k1": true, "k2": false}'); +SELECT data, JSONAllPathsWithTypes(data) FROM t_json_bools; + +DROP TABLE t_json_bools; diff --git a/tests/queries/0_stateless/01825_new_type_json_btc.reference b/tests/queries/0_stateless/01825_new_type_json_btc.reference index cf8017c17c1..b7ea598d47b 100644 --- a/tests/queries/0_stateless/01825_new_type_json_btc.reference +++ b/tests/queries/0_stateless/01825_new_type_json_btc.reference @@ -5,6 +5,7 @@ ('inputs','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') ('lock_time','Int64') ('out','Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))') +('rbf','Bool') ('relayed_by','String') ('size','Int64') ('time','Int64') @@ -13,7 +14,6 @@ ('vin_sz','Int64') ('vout_sz','Int64') ('weight','Int64') -('rbf','Bool') ('index','Int64') ('prev_out.addr','String') ('prev_out.n','Int64') diff --git a/tests/queries/0_stateless/01825_new_type_json_distributed.reference b/tests/queries/0_stateless/01825_new_type_json_distributed.reference new file mode 100644 index 00000000000..b2cbe847542 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_distributed.reference @@ -0,0 +1,4 @@ +{"k1":"2","k2":{"k3":"qqq","k4":["44","55"]}} {'k1':'Int64','k2.k3':'String','k2.k4':'Array(Nullable(Int64))'} +{"k1":"2","k2":{"k3":"qqq","k4":["44","55"]}} {'k1':'Int64','k2.k3':'String','k2.k4':'Array(Nullable(Int64))'} +2 qqq [44,55] +2 qqq [44,55] diff --git a/tests/queries/0_stateless/01825_new_type_json_distributed.sql b/tests/queries/0_stateless/01825_new_type_json_distributed.sql new file mode 100644 index 00000000000..0fede046927 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_distributed.sql @@ -0,0 +1,18 @@ +-- Tags: no-fasttest + +SET allow_experimental_json_type = 1; + +DROP TABLE IF EXISTS t_json_local; +DROP TABLE IF EXISTS t_json_dist; + +CREATE TABLE t_json_local(data JSON) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE t_json_dist AS t_json_local ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), t_json_local); + +INSERT INTO t_json_local FORMAT JSONAsObject {"k1": 2, "k2": {"k3": "qqq", "k4": [44, 55]}} +; + +SELECT data, JSONAllPathsWithTypes(data) FROM t_json_dist; +SELECT data.k1, data.k2.k3, data.k2.k4 FROM t_json_dist; + +DROP TABLE IF EXISTS t_json_local; +DROP TABLE IF EXISTS t_json_dist; diff --git a/tests/queries/0_stateless/01825_new_type_json_in_array.reference b/tests/queries/0_stateless/01825_new_type_json_in_array.reference new file mode 100644 index 00000000000..0ca02385389 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_in_array.reference @@ -0,0 +1,28 @@ +{"id":1,"arr":[{"k1":"1","k2":{"k3":"2","k4":"3"}},{"k1":"2","k2":{"k5":"foo"}}]} +{"id":2,"arr":[{"k1":"3","k2":{"k3":"4","k4":"5"}}]} +1 [1,2] [2,NULL] [3,NULL] [NULL,'foo'] +2 [3] [4] [5] [NULL] +{"arr":{"k1":"1","k2":{"k3":"2","k4":"3"}}} +{"arr":{"k1":"2","k2":{"k5":"foo"}}} +{"arr":{"k1":"3","k2":{"k3":"4","k4":"5"}}} +('k1','Int64') +('k2.k3','Int64') +('k2.k4','Int64') +('k2.k5','String') +{"id":1,"arr":[{"k1":[{"k2":"aaa","k3":"bbb"},{"k2":"ccc"}]}]} +{"id":2,"arr":[{"k1":[{"k3":"ddd","k4":"10"},{"k4":"20"}],"k5":{"k6":"foo"}}]} +1 [['aaa','ccc']] [['bbb',NULL]] [[NULL,NULL]] [NULL] +2 [[NULL,NULL]] [['ddd',NULL]] [[10,20]] ['foo'] +{"k1":{"k2":"aaa","k3":"bbb"}} +{"k1":{"k2":"ccc"}} +{"k1":{"k3":"ddd","k4":"10"}} +{"k1":{"k4":"20"}} +('k2','String') +('k3','String') +('k4','Int64') +{"arr":[{"x":1}]} +{"arr":{"x":{"y":1},"t":{"y":2}}} +{"arr":[1,{"y":1}]} +{"arr":[2,{"y":2}]} +{"arr":[{"x":"aaa","y":["1","2","3"]}]} +{"arr":[{"x":1}]} diff --git a/tests/queries/0_stateless/01825_new_type_json_in_array.sql b/tests/queries/0_stateless/01825_new_type_json_in_array.sql new file mode 100644 index 00000000000..e9ba6e402ac --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_in_array.sql @@ -0,0 +1,34 @@ +-- Tags: no-fasttest + +SET allow_experimental_json_type = 1; +DROP TABLE IF EXISTS t_json_array; + +CREATE TABLE t_json_array (id UInt32, arr Array(JSON)) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": 1, "k2": {"k3": 2, "k4": 3}}, {"k1": 2, "k2": {"k5": "foo"}}]} +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": 3, "k2": {"k3": 4, "k4": 5}}]} + + +SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow; +SELECT id, arr.k1, arr.k2.k3, arr.k2.k4, arr.k2.k5 FROM t_json_array ORDER BY id; +SELECT arr FROM t_json_array ARRAY JOIN arr ORDER BY arr.k1 FORMAT JSONEachRow; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(arr))) as path FROM t_json_array order by path; + +TRUNCATE TABLE t_json_array; + +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} + +SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow; +SELECT id, arr.k1[].k2, arr.k1[].k3, arr.k1[].k4, arr.k5.k6 FROM t_json_array ORDER BY id; + +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; + +DROP TABLE t_json_array; + +SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow; +SELECT * FROM values('arr Map(String, JSON)', '{\'x\' : \'{"y" : 1}\', \'t\' : \'{"y" : 2}\'}') FORMAT JSONEachRow; +SELECT * FROM values('arr Tuple(Int32, JSON)', '(1, \'{"y" : 1}\')', '(2, \'{"y" : 2}\')') FORMAT JSONEachRow; +SELECT * FROM format(JSONEachRow, '{"arr" : [{"x" : "aaa", "y" : [1,2,3]}]}') FORMAT JSONEachRow; +SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow; diff --git a/tests/queries/0_stateless/01825_new_type_json_in_other_types.reference b/tests/queries/0_stateless/01825_new_type_json_in_other_types.reference new file mode 100644 index 00000000000..03913e5098e --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_in_other_types.reference @@ -0,0 +1,17 @@ +Tuple(String, Map(String, Array(JSON)), JSON) +============= +{"id":1,"data":["foo",{"aa":[{"k1":[{"k2":"1","k3":"2"},{"k3":"3"}]},{"k1":[{"k2":"4"},{"k3":"5"},{"k2":"6"}],"k4":"qqq"}],"bb":[{"k4":"www"},{"k1":[{"k2":"7","k3":"8"},{"k2":"9","k3":"10"},{"k2":"11","k3":"12"}]}]},{"k1":"aa","k2":{"k3":"bb","k4":"c"}}]} +{"id":2,"data":["bar",{"aa":[{"k1":[{"k2":"13","k3":"14"},{"k2":"15","k3":"16"}],"k4":"www"}]},{}]} +{"id":3,"data":["some",{"aa":[{"k1":[{"k3":"20","k5":"some"}]}]},{"k1":"eee"}]} +============= +{"aa":[{"k1":[{"k2":"1","k3":"2"},{"k3":"3"}]},{"k1":[{"k2":"4"},{"k3":"5"},{"k2":"6"}],"k4":"qqq"}],"bb":[{"k4":"www"},{"k1":[{"k2":"7","k3":"8"},{"k2":"9","k3":"10"},{"k2":"11","k3":"12"}]}]} +{"aa":[{"k1":[{"k2":"13","k3":"14"},{"k2":"15","k3":"16"}],"k4":"www"}],"bb":[]} +{"aa":[{"k1":[{"k3":"20","k5":"some"}]}],"bb":[]} +============= +{"k1":[[{"k2":"1","k3":"2"},{"k3":"3"}],[{"k2":"4"},{"k3":"5"},{"k2":"6"}]],"k4":[null,"qqq"]} +{"k1":[[{"k2":"13","k3":"14"},{"k2":"15","k3":"16"}]],"k4":["www"]} +{"k1":[[{"k3":"20","k5":"some"}]],"k4":[null]} +============= +{"obj":{"k1":"aa","k2":{"k3":"bb","k4":"c"}}} +{"obj":{}} +{"obj":{"k1":"eee"}} diff --git a/tests/queries/0_stateless/01825_new_type_json_in_other_types.sh b/tests/queries/0_stateless/01825_new_type_json_in_other_types.sh new file mode 100755 index 00000000000..cd009c3ef70 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_in_other_types.sh @@ -0,0 +1,90 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_nested" + +${CLICKHOUSE_CLIENT} -q " + CREATE TABLE t_json_nested + ( + id UInt32, + data Tuple(String, Map(String, Array(JSON)), JSON) + ) + ENGINE = MergeTree ORDER BY id" --allow_experimental_json_type 1 + +cat < 1; + +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; +SELECT id, data FROM type_json_dst ORDER BY id; + +INSERT INTO type_json_dst VALUES (4, '{"arr": [{"k11": 5, "k22": 6}, {"k11": 7, "k33": 8}]}'); + +INSERT INTO type_json_src VALUES (5, '{"arr": "not array"}'); +INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; + +TRUNCATE TABLE type_json_src; +INSERT INTO type_json_src VALUES (6, '{"arr": [{"k22": "str1"}]}') +INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; + +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; +SELECT id, data FROM type_json_dst ORDER BY id; + +DROP TABLE type_json_src; +DROP TABLE type_json_dst; + +CREATE TABLE type_json_dst (data JSON) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE type_json_src (data String) ENGINE = MergeTree ORDER BY tuple(); + +SYSTEM STOP MERGES type_json_src; + +SET max_threads = 1; +SET max_insert_threads = 1; +SET output_format_json_named_tuples_as_objects = 1; + +INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]}; +INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]}; + +INSERT INTO type_json_dst SELECT data FROM type_json_src; + +SELECT * FROM type_json_dst ORDER BY data.k1 FORMAT JSONEachRow; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; + +TRUNCATE TABLE type_json_src; +TRUNCATE TABLE type_json_dst; + +INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]}; +INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]}; + +INSERT INTO type_json_dst SELECT data FROM type_json_src; + +SELECT * FROM type_json_dst ORDER BY data.k1 FORMAT JSONEachRow; +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; + +DROP TABLE type_json_src; +DROP TABLE type_json_dst; diff --git a/tests/queries/0_stateless/01825_new_type_json_missed_values.reference b/tests/queries/0_stateless/01825_new_type_json_missed_values.reference new file mode 100644 index 00000000000..952b5652bc1 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_missed_values.reference @@ -0,0 +1,4 @@ +('foo','Int64') +('k1','Int64') +('k2','Int64') +1 diff --git a/tests/queries/0_stateless/01825_new_type_json_missed_values.sql b/tests/queries/0_stateless/01825_new_type_json_missed_values.sql new file mode 100644 index 00000000000..84bd8a19c18 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_missed_values.sql @@ -0,0 +1,19 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json; + +SET allow_experimental_json_type = 1; + +CREATE TABLE t_json(id UInt64, obj JSON) +ENGINE = MergeTree ORDER BY id +SETTINGS min_bytes_for_wide_part = 0; + +SYSTEM STOP MERGES t_json; + +INSERT INTO t_json SELECT number, '{"k1": 1, "k2": 2}' FROM numbers(1000000); +INSERT INTO t_json VALUES (1000001, '{"foo": 1}'); + +SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(obj)) AS path FROM t_json ORDER BY path; +SELECT count() FROM t_json WHERE obj.foo IS NOT NULL; + +DROP TABLE IF EXISTS t_json; diff --git a/tests/queries/0_stateless/01825_new_type_json_multiple_files.reference b/tests/queries/0_stateless/01825_new_type_json_multiple_files.reference new file mode 100644 index 00000000000..63c12792c17 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_multiple_files.reference @@ -0,0 +1,22 @@ +{"data":{"k0":"100"}} +{"data":{"k1":"100"}} +{"data":{"k2":"100"}} +{"data":{"k3":"100"}} +{"data":{"k4":"100"}} +{"data":{"k5":"100"}} +('k0','Int64') +('k1','Int64') +('k2','Int64') +('k3','Int64') +('k4','Int64') +('k5','Int64') +{"data":{"k0":"100"}} +{"data":{"k1":"100"}} +{"data":{"k2":"100"}} +('k0','Int64') +('k1','Int64') +('k2','Int64') +{"data":{"k1":"100"}} +{"data":{"k3":"100"}} +('k1','Int64') +('k3','Int64') diff --git a/tests/queries/0_stateless/01825_new_type_json_multiple_files.sh b/tests/queries/0_stateless/01825_new_type_json_multiple_files.sh new file mode 100755 index 00000000000..9cb37987628 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_multiple_files.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +for f in "${USER_FILES_PATH:?}/${CLICKHOUSE_DATABASE}"_*.json; do + [ -e $f ] && rm $f +done + +for i in {0..5}; do + echo "{\"k$i\": 100}" > "$USER_FILES_PATH/${CLICKHOUSE_DATABASE}_$i.json" +done + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_files (file String, data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_json_type 1 + +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('${CLICKHOUSE_DATABASE}_*.json', 'JSONAsObject', 'data JSON')" --allow_experimental_json_type 1 + +${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_files ORDER BY path" + +${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \ + SELECT _file, data FROM file('${CLICKHOUSE_DATABASE}_*.json', 'JSONAsObject', 'data JSON') \ + ORDER BY _file LIMIT 3" --max_threads 1 --min_insert_block_size_rows 1 --max_insert_block_size 1 --max_block_size 1 --allow_experimental_json_type 1 + +${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file, data FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_files ORDER BY path" + +${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \ + SELECT _file, data FROM file('${CLICKHOUSE_DATABASE}_*.json', 'JSONAsObject', 'data JSON') \ + WHERE _file IN ('${CLICKHOUSE_DATABASE}_1.json', '${CLICKHOUSE_DATABASE}_3.json')" --allow_experimental_json_type 1 + +${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_files ORDER BY path" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files" +rm "$USER_FILES_PATH"/${CLICKHOUSE_DATABASE}_*.json diff --git a/tests/queries/0_stateless/01825_new_type_json_mutations.reference b/tests/queries/0_stateless/01825_new_type_json_mutations.reference new file mode 100644 index 00000000000..c7523661a3b --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_mutations.reference @@ -0,0 +1,7 @@ +1 q {"k1":"1","k2":"2","k3":[{"k4":"aaa"},{"k4":"bbb"}]} +2 w {"k1":"3","k2":"4","k3":[{"k4":"ccc"}]} +3 e {"k1":"5","k2":"6"} +1 q {"k1":"1","k2":"2","k3":[{"k4":"aaa"},{"k4":"bbb"}]} +3 e {"k1":"5","k2":"6"} +1 foo +3 foo diff --git a/tests/queries/0_stateless/01825_new_type_json_mutations.sql b/tests/queries/0_stateless/01825_new_type_json_mutations.sql new file mode 100644 index 00000000000..77feee692d9 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_mutations.sql @@ -0,0 +1,21 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json_mutations; + +SET allow_experimental_json_type = 1; +SET output_format_json_named_tuples_as_objects = 1; +SET mutations_sync = 2; + +CREATE TABLE t_json_mutations(id UInt32, s String, obj JSON) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_json_mutations VALUES (1, 'q', '{"k1": 1, "k2": 2, "k3": [{"k4": "aaa"}, {"k4": "bbb"}]}'); +INSERT INTO t_json_mutations VALUES (2, 'w', '{"k1": 3, "k2": 4, "k3": [{"k4": "ccc"}]}'); +INSERT INTO t_json_mutations VALUES (3, 'e', '{"k1": 5, "k2": 6}'); + +SELECT * FROM t_json_mutations ORDER BY id; +ALTER TABLE t_json_mutations DELETE WHERE id = 2; +SELECT * FROM t_json_mutations ORDER BY id; +ALTER TABLE t_json_mutations DROP COLUMN s, DROP COLUMN obj, ADD COLUMN t String DEFAULT 'foo'; +SELECT * FROM t_json_mutations ORDER BY id; + +DROP TABLE t_json_mutations; diff --git a/tests/queries/0_stateless/01825_new_type_json_order_by.reference b/tests/queries/0_stateless/01825_new_type_json_order_by.reference new file mode 100644 index 00000000000..611d2835127 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_order_by.reference @@ -0,0 +1,6 @@ +0 +0 +{"k":"v"} + +{"k":"v"} +{"k":"v"} diff --git a/tests/queries/0_stateless/01825_new_type_json_order_by.sql b/tests/queries/0_stateless/01825_new_type_json_order_by.sql new file mode 100644 index 00000000000..6b5fb40aed4 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_order_by.sql @@ -0,0 +1,6 @@ +-- Tags: no-fasttest + +SET allow_experimental_json_type = 1; +SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON); +SELECT dummy FROM system.one ORDER BY materialize('{"k":"v"}'::JSON), dummy; +SELECT materialize('{"k":"v"}'::JSON) SETTINGS extremes = 1; diff --git a/tests/queries/0_stateless/01825_new_type_json_partitions.reference b/tests/queries/0_stateless/01825_new_type_json_partitions.reference new file mode 100644 index 00000000000..c5839472132 --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_partitions.reference @@ -0,0 +1,2 @@ +{"id":1,"obj":{"k1":"v1"}} +{"id":2,"obj":{"k2":"v2"}} diff --git a/tests/queries/0_stateless/01825_new_type_json_partitions.sql b/tests/queries/0_stateless/01825_new_type_json_partitions.sql new file mode 100644 index 00000000000..d1f37dedded --- /dev/null +++ b/tests/queries/0_stateless/01825_new_type_json_partitions.sql @@ -0,0 +1,14 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json_partitions; + +SET allow_experimental_json_type = 1; + +CREATE TABLE t_json_partitions (id UInt32, obj JSON) +ENGINE MergeTree ORDER BY id PARTITION BY id; + +INSERT INTO t_json_partitions FORMAT JSONEachRow {"id": 1, "obj": {"k1": "v1"}} {"id": 2, "obj": {"k2": "v2"}}; + +SELECT * FROM t_json_partitions ORDER BY id FORMAT JSONEachRow; + +DROP TABLE t_json_partitions; diff --git a/tests/queries/0_stateless/01825_type_json_10.reference b/tests/queries/0_stateless/01825_type_json_10.reference index e69de29bb2d..4161fb59c93 100644 --- a/tests/queries/0_stateless/01825_type_json_10.reference +++ b/tests/queries/0_stateless/01825_type_json_10.reference @@ -0,0 +1,9 @@ +Tuple(\n a Tuple(\n b Int8,\n c Nested(d Int8, e Array(Int16), f Int8))) +{"o":{"a":{"b":1,"c":[{"d":10,"e":[31],"f":0},{"d":20,"e":[63,127],"f":0}]}}} +{"o":{"a":{"b":2,"c":[]}}} +{"o":{"a":{"b":3,"c":[{"d":0,"e":[32],"f":20},{"d":0,"e":[64,128],"f":30}]}}} +{"o":{"a":{"b":4,"c":[]}}} +1 [10,20] [[31],[63,127]] [0,0] +2 [] [] [] +3 [0,0] [[32],[64,128]] [20,30] +4 [] [] [] diff --git a/tests/queries/0_stateless/01825_type_json_5.sql b/tests/queries/0_stateless/01825_type_json_5.sql index f508042c4ce..f50eee6b8ef 100644 --- a/tests/queries/0_stateless/01825_type_json_5.sql +++ b/tests/queries/0_stateless/01825_type_json_5.sql @@ -2,8 +2,8 @@ SET allow_experimental_object_type = 1; -SELECT '{"a": {"b": 1, "c": 2}}'::JSON AS s; -SELECT '{"a": {"b": 1, "c": 2}}'::JSON AS s format JSONEachRow; +SELECT '{"a": {"b": 1, "c": 2}}'::Object('JSON') AS s; +SELECT '{"a": {"b": 1, "c": 2}}'::Object('JSON') AS s format JSONEachRow; DROP TABLE IF EXISTS t_json_5; DROP TABLE IF EXISTS t_json_str_5; diff --git a/tests/queries/0_stateless/02421_new_type_json_async_insert.reference b/tests/queries/0_stateless/02421_new_type_json_async_insert.reference new file mode 100644 index 00000000000..fdd133460c6 --- /dev/null +++ b/tests/queries/0_stateless/02421_new_type_json_async_insert.reference @@ -0,0 +1,5 @@ +INCORRECT_DATA +0 +0 +INCORRECT_DATA +aaa diff --git a/tests/queries/0_stateless/02421_new_type_json_async_insert.sh b/tests/queries/0_stateless/02421_new_type_json_async_insert.sh new file mode 100755 index 00000000000..b23470a4179 --- /dev/null +++ b/tests/queries/0_stateless/02421_new_type_json_async_insert.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_async_insert" +$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 -q "CREATE TABLE t_json_async_insert (data JSON) ENGINE = MergeTree ORDER BY tuple()" + +$CLICKHOUSE_CLIENT --async_insert=1 --wait_for_async_insert=1 -q 'INSERT INTO t_json_async_insert FORMAT JSONAsObject {"aaa"}' 2>&1 | grep -o -m1 "INCORRECT_DATA" +$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_json_async_insert" +$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.parts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_json_async_insert'" + +$CLICKHOUSE_CLIENT --async_insert=1 --wait_for_async_insert=1 -q 'INSERT INTO t_json_async_insert FORMAT JSONAsObject {"aaa"}' 2>&1 | grep -o -m1 "INCORRECT_DATA" & +$CLICKHOUSE_CLIENT --async_insert=1 --wait_for_async_insert=1 -q 'INSERT INTO t_json_async_insert FORMAT JSONAsObject {"k1": "aaa"}' & + +wait + +$CLICKHOUSE_CLIENT -q "SELECT data.k1 FROM t_json_async_insert ORDER BY data.k1" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_async_insert" diff --git a/tests/queries/0_stateless/02421_new_type_json_empty_parts.reference b/tests/queries/0_stateless/02421_new_type_json_empty_parts.reference new file mode 100644 index 00000000000..172ea2d3eed --- /dev/null +++ b/tests/queries/0_stateless/02421_new_type_json_empty_parts.reference @@ -0,0 +1,16 @@ +Collapsing +0 +0 +DELETE all +2 +1 +('k1','String') +('k2','String') +0 +0 +TTL +1 +1 +('k2','String') +0 +0 diff --git a/tests/queries/0_stateless/02421_new_type_json_empty_parts.sh b/tests/queries/0_stateless/02421_new_type_json_empty_parts.sh new file mode 100755 index 00000000000..2714b9586f8 --- /dev/null +++ b/tests/queries/0_stateless/02421_new_type_json_empty_parts.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +set -euo pipefail + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./parts.lib +. "$CURDIR"/parts.lib + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT 'Collapsing';" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, s Int8, data JSON) ENGINE = CollapsingMergeTree(s) ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, 1, '{\"k1\": \"aaa\"}') (1, -1, '{\"k2\": \"bbb\"}');" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_empty_parts ORDER BY path" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT 'DELETE all';" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, data JSON) ENGINE = MergeTree ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, '{\"k1\": \"aaa\"}') (1, '{\"k2\": \"bbb\"}');" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_empty_parts ORDER BY path" +${CLICKHOUSE_CLIENT} -q "ALTER TABLE t_json_empty_parts DELETE WHERE 1 SETTINGS mutations_sync = 1;" +timeout 60 bash -c 'wait_for_delete_empty_parts t_json_empty_parts' +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_empty_parts ORDER BY path" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT 'TTL';" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, d Date, data JSON) ENGINE = MergeTree ORDER BY id TTL d WHERE id % 2 = 1 SETTINGS old_parts_lifetime=5;" --allow_experimental_json_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, '2000-01-01', '{\"k1\": \"aaa\"}') (2, '2000-01-01', '{\"k2\": \"bbb\"}');" +${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE t_json_empty_parts FINAL;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_empty_parts ORDER BY path" +${CLICKHOUSE_CLIENT} -q "ALTER TABLE t_json_empty_parts MODIFY TTL d;" +${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE t_json_empty_parts FINAL;" +timeout 60 bash -c 'wait_for_delete_empty_parts t_json_empty_parts' +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM t_json_empty_parts ORDER BY path" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" diff --git a/tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.reference b/tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.reference new file mode 100644 index 00000000000..3eb1f72bfd6 --- /dev/null +++ b/tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.reference @@ -0,0 +1 @@ +{"list":[{"nested":{"x":[{"r":"1"},{"r":"2"}]},"x":[{"r":"1"}]}]} diff --git a/tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.sh b/tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.sh new file mode 100755 index 00000000000..ae98946ad73 --- /dev/null +++ b/tests/queries/0_stateless/02482_new_json_nested_arrays_with_same_keys.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo ' +{ + "obj" : + { + "list" : + [ + { + "nested" : { + "x" : [{"r" : 1}, {"r" : 2}] + }, + "x" : [{"r" : 1}] + } + ] + } +}' > 02482_object_data.jsonl + +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select * from file(02482_object_data.jsonl, auto, 'obj JSON')" + +rm 02482_object_data.jsonl + diff --git a/tests/queries/0_stateless/02553_new_type_json_attach_partition.reference b/tests/queries/0_stateless/02553_new_type_json_attach_partition.reference new file mode 100644 index 00000000000..1556b015503 --- /dev/null +++ b/tests/queries/0_stateless/02553_new_type_json_attach_partition.reference @@ -0,0 +1,2 @@ +{"b":"1","c":{"k1":"1"}} +{"b":"1","c":{"k1":["1","2"]}} diff --git a/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql b/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql new file mode 100644 index 00000000000..8a5abd31cb4 --- /dev/null +++ b/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql @@ -0,0 +1,14 @@ +SET allow_experimental_json_type = 1; + +DROP TABLE IF EXISTS t_json_attach_partition; + +CREATE TABLE t_json_attach_partition(b UInt64, c JSON) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": 1}}; +ALTER TABLE t_json_attach_partition DETACH PARTITION tuple(); +INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": [1, 2]}}; + +ALTER TABLE t_json_attach_partition ATTACH PARTITION tuple(); +SELECT * FROM t_json_attach_partition ORDER BY toString(c) FORMAT JSONEachRow; + +DROP TABLE t_json_attach_partition; diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1.reference deleted file mode 100644 index 90a96b1bb2f..00000000000 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1.reference +++ /dev/null @@ -1,2080 +0,0 @@ -Memory -insert -test -('a.b.c','UInt32') -('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') -('a.b.d','Int64') -('a.b.e','String') -('b.b._25','Int64') -('b.b._26','Int64') -('b.b._27','Int64') -('b.b._28','Int64') -('b.b._29','Int64') -('b.b.d','Int64') -('b.b.e','String') -('d.a','Array(Nullable(Int64))') -('d.a','Int64') -('d.b','Int64') -('d.c','Date') -{ - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json.non.existing.path": [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] -} -{ - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -MergeTree compact -No merges -insert -test -('a.b.c','UInt32') -('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') -('a.b.d','Int64') -('a.b.e','String') -('b.b._25','Int64') -('b.b._26','Int64') -('b.b._27','Int64') -('b.b._28','Int64') -('b.b._29','Int64') -('b.b.d','Int64') -('b.b.e','String') -('d.a','Array(Nullable(Int64))') -('d.a','Int64') -('d.b','Int64') -('d.c','Date') -{ - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json.non.existing.path": [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] -} -{ - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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)') -('a.b.d','Int64') -('a.b.e','String') -('b.b._25','Int64') -('b.b._26','Int64') -('b.b._27','Int64') -('b.b._28','Int64') -('b.b._29','Int64') -('b.b.d','Int64') -('b.b.e','String') -('d.a','Array(Nullable(Int64))') -('d.a','Int64') -('d.b','Int64') -('d.c','Date') -{ - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json.non.existing.path": [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] -} -{ - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -MergeTree wide -No merges -insert -test -('a.b.c','UInt32') -('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') -('a.b.d','Int64') -('a.b.e','String') -('b.b._25','Int64') -('b.b._26','Int64') -('b.b._27','Int64') -('b.b._28','Int64') -('b.b._29','Int64') -('b.b.d','Int64') -('b.b.e','String') -('d.a','Array(Nullable(Int64))') -('d.a','Int64') -('d.b','Int64') -('d.c','Date') -{ - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json.non.existing.path": [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] -} -{ - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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)') -('a.b.d','Int64') -('a.b.e','String') -('b.b._25','Int64') -('b.b._26','Int64') -('b.b._27','Int64') -('b.b._28','Int64') -('b.b._29','Int64') -('b.b.d','Int64') -('b.b.e','String') -('d.a','Array(Nullable(Int64))') -('d.a','Int64') -('d.b','Int64') -('d.c','Date') -{ - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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.:`UUID`": [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], - "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], - "json.a.b.e.:`UUID`": [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], - "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._25.:`UUID`": [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], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`UUID`": [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], - "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._27.:`UUID`": [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], - "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._28.:`UUID`": [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], - "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], - "json.b.b._29.:`UUID`": [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], - "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.d.:`UUID`": [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], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`UUID`": [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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`UUID`": [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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`UUID`": [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], - "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], - "json.d.c.:`UUID`": [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], - "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], - "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] -} -{ - "json.non.existing.path": [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] -} -{ - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.non.existing.path": [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], - "json.non.existing.path.:`Int64`": [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] -} -{ - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b.e.:`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], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.b": [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, "35", "36", "37", "38", "39"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.d.b": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], - "json.d.b.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b._26.:`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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], - "json.d.a.:`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], - "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.b.b": [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], - "json.b.b._26.:`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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "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] -} -{ - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], - "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] -} diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference new file mode 100644 index 00000000000..16150ee7b45 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference @@ -0,0 +1,831 @@ +No merges +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh similarity index 95% rename from tests/queries/0_stateless/03207_json_read_subcolumns_1.sh rename to tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh index 3c63b513ad6..75b13f89a06 100755 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1.sh +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh @@ -96,13 +96,6 @@ function test() $CH_CLIENT -q "drop table if exists test;" -echo "Memory" -$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;" - -echo "MergeTree compact" $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" @@ -112,14 +105,3 @@ 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=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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference new file mode 100644 index 00000000000..a7361856bc1 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference @@ -0,0 +1,413 @@ +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql new file mode 100755 index 00000000000..51e6970759d --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql @@ -0,0 +1,87 @@ +-- Tags: no-fasttest, long +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(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); + +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; + +drop table test; diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference new file mode 100644 index 00000000000..16150ee7b45 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference @@ -0,0 +1,831 @@ +No merges +insert +test +('a.b.c','UInt32') +('a.b.d','Array(Nullable(String))') +('a.b.d','DateTime64(9)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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)') +('a.b.d','Int64') +('a.b.e','String') +('b.b._25','Int64') +('b.b._26','Int64') +('b.b._27','Int64') +('b.b._28','Int64') +('b.b._29','Int64') +('b.b.d','Int64') +('b.b.e','String') +('d.a','Array(Nullable(Int64))') +('d.a','Int64') +('d.b','Int64') +('d.c','Date') +{ + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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.:`UUID`": [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], + "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], + "json.a.b.e.:`UUID`": [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], + "json.b.b._25": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`Int64`": [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, "25", null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._25.:`UUID`": [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], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`UUID`": [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], + "json.b.b._27": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`Int64`": [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, "27", null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._27.:`UUID`": [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], + "json.b.b._28": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`Int64`": [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, "28", null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._28.:`UUID`": [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], + "json.b.b._29": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`Int64`": [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, "29", null, null, null, null, null, null, null, null, null, null], + "json.b.b._29.:`UUID`": [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], + "json.b.b.d": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "15", "16", "17", "18", "19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.d.:`UUID`": [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], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`UUID`": [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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`UUID`": [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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`UUID`": [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], + "json.d.c": [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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], + "json.d.c.:`UUID`": [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], + "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], + "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] +} +{ + "json.non.existing.path": [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] +} +{ + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.non.existing.path": [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], + "json.non.existing.path.:`Int64`": [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] +} +{ + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b.e.:`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], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.b": [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, "35", "36", "37", "38", "39"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.d.b": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], + "json.d.b.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b._26.:`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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], + "json.d.a.:`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], + "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], + "json.b.b": [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], + "json.b.b._26.:`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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "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] +} +{ + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "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] +} diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh new file mode 100755 index 00000000000..144a88d8ec6 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh @@ -0,0 +1,107 @@ +#!/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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh index 128c525bf02..6df95ad4ad6 100755 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# 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 diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh index bf6360a89a4..154c1aa5f9c 100755 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# 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 diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh index 5e8af0c1f80..d20d9a4d79d 100755 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# 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 diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh index 8fa1b809390..c39411f7b09 100755 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# 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 diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh index f7c0fdad1a6..b179baeadef 100755 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# 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 diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh index 3106702faf4..6686179e801 100755 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# 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 From a7bf79b30833d9cb8fe9a20cab7025b3db95bef7 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 19:27:39 +0000 Subject: [PATCH 0349/1722] Update docs --- docs/en/sql-reference/data-types/json.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index c6fa6f0e882..8e16d6e4d57 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -80,4 +80,5 @@ SELECT * FROM json FORMAT JSONEachRow ## Related Content - [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) -- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) \ No newline at end of file +- [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) +- \ No newline at end of file From 9d3a5cb699d9933b712c970c690b4ba33cf9b133 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 Jul 2024 21:27:04 +0000 Subject: [PATCH 0350/1722] Fix style --- tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.sql old mode 100755 new mode 100644 From 4978869d2f709a9ad93cefc04cda43bcc739fb22 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 23 Jul 2024 18:04:38 +0800 Subject: [PATCH 0351/1722] stash --- src/Functions/FunctionOverlay.cpp | 481 ++++++++++++++++++++++++++++++ 1 file changed, 481 insertions(+) create mode 100644 src/Functions/FunctionOverlay.cpp diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp new file mode 100644 index 00000000000..6160335ad79 --- /dev/null +++ b/src/Functions/FunctionOverlay.cpp @@ -0,0 +1,481 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ZERO_ARRAY_OR_TUPLE_INDEX; +} + +namespace +{ + +/// If 'is_utf8' - measure offset and length in code points instead of bytes. +/// Syntax: overlay(input, replace, offset[, length]) +template +class FunctionOverlay : public IFunction +{ +public: + static constexpr auto name = is_utf8 ? "OverlayUTF8" : "Overlay"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const size_t number_of_arguments = arguments.size(); + if (number_of_arguments < 3 || number_of_arguments > 4) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: " + "passed {}, should be 3 or 4", + getName(), + number_of_arguments); + + /// first argument is string + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected String", + arguments[0]->getName(), + getName()); + + /// second argument is string + if (!isString(arguments[1])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, expected String", + arguments[1]->getName(), + getName()); + + if (!isNativeNumber(arguments[2])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {}, expected (U)Int8|16|32|64", + arguments[2]->getName(), + getName()); + + if (number_of_arguments == 4 && !isNativeNumber(arguments[3])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, expected (U)Int8|16|32|64", + arguments[3]->getName(), + getName()); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const size_t number_of_arguments = arguments.size(); + + ColumnPtr column_string = arguments[0].column; + ColumnPtr column_offset = arguments[1].column; + ColumnPtr column_length; + if (number_of_arguments == 3) + column_length = arguments[2].column; + + const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); + const ColumnConst * column_length_const = nullptr; + if (number_of_arguments == 3) + column_length_const = checkAndGetColumn(column_length.get()); + + Int64 offset = 0; + Int64 length = 0; + if (column_offset_const) + offset = column_offset_const->getInt(0); + if (column_length_const) + length = column_length_const->getInt(0); + + auto res_col = ColumnString::create(); + auto & res_data = res_col->getChars(); + auto & res_offsets = res_col->getOffsets(); + } + +private: +template + void constantConstant( + size_t rows, + const StringRef & input, + const StringRef & replace, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + constantConstant(input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); + return; + } + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + size_t replace_size = replace.size; + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + else if constexpr (three_args) + { + valid_length = replace_size; + } + + size_t res_offset = 0; + size_t input_size = input.size; + for (size_t i = 0; i < rows; ++i) + { + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (!three_args && !length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } + + template + void vectorConstant( + const ColumnString::Chars & input_data, + const ColumnString::Offsets & input_offsets, + const StringRef & replace, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + vectorConstant(input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); + return; + } + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + size_t replace_size = replace.size; + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + else if constexpr (three_args) + { + valid_length = replace_size; + } + + size_t rows = input_offsets.size(); + size_t res_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + size_t input_offset = input_offsets[i - 1]; + size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (!three_args && !length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } + + template + void constantVector( + const StringRef & input, + const ColumnString::Chars & replace_data, + const ColumnString::Offsets & replace_offsets, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + constantVector(input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); + return; + } + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + + size_t rows = replace_offsets.size(); + size_t input_size = input.size; + size_t res_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + size_t replace_offset = replace_offsets[i - 1]; + size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (three_args) + { + // length = replace_size; + valid_length = replace_size; + } + else if constexpr (!length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } + + template + void vectorVector( + const ColumnString::Chars & input_data, + const ColumnString::Offsets & input_offsets, + const ColumnString::Chars & replace_data, + const ColumnString::Offsets & replace_offsets, + const ColumnPtr & column_offset, + const ColumnPtr & column_length, + Int64 const_offset, + Int64 const_length, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + if (!three_args && length_is_const && const_length < 0) + { + vectorVector( + input_data, + input_offsets, + replace_data, + replace_offsets, + column_offset, + column_length, + const_offset, + -1, + res_data, + res_offsets); + return; + } + + + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative + if constexpr (offset_is_const) + { + offset = const_offset; + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + Int64 length = 0; // maybe negative + size_t valid_length = 0; // not negative + if constexpr (!three_args && length_is_const) + { + assert(const_length >= 0); + valid_length = const_length; + } + + size_t rows = input_offsets.size(); + size_t res_offset = 0; + for (size_t i = 0; i < rows; ++i) + { + size_t input_offset = input_offsets[i - 1]; + size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + size_t replace_offset = replace_offsets[i - 1]; + size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + + if constexpr (!offset_is_const) + { + offset = column_offset->getInt(i); + valid_offset = offset > 0 ? (offset - 1) : (-offset); + } + + if constexpr (three_args) + { + // length = replace_size; + valid_length = replace_size; + } + else if constexpr (!length_is_const) + { + length = column_length->getInt(i); + valid_length = length >= 0 ? length : replace_size; + } + + size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + + /// add zero terminator + res_data[res_offset] = 0; + ++res_offset; + + res_offsets[i] = res_offset; + } + } +}; + +} + +REGISTER_FUNCTION(Overlay) +{ + factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::CaseSensitive); +} + +} From bc7fd604b7d6489733ed362a9009f91428c064e7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 10:10:15 +0000 Subject: [PATCH 0352/1722] Fix tests, builds and docs --- docs/en/operations/settings/settings.md | 4 +- docs/en/sql-reference/data-types/index.md | 3 +- .../sql-reference/functions/json-functions.md | 28 +++++----- src/Columns/ColumnObject.cpp | 2 +- .../Serializations/SerializationObject.cpp | 2 +- .../test_distributed_type_object/test.py | 2 +- .../0_stateless/01825_new_type_json_13.sh | 2 +- .../0_stateless/01825_new_type_json_ghdata.sh | 2 +- .../01825_new_type_json_in_array.sql | 1 + .../01825_new_type_json_in_other_types.sh | 2 +- .../01825_new_type_json_nbagames.sh | 2 +- .../queries/0_stateless/03205_json_syntax.sql | 51 ++++++++++++------- .../03209_json_type_horizontal_merges.sh | 2 +- .../03209_json_type_vertical_merges.sh | 2 +- .../0_stateless/03211_nested_json_merges.sh | 2 +- 15 files changed, 63 insertions(+), 44 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 720cf1ce959..e8957a24de0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5611,13 +5611,13 @@ Default value: `1GiB`. ## use_json_alias_for_old_object_type -When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/object-json.md) type instead of the new [JSON](../../sql-reference/data-types/json.md) type. +When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/json.md) type. This setting requires server restart to take effect when changed. Default value: `false`. ## type_json_skip_duplicated_paths -When enabled, ClickHouse will skip duplicated paths during parsing of [JSON](../../sql-reference/data-types/json.md) object. Only the value of the first occurrence of each path will be inserted. +When enabled, ClickHouse will skip duplicated paths during parsing of [JSON](../../sql-reference/data-types/newjson.md) object. Only the value of the first occurrence of each path will be inserted. Default value: `false` diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index fcb0b60d022..2b89dd145e6 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -19,7 +19,8 @@ ClickHouse data types include: - **Boolean**: ClickHouse has a [`Boolean` type](./boolean.md) - **Strings**: [`String`](./string.md) and [`FixedString`](./fixedstring.md) - **Dates**: use [`Date`](./date.md) and [`Date32`](./date32.md) for days, and [`DateTime`](./datetime.md) and [`DateTime64`](./datetime64.md) for instances in time -- **JSON**: the [`JSON` object](./json.md) stores a JSON document in a single column +- **Object**: the [`Object`](./json.md) stores a JSON document in a single column (deprecated) +- **JSON**: the [`JSON` object](./newjson.md) stores a JSON document in a single column - **UUID**: a performant option for storing [`UUID` values](./uuid.md) - **Low cardinality types**: use an [`Enum`](./enum.md) when you have a handful of unique values, or use [`LowCardinality`](./lowcardinality.md) when you have up to 10,000 unique values of a column - **Arrays**: any column can be defined as an [`Array` of values](./array.md) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index b02dc3d1d8a..fd1f924c18f 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -1158,7 +1158,7 @@ SELECT jsonMergePatch('{"a":1}', '{"name": "joey"}', '{"name": "tom"}', '{"name" ### JSONAllPaths -Returns the list of all paths stored in each row in [JSON](../data-types/json.md) column. +Returns the list of all paths stored in each row in [JSON](../data-types/newjson.md) column. **Syntax** @@ -1168,7 +1168,7 @@ JSONAllPaths(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** @@ -1192,7 +1192,7 @@ SELECT json, JSONAllPaths(json) FROM test; ### JSONAllPathsWithTypes -Returns the map of all paths and their data types stored in each row in [JSON](../data-types/json.md) column. +Returns the map of all paths and their data types stored in each row in [JSON](../data-types/newjson.md) column. **Syntax** @@ -1202,7 +1202,7 @@ JSONAllPathsWithTypes(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** @@ -1226,7 +1226,7 @@ SELECT json, JSONAllPathsWithTypes(json) FROM test; ### JSONDynamicPaths -Returns the list of dynamic paths that are stored as separate subcolumns in [JSON](../data-types/json.md) column. +Returns the list of dynamic paths that are stored as separate subcolumns in [JSON](../data-types/newjson.md) column. **Syntax** @@ -1236,7 +1236,7 @@ JSONDynamicPaths(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** @@ -1260,7 +1260,7 @@ SELECT json, JSONDynamicPaths(json) FROM test; ### JSONDynamicPathsWithTypes -Returns the map of dynamic paths that are stored as separate subcolumns and their types in each row in [JSON](../data-types/json.md) column. +Returns the map of dynamic paths that are stored as separate subcolumns and their types in each row in [JSON](../data-types/newjson.md) column. **Syntax** @@ -1270,7 +1270,7 @@ JSONAllPathsWithTypes(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** @@ -1294,7 +1294,7 @@ SELECT json, JSONDynamicPathsWithTypes(json) FROM test; ### JSONSharedDataPaths -Returns the list of paths that are stored in shared data structure in [JSON](../data-types/json.md) column. +Returns the list of paths that are stored in shared data structure in [JSON](../data-types/newjson.md) column. **Syntax** @@ -1304,7 +1304,7 @@ JSONSharedDataPaths(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** @@ -1328,7 +1328,7 @@ SELECT json, JSONSharedDataPaths(json) FROM test; ### JSONSharedDataPathsWithTypes -Returns the map of paths that are stored in shared data structure and their types in each row in [JSON](../data-types/json.md) column. +Returns the map of paths that are stored in shared data structure and their types in each row in [JSON](../data-types/newjson.md) column. **Syntax** @@ -1338,7 +1338,7 @@ JSONSharedDataPathsWithTypes(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** @@ -1362,7 +1362,7 @@ SELECT json, JSONSharedDataPathsWithTypes(json) FROM test; ### JSONEmpty -Checks whether the input [JSON](../data-types/json.md) object is empty. +Checks whether the input [JSON](../data-types/newjson.md) object is empty. ``` sql JSONEmpty(json) @@ -1370,7 +1370,7 @@ JSONEmpty(json) **Arguments** -- `json` — [JSON](../data-types/json.md). +- `json` — [JSON](../data-types/newjson.md). **Returned value** diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 65aacf2b539..47eb517e81b 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1052,7 +1052,7 @@ bool ColumnObject::isFinalized() const void ColumnObject::getExtremes(DB::Field & min, DB::Field & max) const { - if (size() == 0) + if (empty()) { min = Object(); max = Object(); diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 7c8c23e8a29..305ebfa2e16 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -466,7 +466,7 @@ void SerializationObject::deserializeBinaryBulkWithMultipleStreams( void SerializationObject::serializeBinary(const Field & field, WriteBuffer & ostr, const DB::FormatSettings & settings) const { - auto & object = field.get(); + const auto & object = field.get(); /// Serialize number of paths and then pairs (path, value). writeVarUInt(object.size(), ostr); for (const auto & [path, value] : object) diff --git a/tests/integration/test_distributed_type_object/test.py b/tests/integration/test_distributed_type_object/test.py index 360087c9dda..ed27962f075 100644 --- a/tests/integration/test_distributed_type_object/test.py +++ b/tests/integration/test_distributed_type_object/test.py @@ -16,7 +16,7 @@ def started_cluster(): for node in (node1, node2): node.query( - "CREATE TABLE local_table(id UInt32, data JSON) ENGINE = MergeTree ORDER BY id", + "CREATE TABLE local_table(id UInt32, data Object('json')) ENGINE = MergeTree ORDER BY id", settings={"allow_experimental_object_type": 1}, ) node.query( diff --git a/tests/queries/0_stateless/01825_new_type_json_13.sh b/tests/queries/0_stateless/01825_new_type_json_13.sh index 765b62b1f4a..316e6890d5e 100755 --- a/tests/queries/0_stateless/01825_new_type_json_13.sh +++ b/tests/queries/0_stateless/01825_new_type_json_13.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_13" -$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_13 (obj JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_json_type 1 +$CLICKHOUSE_CLIENT -q "CREATE TABLE t_json_13 (obj JSON) ENGINE = MergeTree ORDER BY tuple() settings replace_long_file_name_to_hash=1" --allow_experimental_json_type 1 cat < Date: Tue, 23 Jul 2024 10:27:12 +0000 Subject: [PATCH 0353/1722] Add comments about fictitious subcolumns --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeReaderWide.cpp | 2 ++ src/Storages/MergeTree/checkDataPart.cpp | 1 + 3 files changed, 7 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 470ab977719..d507468f3ea 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -132,6 +132,7 @@ void MergeTreeDataPartWriterWide::addStreams( { assert(!substream_path.empty()); + /// Don't create streams for fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return; @@ -208,6 +209,7 @@ ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGett { return [&, this] (const ISerialization::SubstreamPath & substream_path) -> WriteBuffer * { + /// Skip fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return nullptr; @@ -373,6 +375,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( min_compress_block_size = settings.min_compress_block_size; getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { + /// Skip fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return; @@ -414,6 +417,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { + /// Skip fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 2c876fade74..9ab45ec7b56 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -213,6 +213,7 @@ void MergeTreeReaderWide::addStreams( ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) { + /// Don't create streams for fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return; @@ -351,6 +352,7 @@ void MergeTreeReaderWide::prefetchForColumn( deserializePrefix(serialization, name_and_type, current_task_last_mark, cache, deserialize_states_cache); auto callback = [&](const ISerialization::SubstreamPath & substream_path) { + /// Skip fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index dcdcb7f5800..dba64edd00d 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -211,6 +211,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( { get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { + /// Skip fictitious subcolumns that don't store any real data. if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) return; From 83ef3bf32e16deb946047ddf0de6ee4defd77f93 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 10:27:56 +0000 Subject: [PATCH 0354/1722] Fix link in docs --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e8957a24de0..8366351da59 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5611,7 +5611,7 @@ Default value: `1GiB`. ## use_json_alias_for_old_object_type -When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/json.md) type. +When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/newjson.md) type. This setting requires server restart to take effect when changed. Default value: `false`. From 383221b04d57948364514106b2f48b03140bfbd0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Jul 2024 10:36:08 +0000 Subject: [PATCH 0355/1722] Analyzer: Storage Join support joining with nullable columns --- src/Planner/PlannerJoins.cpp | 8 +- ...multiple_joins_with_storage_join.reference | 23 +++++ ...03208_multiple_joins_with_storage_join.sql | 84 +++++++++++++++++++ 3 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference create mode 100644 tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index d9360a58240..04011233ca1 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -494,6 +494,12 @@ JoinClausesAndActions buildJoinClausesAndActions( necessary_names.push_back(name); }; + bool is_join_with_special_storage = false; + if (const auto * right_table_node = join_node.getRightTableExpression()->as()) + { + is_join_with_special_storage = dynamic_cast(right_table_node->getStorage().get()); + } + for (auto & join_clause : result.join_clauses) { const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); @@ -561,7 +567,7 @@ JoinClausesAndActions buildJoinClausesAndActions( if (!left_key_node->result_type->equals(*common_type)) left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); - if (!right_key_node->result_type->equals(*common_type)) + if (!is_join_with_special_storage && !right_key_node->result_type->equals(*common_type)) right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); } diff --git a/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference new file mode 100644 index 00000000000..5ebdaa1c81f --- /dev/null +++ b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference @@ -0,0 +1,23 @@ +----- +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +----- +\N \N \N 0 3 B +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +----- +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +\N \N \N \N 3 B \N \N +\N \N \N \N \N \N 3 B +----- +\N \N \N 3 3 B 0 0 +\N \N \N 0 0 3 3 B +1 1 1 a 1 1 A 1 1 A +2 2 2 b 2 2 B 2 2 B +----- +3 3 \N B B +1 1 1 a A A +2 2 2 b B B +----- +7 diff --git a/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql new file mode 100644 index 00000000000..83be4c3f1d2 --- /dev/null +++ b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql @@ -0,0 +1,84 @@ +#!/usr/bin/env -S ${HOME}/clickhouse-client --queries-file + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab ( `k` Nullable(UInt32), `k1` Nullable(UInt32), `k2` Nullable(UInt32), `v` String ) ENGINE = Memory; +INSERT INTO tab VALUES (1, 1, 1, 'a'), (2, 2, 2, 'b'); + +DROP TABLE IF EXISTS mem; +CREATE TABLE mem ( `k` UInt64, `v` String ) ENGINE = Join(ANY, LEFT, k); +INSERT INTO mem VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem2; +CREATE TABLE mem2 ( `k` UInt64, `v` String ) ENGINE = Join(ANY, RIGHT, k); +INSERT INTO mem2 VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem3; +CREATE TABLE mem3 ( `k` UInt64, `v` String ) ENGINE = Join(ALL, FULL, k) SETTINGS join_use_nulls = 1; +INSERT INTO mem3 VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem4; +CREATE TABLE mem4 ( `k1` UInt64, `k2` UInt64, `v` String ) ENGINE = Join(ALL, FULL, k1, k2); +INSERT INTO mem4 VALUES (1, 1, 'A'), (2, 2, 'B'), (3, 3, 'B'); + +SET allow_experimental_analyzer = 1; + +SELECT '-----'; + +SELECT * +FROM tab +ANY LEFT JOIN mem ON k1 = mem.k +ANY LEFT JOIN mem AS t ON k2 = t.k +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +ANY LEFT JOIN mem ON k1 = mem.k +ANY RIGHT JOIN mem2 ON k2 = mem2.k +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem3 AS t1 ON k1 = t1.k +FULL JOIN mem3 AS t2 ON k2 = t2.k +ORDER BY tab.v +SETTINGS join_use_nulls = 1 +; +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem4 AS t1 ON tab.k1 = t1.k1 AND tab.k2 = t1.k2 +FULL JOIN mem4 AS t2 ON tab.k1 = t2.k1 AND tab.k2 = t2.k2 +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem4 AS t1 USING (k1, k2) +FULL JOIN mem4 AS t2 USING (k1, k2) +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT count() FROM ( + EXPLAIN PLAN + SELECT * FROM tab + ANY LEFT JOIN mem AS t1 ON tab.k = t1.k + ANY LEFT JOIN mem AS t2 ON tab.k = t2.k + ANY LEFT JOIN mem AS t3 ON tab.k = t3.k + ANY LEFT JOIN mem AS t4 ON tab.k = t4.k + ANY RIGHT JOIN mem2 AS t5 ON tab.k = t5.k + ANY LEFT JOIN mem AS t6 ON tab.k = t6.k + ANY LEFT JOIN mem AS t7 ON tab.k = t7.k +) +WHERE explain like '%FilledJoin%' +; From 0631ff93778801768edde14c5fd5ed22701f6ec3 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Tue, 23 Jul 2024 12:14:54 +0100 Subject: [PATCH 0356/1722] Add jsoncompactwithprogress to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3967179f078..2e403f44e4e 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1855,6 +1855,7 @@ jsoncompactstrings jsoncompactstringseachrow jsoncompactstringseachrowwithnames jsoncompactstringseachrowwithnamesandtypes +jsoncompactwithprogress jsoneachrow jsoneachrowwithprogress jsonobjecteachrow From 53a55221845802e67f9586af615fcb5ed84b20eb Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Tue, 23 Jul 2024 12:19:06 +0100 Subject: [PATCH 0357/1722] Fix example output in the documentation --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 6161cddefdd..8795b71e0ac 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -935,7 +935,7 @@ Each row is either a metadata object, data object, progress information or stati Example: ```json -{"meta": [{"name":"id", "type":"UInt32"}, {"name":"name", "type":"String"}]}} +{"meta": [{"name":"id", "type":"UInt32"}, {"name":"name", "type":"String"}]} {"progress":{"read_rows":"8","read_bytes":"168","written_rows":"0","written_bytes":"0","total_rows_to_read":"2","result_rows":"0","result_bytes":"0","elapsed_ns":"0"}} {"data":["1", "John Doe"]} {"data":["2", "Joe Doe"]} From a4a0611c1c0b40ea6b99d7e5cf3972fda201bb9d Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Tue, 23 Jul 2024 12:29:40 +0100 Subject: [PATCH 0358/1722] Fix docs spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2e403f44e4e..c64cfcb0968 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -406,6 +406,7 @@ JSONCompactStrings JSONCompactStringsEachRow JSONCompactStringsEachRowWithNames JSONCompactStringsEachRowWithNamesAndTypes +JSONCompactWithProgress JSONEachRow JSONEachRowWithProgress JSONExtract From 81688e0efdf75a6a3923d6b95f09579d37e93e2a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 23 Jul 2024 20:29:35 +0800 Subject: [PATCH 0359/1722] almost finish --- src/Functions/FunctionOverlay.cpp | 154 ++++++++++++++++++++++++++---- 1 file changed, 134 insertions(+), 20 deletions(-) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index 6160335ad79..65af4d811f5 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -15,10 +15,8 @@ namespace DB namespace ErrorCodes { -extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int ZERO_ARRAY_OR_TUPLE_INDEX; } namespace @@ -86,32 +84,145 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const size_t number_of_arguments = arguments.size(); + bool three_args = number_of_arguments == 3; - ColumnPtr column_string = arguments[0].column; - ColumnPtr column_offset = arguments[1].column; + ColumnPtr column_offset = arguments[2].column; ColumnPtr column_length; - if (number_of_arguments == 3) - column_length = arguments[2].column; + if (!three_args) + column_length = arguments[3].column; const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); const ColumnConst * column_length_const = nullptr; - if (number_of_arguments == 3) + if (!three_args) column_length_const = checkAndGetColumn(column_length.get()); - Int64 offset = 0; - Int64 length = 0; + bool offset_is_const = false; + bool length_is_const = false; + Int64 offset = -1; + Int64 length = -1; if (column_offset_const) + { offset = column_offset_const->getInt(0); + offset_is_const = true; + } + if (column_length_const) + { length = column_length_const->getInt(0); + length_is_const = true; + } + auto res_col = ColumnString::create(); auto & res_data = res_col->getChars(); auto & res_offsets = res_col->getOffsets(); + res_offsets.resize_exact(input_rows_count); + + ColumnPtr column_input = arguments[0].column; + ColumnPtr column_replace = arguments[1].column; + + const auto * column_input_const = checkAndGetColumn(column_input.get()); + const auto * column_input_string = checkAndGetColumn(column_input.get()); + if (column_input_const) + { + StringRef input = column_input_const->getDataAt(0); + res_data.reserve(input.size * input_rows_count); + } + else + { + res_data.reserve(column_input_string->getChars().size()); + } + + const auto * column_replace_const = checkAndGetColumn(column_replace.get()); + const auto * column_replace_string = checkAndGetColumn(column_replace.get()); + bool input_is_const = column_input_const != nullptr; + bool replace_is_const = column_replace_const != nullptr; + +#define OVERLAY_EXECUTE_CASE(THREE_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ + if (input_is_const && replace_is_const) \ + constantConstant( \ + input_rows_count, \ + column_input_const->getDataAt(0), \ + column_replace_const->getDataAt(0), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); \ + else if (input_is_const) \ + constantVector( \ + column_input_const->getDataAt(0), \ + column_replace_string->getChars(), \ + column_replace_string->getOffsets(), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); \ + else if (replace_is_const) \ + vectorConstant( \ + column_input_string->getChars(), \ + column_input_string->getOffsets(), \ + column_replace_const->getDataAt(0), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); \ + else \ + vectorVector( \ + column_input_string->getChars(), \ + column_input_string->getOffsets(), \ + column_replace_string->getChars(), \ + column_replace_string->getOffsets(), \ + column_offset, \ + column_length, \ + offset, \ + length, \ + res_data, \ + res_offsets); + + if (three_args) + { + if (offset_is_const) + { + OVERLAY_EXECUTE_CASE(true, true, false) + } + else + { + OVERLAY_EXECUTE_CASE(true, false, false) + } + } + else + { + if (offset_is_const && length_is_const) + { + OVERLAY_EXECUTE_CASE(false, true, true) + } + else if (offset_is_const && !length_is_const) + { + OVERLAY_EXECUTE_CASE(false, true, false) + } + else if (!offset_is_const && length_is_const) + { + OVERLAY_EXECUTE_CASE(false, false, true) + } + else + { + OVERLAY_EXECUTE_CASE(false, false, false) + } + } +#undef OVERLAY_EXECUTE_CASE + + return res_col; } + private: -template + template void constantConstant( size_t rows, const StringRef & input, @@ -121,11 +232,12 @@ template ( + rows, input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -194,7 +306,7 @@ template + template void vectorConstant( const ColumnString::Chars & input_data, const ColumnString::Offsets & input_offsets, @@ -204,11 +316,12 @@ template ( + input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -281,7 +394,7 @@ template + template void constantVector( const StringRef & input, const ColumnString::Chars & replace_data, @@ -291,11 +404,12 @@ template ( + input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -379,11 +493,11 @@ template ( + vectorVector( input_data, input_offsets, replace_data, From fce9d607309002d0b8043856dad6905676a2c412 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 12:48:06 +0000 Subject: [PATCH 0360/1722] Try to fix docs --- docs/en/sql-reference/data-types/newjson.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 494bfba3173..524dc7810e6 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/data-types/json +slug: /en/sql-reference/data-types/newjson sidebar_position: 63 sidebar_label: JSON keywords: [json, data type] From 9785f85ca32e3af5760a9bc24e998e7d050fc073 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 23 Jul 2024 21:08:02 +0800 Subject: [PATCH 0361/1722] fix style --- src/Functions/FunctionOverlay.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index 65af4d811f5..7d0e2e86de2 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -1,14 +1,10 @@ #include -#include #include -#include -#include #include #include #include #include -#include - +#include namespace DB { From 3cb35efb3149d0c0cef79ecece53b1995be0e0e9 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 13:31:59 +0000 Subject: [PATCH 0362/1722] Try to improve insert speed --- src/Columns/ColumnArray.cpp | 5 + src/Columns/ColumnArray.h | 1 + src/Columns/ColumnDecimal.h | 1 + src/Columns/ColumnDynamic.cpp | 91 +++++++++------- src/Columns/ColumnDynamic.h | 79 ++++++++------ src/Columns/ColumnFixedString.h | 5 + src/Columns/ColumnLowCardinality.h | 2 + src/Columns/ColumnMap.cpp | 5 + src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 5 + src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnObject.cpp | 103 +++++++++++------- src/Columns/ColumnObject.h | 7 +- src/Columns/ColumnString.cpp | 5 + src/Columns/ColumnString.h | 1 + src/Columns/ColumnTuple.cpp | 8 ++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 5 + src/Columns/ColumnVariant.h | 1 + src/Columns/ColumnVector.h | 5 + src/Columns/IColumn.h | 3 + src/DataTypes/DataTypeObject.cpp | 7 +- .../Serializations/SerializationDynamic.cpp | 9 +- src/Formats/JSONExtractTree.cpp | 6 +- src/Processors/Formats/IRowInputFormat.cpp | 4 + .../Impl/JSONAsStringRowInputFormat.cpp | 63 ++++++----- .../Formats/Impl/JSONAsStringRowInputFormat.h | 14 +-- 27 files changed, 278 insertions(+), 160 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 19cce678cc7..299bba95eb6 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -452,6 +452,11 @@ void ColumnArray::reserve(size_t n) getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1. } +size_t ColumnArray::capacity() const +{ + return getOffsets().capacity(); +} + void ColumnArray::shrinkToFit() { getOffsets().shrink_to_fit(); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 63affb86d9d..906ff014f5b 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -118,6 +118,7 @@ public: void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + size_t capacity() const override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 59bfbd2159c..6c1b257871a 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -53,6 +53,7 @@ public: size_t allocatedBytes() const override { return data.allocated_bytes(); } void protect() override { data.protect(); } void reserve(size_t n) override { data.reserve_exact(n); } + size_t capacity() const override { return data.capacity(); } void shrinkToFit() override { data.shrink_to_fit(); } #if !defined(DEBUG_OR_SANITIZER_BUILD) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index bbaf2e669d8..652448f0220 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -29,11 +29,13 @@ ColumnDynamic::ColumnDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_ variant_info.variant_type = std::make_shared(DataTypes{}); variant_info.variant_name = variant_info.variant_type->getName(); variant_column = variant_info.variant_type->createColumn(); + variant_column_ptr = assert_cast(variant_column.get()); } ColumnDynamic::ColumnDynamic( MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_) : variant_column(std::move(variant_column_)) + , variant_column_ptr(assert_cast(variant_column.get())) , variant_info(variant_info_) , max_dynamic_types(max_dynamic_types_) , statistics(statistics_) @@ -57,6 +59,25 @@ ColumnDynamic::MutablePtr ColumnDynamic::create(MutableColumnPtr variant_column, return create(std::move(variant_column), variant_info, max_dynamic_types_, statistics_); } +void ColumnDynamic::setVariantType(const DB::DataTypePtr & variant_type) +{ + if (!empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting specific variant type is allowed only for empty dynamic column"); + + variant_column = variant_type->createColumn(); + variant_column_ptr = assert_cast(variant_column.get()); + variant_info.variant_type = variant_type; + variant_info.variant_name = variant_type->getName(); + const auto & variants = assert_cast(*variant_type).getVariants(); + variant_info.variant_names.reserve(variants.size()); + variant_info.variant_name_to_discriminator.reserve(variants.size()); + for (ColumnVariant::Discriminator discr = 0; discr != variants.size(); ++discr) + { + const auto & variant_name = variant_info.variant_names.emplace_back(variants[discr]->getName()); + variant_info.variant_name_to_discriminator[variant_name] = discr; + } +} + bool ColumnDynamic::addNewVariant(const DataTypePtr & new_variant, const String & variant_name) { /// Check if we already have such variant. @@ -122,7 +143,7 @@ void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DB::DataTypePt variant_info.variant_name = new_variant_type->getName(); variant_info.variant_names = new_variant_names; variant_info.variant_name_to_discriminator = new_variant_name_to_discriminator; - assert_cast(*variant_column).extend(current_to_new_discriminators, std::move(new_variant_columns_and_discriminators_to_add)); + variant_column_ptr->extend(current_to_new_discriminators, std::move(new_variant_columns_and_discriminators_to_add)); /// Clear mappings cache because now with new Variant we will have new mappings. variant_mappings_cache.clear(); } @@ -188,14 +209,14 @@ std::vector * ColumnDynamic::combineVariants(const void ColumnDynamic::insert(const DB::Field & x) { /// Check if we can insert field without Variant extension. - if (variant_column->tryInsert(x)) + if (variant_column_ptr->tryInsert(x)) return; /// If we cannot insert field into current variant column, extend it with new variant for this field from its type. if (addNewVariant(applyVisitor(FieldToDataType(), x))) { /// Now we should be able to insert this field into extended variant column. - variant_column->insert(x); + variant_column_ptr->insert(x); } else { @@ -203,7 +224,7 @@ void ColumnDynamic::insert(const DB::Field & x) /// This case should be really rare in real use cases. /// We should always be able to add String variant and cast inserted value to String. addStringVariant(); - variant_column->insert(toString(x)); + variant_column_ptr->insert(toString(x)); } } @@ -226,22 +247,20 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) /// Check if we have the same variants in both columns. if (variant_info.variant_name == dynamic_src.variant_info.variant_name) { - variant_column->insertFrom(*dynamic_src.variant_column, n); + variant_column_ptr->insertFrom(*dynamic_src.variant_column, n); return; } - auto & variant_col = assert_cast(*variant_column); - /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_col.insertFrom(*dynamic_src.variant_column, n, *global_discriminators_mapping); + variant_column_ptr->insertFrom(*dynamic_src.variant_column, n, *global_discriminators_mapping); return; } /// We cannot combine 2 Variant types as total number of variants exceeds the limit. /// We need to insert single value, try to add only corresponding variant. - const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); + const auto & src_variant_col = dynamic_src.getVariantColumn(); auto src_global_discr = src_variant_col.globalDiscriminatorAt(n); /// NULL doesn't require Variant extension. @@ -255,7 +274,7 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) if (addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator[dynamic_src.variant_info.variant_names[src_global_discr]]; - variant_col.insertIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); + variant_column_ptr->insertIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); return; } @@ -266,7 +285,7 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) tmp_variant_column->insertFrom(src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); auto tmp_string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); auto string_variant_discr = variant_info.variant_name_to_discriminator["String"]; - variant_col.insertIntoVariantFrom(string_variant_discr, *tmp_string_column, 0); + variant_column_ptr->insertIntoVariantFrom(string_variant_discr, *tmp_string_column, 0); } #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -284,16 +303,14 @@ void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, si /// Check if we have the same variants in both columns. if (variant_info.variant_names == dynamic_src.variant_info.variant_names) { - variant_column->insertRangeFrom(*dynamic_src.variant_column, start, length); + variant_column_ptr->insertRangeFrom(*dynamic_src.variant_column, start, length); return; } - auto & variant_col = assert_cast(*variant_column); - /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping); + variant_column_ptr->insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping); return; } @@ -420,7 +437,7 @@ void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, si auto local_discr = src_local_discriminators[i]; if (local_discr == ColumnVariant::NULL_DISCRIMINATOR) { - variant_col.insertDefault(); + variant_column_ptr->insertDefault(); } else { @@ -429,11 +446,11 @@ void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, si auto it = variants_converted_to_string.find(global_discr); if (it == variants_converted_to_string.end()) { - variant_col.insertIntoVariantFrom(to_global_discr, *src_variant_columns[local_discr], src_offsets[i]); + variant_column_ptr->insertIntoVariantFrom(to_global_discr, *src_variant_columns[local_discr], src_offsets[i]); } else { - variant_col.insertIntoVariantFrom(to_global_discr, *it->second, src_offsets[i] - variants_ranges[global_discr].first); + variant_column_ptr->insertIntoVariantFrom(to_global_discr, *it->second, src_offsets[i] - variants_ranges[global_discr].first); } } } @@ -450,16 +467,14 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, /// Check if we have the same variants in both columns. if (variant_info.variant_names == dynamic_src.variant_info.variant_names) { - variant_column->insertManyFrom(*dynamic_src.variant_column, position, length); + variant_column_ptr->insertManyFrom(*dynamic_src.variant_column, position, length); return; } - auto & variant_col = assert_cast(*variant_column); - /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_col.insertManyFrom(*dynamic_src.variant_column, position, length, *global_discriminators_mapping); + variant_column_ptr->insertManyFrom(*dynamic_src.variant_column, position, length, *global_discriminators_mapping); return; } @@ -477,7 +492,7 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, if (addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator[dynamic_src.variant_info.variant_names[src_global_discr]]; - variant_col.insertManyIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position), length); + variant_column_ptr->insertManyIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position), length); return; } @@ -486,7 +501,7 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, tmp_variant_column->insertFrom(src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position)); auto tmp_string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); auto string_variant_discr = variant_info.variant_name_to_discriminator["String"]; - variant_col.insertManyIntoVariantFrom(string_variant_discr, *tmp_string_column, 0, length); + variant_column_ptr->insertManyIntoVariantFrom(string_variant_discr, *tmp_string_column, 0, length); } @@ -496,8 +511,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co /// but Dynamic doesn't have fixed mapping discriminator <-> variant type /// as different Dynamic column can have different Variants. /// Instead, we serialize null bit + variant type in binary format (size + bytes) + value. - const auto & variant_col = assert_cast(*variant_column); - auto discr = variant_col.globalDiscriminatorAt(n); + auto discr = variant_column_ptr->globalDiscriminatorAt(n); StringRef res; UInt8 null_bit = discr == ColumnVariant::NULL_DISCRIMINATOR; if (null_bit) @@ -519,7 +533,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co res.data = pos; res.size = sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(); - auto value_ref = variant_col.getVariantByGlobalDiscriminator(discr).serializeValueIntoArena(variant_col.offsetAt(n), arena, begin); + auto value_ref = variant_column_ptr->getVariantByGlobalDiscriminator(discr).serializeValueIntoArena(variant_column_ptr->offsetAt(n), arena, begin); res.data = value_ref.data - res.size; res.size += value_ref.size; return res; @@ -527,7 +541,6 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) { - auto & variant_col = assert_cast(*variant_column); UInt8 null_bit = unalignedLoad(pos); pos += sizeof(UInt8); if (null_bit) @@ -550,14 +563,14 @@ const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) if (it != variant_info.variant_name_to_discriminator.end()) { auto discr = it->second; - return variant_col.deserializeVariantAndInsertFromArena(discr, pos); + return variant_column_ptr->deserializeVariantAndInsertFromArena(discr, pos); } /// If we don't have such variant, add it. if (likely(addNewVariant(variant_type))) { auto discr = variant_info.variant_name_to_discriminator[variant_name]; - return variant_col.deserializeVariantAndInsertFromArena(discr, pos); + return variant_column_ptr->deserializeVariantAndInsertFromArena(discr, pos); } /// We reached maximum number of variants and couldn't add new variant. @@ -568,7 +581,7 @@ const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) pos = tmp_variant_column->deserializeAndInsertFromArena(pos); /// Cast temporary column to String and insert this value into String variant. auto str_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - variant_col.insertIntoVariantFrom(variant_info.variant_name_to_discriminator["String"], *str_column, 0); + variant_column_ptr->insertIntoVariantFrom(variant_info.variant_name_to_discriminator["String"], *str_column, 0); return pos; } @@ -591,8 +604,7 @@ const char * ColumnDynamic::skipSerializedInArena(const char * pos) const void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const { - const auto & variant_col = assert_cast(*variant_column); - auto discr = variant_col.globalDiscriminatorAt(n); + auto discr = variant_column_ptr->globalDiscriminatorAt(n); if (discr == ColumnVariant::NULL_DISCRIMINATOR) { hash.update(discr); @@ -600,7 +612,7 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const } hash.update(variant_info.variant_names[discr]); - variant_col.getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_col.offsetAt(n), hash); + variant_column_ptr->getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_column_ptr->offsetAt(n), hash); } #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -609,9 +621,9 @@ int ColumnDynamic::compareAt(size_t n, size_t m, const DB::IColumn & rhs, int na int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const #endif { - const auto & left_variant = assert_cast(*variant_column); + const auto & left_variant = getVariantColumn(); const auto & right_dynamic = assert_cast(rhs); - const auto & right_variant = assert_cast(*right_dynamic.variant_column); + const auto & right_variant = right_dynamic.getVariantColumn(); auto left_discr = left_variant.globalDiscriminatorAt(n); auto right_discr = right_variant.globalDiscriminatorAt(m); @@ -634,7 +646,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int ColumnPtr ColumnDynamic::compress() const { - ColumnPtr variant_compressed = variant_column->compress(); + ColumnPtr variant_compressed = variant_column_ptr->compress(); size_t byte_size = variant_compressed->byteSize(); return ColumnCompressed::create(size(), byte_size, [my_variant_compressed = std::move(variant_compressed), my_variant_info = variant_info, my_max_dynamic_types = max_dynamic_types, my_statistics = statistics]() mutable @@ -740,6 +752,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source } variant_column = variant_info.variant_type->createColumn(); + variant_column_ptr = assert_cast(variant_column.get()); /// Now we have the resulting Variant that will be used in all merged columns. /// Variants can also contain Dynamic columns inside, we should collect @@ -767,12 +780,12 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source void ColumnDynamic::applyNullMap(const ColumnVector::Container & null_map) { - assert_cast(*variant_column).applyNullMap(null_map); + variant_column_ptr->applyNullMap(null_map); } void ColumnDynamic::applyNegatedNullMap(const ColumnVector::Container & null_map) { - assert_cast(*variant_column).applyNegatedNullMap(null_map); + variant_column_ptr->applyNegatedNullMap(null_map); } } diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 3d36186213a..d384dce58d0 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -107,37 +107,37 @@ public: size_t size() const override { - return variant_column->size(); + return variant_column_ptr->size(); } Field operator[](size_t n) const override { - return (*variant_column)[n]; + return (*variant_column_ptr)[n]; } void get(size_t n, Field & res) const override { - variant_column->get(n, res); + variant_column_ptr->get(n, res); } bool isDefaultAt(size_t n) const override { - return variant_column->isDefaultAt(n); + return variant_column_ptr->isDefaultAt(n); } bool isNullAt(size_t n) const override { - return variant_column->isNullAt(n); + return variant_column_ptr->isNullAt(n); } StringRef getDataAt(size_t n) const override { - return variant_column->getDataAt(n); + return variant_column_ptr->getDataAt(n); } void insertData(const char * pos, size_t length) override { - variant_column->insertData(pos, length); + variant_column_ptr->insertData(pos, length); } void insert(const Field & x) override; @@ -155,17 +155,17 @@ public: void insertDefault() override { - variant_column->insertDefault(); + variant_column_ptr->insertDefault(); } void insertManyDefaults(size_t length) override { - variant_column->insertManyDefaults(length); + variant_column_ptr->insertManyDefaults(length); } void popBack(size_t n) override { - variant_column->popBack(n); + variant_column_ptr->popBack(n); } StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; @@ -176,42 +176,42 @@ public: WeakHash32 getWeakHash32() const override { - return variant_column->getWeakHash32(); + return variant_column_ptr->getWeakHash32(); } void updateHashFast(SipHash & hash) const override { - variant_column->updateHashFast(hash); + variant_column_ptr->updateHashFast(hash); } ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - return create(variant_column->filter(filt, result_size_hint), variant_info, max_dynamic_types); + return create(variant_column_ptr->filter(filt, result_size_hint), variant_info, max_dynamic_types); } void expand(const Filter & mask, bool inverted) override { - variant_column->expand(mask, inverted); + variant_column_ptr->expand(mask, inverted); } ColumnPtr permute(const Permutation & perm, size_t limit) const override { - return create(variant_column->permute(perm, limit), variant_info, max_dynamic_types); + return create(variant_column_ptr->permute(perm, limit), variant_info, max_dynamic_types); } ColumnPtr index(const IColumn & indexes, size_t limit) const override { - return create(variant_column->index(indexes, limit), variant_info, max_dynamic_types); + return create(variant_column_ptr->index(indexes, limit), variant_info, max_dynamic_types); } ColumnPtr replicate(const Offsets & replicate_offsets) const override { - return create(variant_column->replicate(replicate_offsets), variant_info, max_dynamic_types); + return create(variant_column_ptr->replicate(replicate_offsets), variant_info, max_dynamic_types); } MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override { - MutableColumns scattered_variant_columns = variant_column->scatter(num_columns, selector); + MutableColumns scattered_variant_columns = variant_column_ptr->scatter(num_columns, selector); MutableColumns scattered_columns; scattered_columns.reserve(num_columns); for (auto & scattered_variant_column : scattered_variant_columns) @@ -228,54 +228,59 @@ public: bool hasEqualValues() const override { - return variant_column->hasEqualValues(); + return variant_column_ptr->hasEqualValues(); } void getExtremes(Field & min, Field & max) const override { - variant_column->getExtremes(min, max); + variant_column_ptr->getExtremes(min, max); } void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override { - variant_column->getPermutation(direction, stability, limit, nan_direction_hint, res); + variant_column_ptr->getPermutation(direction, stability, limit, nan_direction_hint, res); } void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override { - variant_column->updatePermutation(direction, stability, limit, nan_direction_hint, res, equal_ranges); + variant_column_ptr->updatePermutation(direction, stability, limit, nan_direction_hint, res, equal_ranges); } void reserve(size_t n) override { - variant_column->reserve(n); + variant_column_ptr->reserve(n); + } + + size_t capacity() const override + { + return variant_column_ptr->capacity(); } void ensureOwnership() override { - variant_column->ensureOwnership(); + variant_column_ptr->ensureOwnership(); } size_t byteSize() const override { - return variant_column->byteSize(); + return variant_column_ptr->byteSize(); } size_t byteSizeAt(size_t n) const override { - return variant_column->byteSizeAt(n); + return variant_column_ptr->byteSizeAt(n); } size_t allocatedBytes() const override { - return variant_column->allocatedBytes(); + return variant_column_ptr->allocatedBytes(); } void protect() override { - variant_column->protect(); + variant_column_ptr->protect(); } void forEachSubcolumn(MutableColumnCallback callback) override @@ -300,27 +305,27 @@ public: double getRatioOfDefaultRows(double sample_ratio) const override { - return variant_column->getRatioOfDefaultRows(sample_ratio); + return variant_column_ptr->getRatioOfDefaultRows(sample_ratio); } UInt64 getNumberOfDefaultRows() const override { - return variant_column->getNumberOfDefaultRows(); + return variant_column_ptr->getNumberOfDefaultRows(); } void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { - variant_column->getIndicesOfNonDefaultRows(indices, from, limit); + variant_column_ptr->getIndicesOfNonDefaultRows(indices, from, limit); } void finalize() override { - variant_column->finalize(); + variant_column_ptr->finalize(); } bool isFinalized() const override { - return variant_column->isFinalized(); + return variant_column_ptr->isFinalized(); } /// Apply null map to a nested Variant column. @@ -332,8 +337,8 @@ public: const ColumnPtr & getVariantColumnPtr() const { return variant_column; } ColumnPtr & getVariantColumnPtr() { return variant_column; } - const ColumnVariant & getVariantColumn() const { return assert_cast(*variant_column); } - ColumnVariant & getVariantColumn() { return assert_cast(*variant_column); } + const ColumnVariant & getVariantColumn() const { return *variant_column_ptr; } + ColumnVariant & getVariantColumn() { return *variant_column_ptr; } bool addNewVariant(const DataTypePtr & new_variant) { return addNewVariant(new_variant, new_variant->getName()); } bool addNewVariant(const DataTypePtr & new_variant, const String & variant_name); @@ -342,10 +347,13 @@ public: bool hasDynamicStructure() const override { return true; } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void setStatistics(const Statistics & statistics_) { statistics = statistics_; } const Statistics & getStatistics() const { return statistics; } size_t getMaxDynamicTypes() const { return max_dynamic_types; } + void setVariantType(const DataTypePtr & variant_type); + private: /// Combine current variant with the other variant and return global discriminators mapping /// from other variant to the combined one. It's used for inserting from @@ -356,6 +364,7 @@ private: void updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type); WrappedPtr variant_column; + ColumnVariant * variant_column_ptr; /// Store the type of current variant with some additional information. VariantInfo variant_info; /// The maximum number of different types that can be stored in this Dynamic column. diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 676ac7712ba..8cf0a6a57da 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -182,6 +182,11 @@ public: chars.reserve_exact(n * size); } + size_t capacity() const override + { + return chars.capacity() / n; + } + void shrinkToFit() override { chars.shrink_to_fit(); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 3766b247d60..0b12b50b97d 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -172,6 +172,7 @@ public: } void reserve(size_t n) override { idx.reserve(n); } + size_t capacity() const override { return idx.capacity(); } void shrinkToFit() override { idx.shrinkToFit(); } /// Don't count the dictionary size as it can be shared between different blocks. @@ -309,6 +310,7 @@ public: void popBack(size_t n) { positions->popBack(n); } void reserve(size_t n) { positions->reserve(n); } + size_t capacity() const { return positions->capacity(); } void shrinkToFit() { positions->shrinkToFit(); } UInt64 getMaxPositionForCurrentType() const; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 1025b4e77b9..56f5f754495 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -249,6 +249,11 @@ void ColumnMap::reserve(size_t n) nested->reserve(n); } +size_t ColumnMap::capacity() const +{ + return nested->capacity(); +} + void ColumnMap::shrinkToFit() { nested->shrinkToFit(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 3eaaa0ad562..246feff116e 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -94,6 +94,7 @@ public: void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + size_t capacity() const override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 6529f0b78db..c65fed63562 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -706,6 +706,11 @@ void ColumnNullable::reserve(size_t n) getNullMapData().reserve(n); } +size_t ColumnNullable::capacity() const +{ + return getNullMapData().capacity(); +} + void ColumnNullable::shrinkToFit() { getNestedColumn().shrinkToFit(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index fe9f5b6dcc2..c4d2eaf253c 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -125,6 +125,7 @@ public: size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override; void reserve(size_t n) override; + size_t capacity() const override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 47eb517e81b..4181c6081c7 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -49,8 +49,12 @@ ColumnObject::ColumnObject( typed_paths[path] = std::move(column); dynamic_paths.reserve(dynamic_paths_.size()); + dynamic_paths_ptrs.reserve(dynamic_paths_.size()); for (auto & [path, column] : dynamic_paths_) + { dynamic_paths[path] = std::move(column); + dynamic_paths_ptrs[path] = assert_cast(dynamic_paths[path].get()); + } } ColumnObject::ColumnObject( @@ -161,7 +165,7 @@ Field ColumnObject::operator[](size_t n) const for (const auto & [path, column] : typed_paths) object[path] = (*column)[n]; - for (const auto & [path, column] : dynamic_paths) + for (const auto & [path, column] : dynamic_paths_ptrs) { /// Output only non-null values from dynamic paths. We cannot distinguish cases when /// dynamic path has Null value and when it's absent in the row and consider them equivalent. @@ -199,7 +203,7 @@ bool ColumnObject::isDefaultAt(size_t n) const return false; } - for (const auto & [path, column] : dynamic_paths) + for (const auto & [path, column] : dynamic_paths_ptrs) { if (!column->isDefaultAt(n)) return false; @@ -221,15 +225,17 @@ void ColumnObject::insertData(const char *, size_t) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertData is not supported for {}", getName()); } -IColumn * ColumnObject::tryToAddNewDynamicPath(const String & path) +ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(const String & path) { if (dynamic_paths.size() == max_dynamic_paths) return nullptr; auto new_dynamic_column = ColumnDynamic::create(max_dynamic_types); + new_dynamic_column->reserve(shared_data->capacity()); new_dynamic_column->insertManyDefaults(size()); auto it = dynamic_paths.emplace(path, std::move(new_dynamic_column)).first; - return it->second.get(); + auto it_ptr = dynamic_paths_ptrs.emplace(path, assert_cast(it->second.get())).first; + return it_ptr->second; } void ColumnObject::setDynamicPaths(const std::vector & paths) @@ -244,6 +250,7 @@ void ColumnObject::setDynamicPaths(const std::vector & paths) if (size) new_dynamic_column->insertManyDefaults(size); dynamic_paths[path] = std::move(new_dynamic_column); + dynamic_paths_ptrs[path] = assert_cast(dynamic_paths[path].get()); } } @@ -259,7 +266,7 @@ void ColumnObject::insert(const Field & x) { typed_it->second->insert(value_field); } - else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) + else if (auto dynamic_it = dynamic_paths_ptrs.find(path); dynamic_it != dynamic_paths_ptrs.end()) { dynamic_it->second->insert(value_field); } @@ -291,7 +298,7 @@ void ColumnObject::insert(const Field & x) column->insertDefault(); } - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() == current_size) column->insertDefault(); @@ -317,7 +324,7 @@ bool ColumnObject::tryInsert(const Field & x) column->popBack(column->size() - prev_size); } - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() != prev_size) column->popBack(column->size() - prev_size); @@ -339,7 +346,7 @@ bool ColumnObject::tryInsert(const Field & x) return false; } } - else if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) + else if (auto dynamic_it = dynamic_paths_ptrs.find(path); dynamic_it != dynamic_paths_ptrs.end()) { if (!dynamic_it->second->tryInsert(value_field)) { @@ -376,7 +383,7 @@ bool ColumnObject::tryInsert(const Field & x) column->insertDefault(); } - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() == prev_size) column->insertDefault(); @@ -404,7 +411,7 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) for (const auto & [path, column] : src_object_column.dynamic_paths) { /// Check if we already have such dynamic path. - if (auto it = dynamic_paths.find(path); it != dynamic_paths.end()) + if (auto it = dynamic_paths_ptrs.find(path); it != dynamic_paths_ptrs.end()) it->second->insertFrom(*column, n); /// Try to add a new dynamic path. else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) @@ -438,7 +445,7 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l for (const auto & [path, column] : src_object_column.dynamic_paths) { /// Check if we already have such dynamic path. - if (auto it = dynamic_paths.find(path); it != dynamic_paths.end()) + if (auto it = dynamic_paths_ptrs.find(path); it != dynamic_paths_ptrs.end()) it->second->insertRangeFrom(*column, start, length); /// Try to add a new dynamic path. else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) @@ -484,7 +491,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co } /// Insert default values in all remaining dynamic paths. - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() == current_size) column->insertManyDefaults(length); @@ -508,7 +515,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co { auto path = src_shared_data_paths->getDataAt(i); /// Check if we have this path in dynamic paths. - if (auto it = dynamic_paths.find(path.toString()); it != dynamic_paths.end()) + if (auto it = dynamic_paths_ptrs.find(path.toString()); it != dynamic_paths_ptrs.end()) { /// Deserialize binary value into dynamic column from shared data. deserializeValueFromSharedData(src_shared_data_values, i, *it->second); @@ -541,7 +548,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co shared_data_offsets.push_back(shared_data_paths->size()); /// Insert default value in all remaining dynamic paths. - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() == current_size) column->insertDefault(); @@ -576,7 +583,7 @@ void ColumnObject::insertDefault() { for (auto & [_, column] : typed_paths) column->insertDefault(); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->insertDefault(); shared_data->insertDefault(); } @@ -585,7 +592,7 @@ void ColumnObject::insertManyDefaults(size_t length) { for (auto & [_, column] : typed_paths) column->insertManyDefaults(length); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->insertManyDefaults(length); shared_data->insertManyDefaults(length); } @@ -594,7 +601,7 @@ void ColumnObject::popBack(size_t n) { for (auto & [_, column] : typed_paths) column->popBack(n); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->popBack(n); shared_data->popBack(n); } @@ -742,7 +749,7 @@ void ColumnObject::updateHashWithValue(size_t n, SipHash & hash) const { for (const auto & [_, column] : typed_paths) column->updateHashWithValue(n, hash); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) column->updateHashWithValue(n, hash); shared_data->updateHashWithValue(n, hash); } @@ -752,7 +759,7 @@ WeakHash32 ColumnObject::getWeakHash32() const WeakHash32 hash(size()); for (const auto & [_, column] : typed_paths) hash.update(column->getWeakHash32()); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) hash.update(column->getWeakHash32()); hash.update(shared_data->getWeakHash32()); return hash; @@ -762,7 +769,7 @@ void ColumnObject::updateHashFast(SipHash & hash) const { for (const auto & [_, column] : typed_paths) column->updateHashFast(hash); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) column->updateHashFast(hash); shared_data->updateHashFast(hash); } @@ -775,8 +782,8 @@ ColumnPtr ColumnObject::filter(const Filter & filt, ssize_t result_size_hint) co filtered_typed_paths[path] = column->filter(filt, result_size_hint); std::unordered_map filtered_dynamic_paths; - filtered_dynamic_paths.reserve(dynamic_paths.size()); - for (const auto & [path, column] : dynamic_paths) + filtered_dynamic_paths.reserve(dynamic_paths_ptrs.size()); + for (const auto & [path, column] : dynamic_paths_ptrs) filtered_dynamic_paths[path] = column->filter(filt, result_size_hint); auto filtered_shared_data = shared_data->filter(filt, result_size_hint); @@ -787,7 +794,7 @@ void ColumnObject::expand(const Filter & mask, bool inverted) { for (auto & [_, column] : typed_paths) column->expand(mask, inverted); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->expand(mask, inverted); shared_data->expand(mask, inverted); } @@ -800,8 +807,8 @@ ColumnPtr ColumnObject::permute(const Permutation & perm, size_t limit) const permuted_typed_paths[path] = column->permute(perm, limit); std::unordered_map permuted_dynamic_paths; - permuted_dynamic_paths.reserve(dynamic_paths.size()); - for (const auto & [path, column] : dynamic_paths) + permuted_dynamic_paths.reserve(dynamic_paths_ptrs.size()); + for (const auto & [path, column] : dynamic_paths_ptrs) permuted_dynamic_paths[path] = column->permute(perm, limit); auto permuted_shared_data = shared_data->permute(perm, limit); @@ -816,8 +823,8 @@ ColumnPtr ColumnObject::index(const IColumn & indexes, size_t limit) const indexed_typed_paths[path] = column->index(indexes, limit); std::unordered_map indexed_dynamic_paths; - indexed_dynamic_paths.reserve(dynamic_paths.size()); - for (const auto & [path, column] : dynamic_paths) + indexed_dynamic_paths.reserve(dynamic_paths_ptrs.size()); + for (const auto & [path, column] : dynamic_paths_ptrs) indexed_dynamic_paths[path] = column->index(indexes, limit); auto indexed_shared_data = shared_data->index(indexes, limit); @@ -832,8 +839,8 @@ ColumnPtr ColumnObject::replicate(const Offsets & replicate_offsets) const replicated_typed_paths[path] = column->replicate(replicate_offsets); std::unordered_map replicated_dynamic_paths; - replicated_dynamic_paths.reserve(dynamic_paths.size()); - for (const auto & [path, column] : dynamic_paths) + replicated_dynamic_paths.reserve(dynamic_paths_ptrs.size()); + for (const auto & [path, column] : dynamic_paths_ptrs) replicated_dynamic_paths[path] = column->replicate(replicate_offsets); auto replicated_shared_data = shared_data->replicate(replicate_offsets); @@ -855,9 +862,9 @@ MutableColumns ColumnObject::scatter(ColumnIndex num_columns, const Selector & s std::vector> scattered_dynamic_paths(num_columns); for (auto & dynamic_paths_ : scattered_dynamic_paths) - dynamic_paths_.reserve(dynamic_paths.size()); + dynamic_paths_.reserve(dynamic_paths_ptrs.size()); - for (const auto & [path, column] : dynamic_paths) + for (const auto & [path, column] : dynamic_paths_ptrs) { auto scattered_columns = column->scatter(num_columns, selector); for (size_t i = 0; i != num_columns; ++i) @@ -883,16 +890,21 @@ void ColumnObject::reserve(size_t n) { for (auto & [_, column] : typed_paths) column->reserve(n); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->reserve(n); shared_data->reserve(n); } +size_t ColumnObject::capacity() const +{ + return shared_data->capacity(); +} + void ColumnObject::ensureOwnership() { for (auto & [_, column] : typed_paths) column->ensureOwnership(); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->ensureOwnership(); shared_data->ensureOwnership(); } @@ -902,7 +914,7 @@ size_t ColumnObject::byteSize() const size_t size = 0; for (const auto & [_, column] : typed_paths) size += column->byteSize(); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) size += column->byteSize(); size += shared_data->byteSize(); return size; @@ -913,7 +925,7 @@ size_t ColumnObject::byteSizeAt(size_t n) const size_t size = 0; for (const auto & [_, column] : typed_paths) size += column->byteSizeAt(n); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) size += column->byteSizeAt(n); size += shared_data->byteSizeAt(n); return size; @@ -924,7 +936,7 @@ size_t ColumnObject::allocatedBytes() const size_t size = 0; for (const auto & [_, column] : typed_paths) size += column->allocatedBytes(); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) size += column->allocatedBytes(); size += shared_data->allocatedBytes(); return size; @@ -934,7 +946,7 @@ void ColumnObject::protect() { for (auto & [_, column] : typed_paths) column->protect(); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->protect(); shared_data->protect(); } @@ -955,7 +967,7 @@ void ColumnObject::forEachSubcolumnRecursively(DB::IColumn::RecursiveMutableColu callback(*column); column->forEachSubcolumnRecursively(callback); } - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) { callback(*column); column->forEachSubcolumnRecursively(callback); @@ -994,8 +1006,8 @@ ColumnPtr ColumnObject::compress() const } std::unordered_map compressed_dynamic_paths; - compressed_dynamic_paths.reserve(dynamic_paths.size()); - for (const auto & [path, column] : dynamic_paths) + compressed_dynamic_paths.reserve(dynamic_paths_ptrs.size()); + for (const auto & [path, column] : dynamic_paths_ptrs) { auto compressed_column = column->compress(); byte_size += compressed_column->byteSize(); @@ -1034,7 +1046,7 @@ void ColumnObject::finalize() { for (auto & [_, column] : typed_paths) column->finalize(); - for (auto & [_, column] : dynamic_paths) + for (auto & [_, column] : dynamic_paths_ptrs) column->finalize(); shared_data->finalize(); } @@ -1044,7 +1056,7 @@ bool ColumnObject::isFinalized() const bool finalized = true; for (const auto & [_, column] : typed_paths) finalized &= column->isFinalized(); - for (const auto & [_, column] : dynamic_paths) + for (const auto & [_, column] : dynamic_paths_ptrs) finalized &= column->isFinalized(); finalized &= shared_data->isFinalized(); return finalized; @@ -1101,6 +1113,7 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou } dynamic_paths.clear(); + dynamic_paths_ptrs.clear(); /// Check if the number of all dynamic paths exceeds the limit. if (path_to_total_number_of_non_null_values.size() > max_dynamic_paths) @@ -1114,13 +1127,19 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou /// Fill dynamic_paths with first max_dynamic_paths paths in sorted list. for (size_t i = 0; i != max_dynamic_paths; ++i) + { dynamic_paths[paths_with_sizes[i].second] = ColumnDynamic::create(max_dynamic_types); + dynamic_paths_ptrs[paths_with_sizes[i].second] = assert_cast(dynamic_paths[paths_with_sizes[i].second].get()); + } } /// Use all dynamic paths from all source columns. else { for (const auto & [path, _] : path_to_total_number_of_non_null_values) + { dynamic_paths[path] = ColumnDynamic::create(max_dynamic_types); + dynamic_paths_ptrs[path] = assert_cast(dynamic_paths[path].get()); + } } /// Fill statistics for the merged part. diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 0b949cbcb3a..4a27edfc295 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -116,6 +116,7 @@ public: void getExtremes(Field & min, Field & max) const override; void reserve(size_t n) override; + size_t capacity() const override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; @@ -142,6 +143,9 @@ public: const std::unordered_map & getDynamicPaths() const { return dynamic_paths; } std::unordered_map & getDynamicPaths() { return dynamic_paths; } + const std::unordered_map & getDynamicPathsPtrs() const { return dynamic_paths_ptrs; } + std::unordered_map & getDynamicPathsPtrs() { return dynamic_paths_ptrs; } + const Statistics & getStatistics() const { return statistics; } const ColumnPtr & getSharedDataPtr() const { return shared_data; } @@ -173,7 +177,7 @@ public: /// Try to add new dynamic path. Returns pointer to the new dynamic /// path column or nullptr if limit on dynamic paths is reached. - IColumn * tryToAddNewDynamicPath(const String & path); + ColumnDynamic * tryToAddNewDynamicPath(const String & path); void setDynamicPaths(const std::vector & paths); void setStatistics(const Statistics & statistics_) { statistics = statistics_; } @@ -197,6 +201,7 @@ private: /// here are Dynamic columns. This set of paths can be extended /// during inerts into the column. std::unordered_map dynamic_paths; + std::unordered_map dynamic_paths_ptrs; /// Shared storage for all other paths and values. It's filled /// when the number of dynamic paths reaches the limit. /// It has type Array(Tuple(String, String)) and stores diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 7cfa2571f5a..4cf0eb894e6 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -557,6 +557,11 @@ void ColumnString::reserve(size_t n) offsets.reserve_exact(n); } +size_t ColumnString::capacity() const +{ + return offsets.capacity(); +} + void ColumnString::shrinkToFit() { chars.shrink_to_fit(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c1012e1e55e..6b94de83530 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -283,6 +283,7 @@ public: ColumnPtr compress() const override; void reserve(size_t n) override; + size_t capacity() const override; void shrinkToFit() override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 4fc3f88a87c..8cef8b0a499 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -595,6 +595,14 @@ void ColumnTuple::reserve(size_t n) getColumn(i).reserve(n); } +size_t ColumnTuple::capacity() const +{ + if (columns.empty()) + return size(); + + return getColumn(0).capacity(); +} + void ColumnTuple::shrinkToFit() { const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 16b47a993f6..502022c9532 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -110,6 +110,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + size_t capacity() const override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index de7efb41d19..adba4e9691f 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1251,6 +1251,11 @@ void ColumnVariant::reserve(size_t n) offsets->reserve(n); } +size_t ColumnVariant::capacity() const +{ + return local_discriminators->capacity(); +} + void ColumnVariant::ensureOwnership() { const size_t num_variants = variants.size(); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 34c24b5428d..b4bc63e8d4c 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -237,6 +237,7 @@ public: size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + size_t capacity() const override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 2fe5b635bd2..81d514afd4a 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -180,6 +180,11 @@ public: data.reserve_exact(n); } + size_t capacity() const override + { + return data.capacity(); + } + void shrinkToFit() override { data.shrink_to_fit(); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index f9c1a3e7034..e75275d03c5 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -475,6 +475,9 @@ public: /// It affects performance only (not correctness). virtual void reserve(size_t /*n*/) {} + /// Returns the number of elements allocated in reserve. + virtual size_t capacity() const { return size(); } + /// Requests the removal of unused capacity. /// It is a non-binding request to reduce the capacity of the underlying container to its size. virtual void shrinkToFit() {} diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index a6c5c201f36..eeda6476fd0 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -314,10 +314,15 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu } auto & result_dynamic_columns = result_object_column.getDynamicPaths(); + auto & result_dynamic_columns_ptrs = result_object_column.getDynamicPathsPtrs(); for (const auto & [path, column] : object_column.getDynamicPaths()) { if (path.starts_with(prefix) && path.size() != prefix.size()) - result_dynamic_columns[getSubPath(path, prefix)] = column; + { + auto sub_path = getSubPath(path, prefix); + result_dynamic_columns[sub_path] = column; + result_dynamic_columns_ptrs[sub_path] = assert_cast(result_dynamic_columns[sub_path].get()); + } } const auto & shared_data_offsets = object_column.getSharedDataOffsets(); diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index c1ef67b2aaa..bd760648498 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -279,13 +279,16 @@ void SerializationDynamic::deserializeBinaryBulkWithMultipleStreams( return; auto mutable_column = column->assumeMutable(); + auto & column_dynamic = assert_cast(*mutable_column); auto * dynamic_state = checkAndGetState(state); auto * structure_state = checkAndGetState(dynamic_state->structure_state); - if (mutable_column->empty()) - mutable_column = ColumnDynamic::create(structure_state->variant_type->createColumn(), structure_state->variant_type, max_dynamic_types, structure_state->statistics); + if (column_dynamic.empty()) + { + column_dynamic.setVariantType(structure_state->variant_type); + column_dynamic.setStatistics(structure_state->statistics); + } - auto & column_dynamic = assert_cast(*mutable_column); const auto & variant_info = column_dynamic.getVariantInfo(); if (!variant_info.variant_type->equals(*structure_state->variant_type)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of internal columns of Dynamic. Expected: {}, Got: {}", structure_state->variant_type->getName(), variant_info.variant_type->getName()); diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index e8a78a40749..f5b5a7bb984 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1650,7 +1650,7 @@ public: typed_column->insertDefault(); } - for (auto & [_, dynamic_column] : column_object.getDynamicPaths()) + for (auto & [_, dynamic_column] : column_object.getDynamicPathsPtrs()) { if (dynamic_column->size() == prev_size) dynamic_column->insertDefault(); @@ -1689,7 +1689,7 @@ private: } auto & typed_paths = column_object.getTypedPaths(); - auto & dynamic_paths = column_object.getDynamicPaths(); + auto & dynamic_paths_ptrs = column_object.getDynamicPathsPtrs(); /// Check if we have this path in typed paths. if (auto typed_it = typed_paths.find(current_path); typed_it != typed_paths.end()) { @@ -1709,7 +1709,7 @@ private: } } /// Check if we have this path in dynamic paths. - else if (auto dynamic_it = dynamic_paths.find(current_path); dynamic_it != dynamic_paths.end()) + else if (auto dynamic_it = dynamic_paths_ptrs.find(current_path); dynamic_it != dynamic_paths_ptrs.end()) { /// Check if we already had this path. if (dynamic_it->second->size() > current_size) diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 0b6c81923db..31d7fdbf67b 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -124,6 +124,10 @@ Chunk IRowInputFormat::read() return getChunkForCount(num_rows); } + /// Reserve params.max_block_size rows in advance. + for (auto & column : columns) + column->reserve(params.max_block_size); + RowReadExtension info; bool continue_reading = true; for (size_t rows = 0; (rows < params.max_block_size || num_rows == 0) && continue_reading; ++rows) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index c5e36046c62..dde8a86babd 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -15,11 +15,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) - : JSONAsRowInputFormat(header_, std::make_unique(in_), params_, format_settings_) {} - -JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & format_settings_) : - JSONEachRowRowInputFormat(*buf_, header_, std::move(params_), format_settings_, false), buf(std::move(buf_)) +JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) : + JSONEachRowRowInputFormat(in_, header_, std::move(params_), format_settings_, false) { if (header_.columns() > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -27,19 +24,6 @@ JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_pt header_.columns()); } - -void JSONAsRowInputFormat::setReadBuffer(ReadBuffer & in_) -{ - buf = std::make_unique(in_); - JSONEachRowRowInputFormat::setReadBuffer(*buf); -} - -void JSONAsRowInputFormat::resetReadBuffer() -{ - buf.reset(); - JSONEachRowRowInputFormat::resetReadBuffer(); -} - bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { assert(columns.size() == 1); @@ -48,35 +32,44 @@ bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) if (!allow_new_rows) return false; - skipWhitespaceIfAny(*buf); - if (!buf->eof()) + skipWhitespaceIfAny(*in); + if (!in->eof()) { - if (!data_in_square_brackets && *buf->position() == ';') + if (!data_in_square_brackets && *in->position() == ';') { /// ';' means the end of query, but it cannot be before ']'. return allow_new_rows = false; } - else if (data_in_square_brackets && *buf->position() == ']') + else if (data_in_square_brackets && *in->position() == ']') { /// ']' means the end of query. return allow_new_rows = false; } } - if (!buf->eof()) + if (!in->eof()) readJSONObject(*columns[0]); - skipWhitespaceIfAny(*buf); - if (!buf->eof() && *buf->position() == ',') - ++buf->position(); - skipWhitespaceIfAny(*buf); + skipWhitespaceIfAny(*in); + if (!in->eof() && *in->position() == ',') + ++in->position(); + skipWhitespaceIfAny(*in); - return !buf->eof(); + return !in->eof(); } JSONAsStringRowInputFormat::JSONAsStringRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) : JSONAsRowInputFormat(header_, in_, params_, format_settings_) +{ +} + +JSONAsStringRowInputFormat::JSONAsStringRowInputFormat( + const DB::Block & header_, + std::unique_ptr buf_, + DB::IRowInputFormat::Params params_, + const DB::FormatSettings & format_settings_) + : JSONAsRowInputFormat(header_, *buf_, params_, format_settings_), buf(std::move(buf_)) { if (!isString(removeNullable(removeLowCardinality(header_.getByPosition(0).type)))) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -84,6 +77,18 @@ JSONAsStringRowInputFormat::JSONAsStringRowInputFormat( header_.getByPosition(0).type->getName()); } +void JSONAsStringRowInputFormat::setReadBuffer(ReadBuffer & in_) +{ + buf = std::make_unique(in_); + JSONAsRowInputFormat::setReadBuffer(*buf); +} + +void JSONAsStringRowInputFormat::resetReadBuffer() +{ + buf.reset(); + JSONAsRowInputFormat::resetReadBuffer(); +} + void JSONAsStringRowInputFormat::readJSONObject(IColumn & column) { PeekableReadBufferCheckpoint checkpoint{*buf}; @@ -175,7 +180,7 @@ JSONAsObjectRowInputFormat::JSONAsObjectRowInputFormat( void JSONAsObjectRowInputFormat::readJSONObject(IColumn & column) { - serializations[0]->deserializeTextJSON(column, *buf, format_settings); + serializations[0]->deserializeTextJSON(column, *in, format_settings); } Chunk JSONAsObjectRowInputFormat::getChunkForCount(size_t rows) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index a5a436260ba..7c3c1fb9fb7 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -19,17 +19,11 @@ class JSONAsRowInputFormat : public JSONEachRowRowInputFormat public: JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings); - void setReadBuffer(ReadBuffer & in_) override; - void resetReadBuffer() override; - private: - JSONAsRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & format_settings); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; protected: virtual void readJSONObject(IColumn & column) = 0; - std::unique_ptr buf; }; /// Each JSON object is parsed as a whole to string. @@ -37,11 +31,17 @@ protected: class JSONAsStringRowInputFormat final : public JSONAsRowInputFormat { public: - JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings); + JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_); String getName() const override { return "JSONAsStringRowInputFormat"; } + void setReadBuffer(ReadBuffer & in_) override; + void resetReadBuffer() override; + private: + JSONAsStringRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & format_settings); + void readJSONObject(IColumn & column) override; + std::unique_ptr buf; }; From ef4ad01d7d57942a79b46ad05c10160d557ff75f Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Tue, 23 Jul 2024 15:03:01 +0100 Subject: [PATCH 0363/1722] Fix totals formatting --- .../Impl/JSONCompactWithProgressRowOutputFormat.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 9603ce0265d..ddf071324c6 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -55,8 +55,8 @@ void JSONCompactWithProgressRowOutputFormat::writeRowBetweenDelimiter() void JSONCompactWithProgressRowOutputFormat::writeBeforeTotals() { - JSONUtils::writeFieldDelimiter(*ostr, 2); - JSONUtils::writeCompactArrayStart(*ostr, 1, "totals"); + JSONUtils::writeCompactObjectStart(*ostr); + JSONUtils::writeCompactArrayStart(*ostr, 0, "totals"); } void JSONCompactWithProgressRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) @@ -67,6 +67,8 @@ void JSONCompactWithProgressRowOutputFormat::writeTotals(const Columns & columns void JSONCompactWithProgressRowOutputFormat::writeAfterTotals() { JSONUtils::writeCompactArrayEnd(*ostr); + JSONUtils::writeCompactObjectEnd(*ostr); + writeCString("}\n", *ostr); } void JSONCompactWithProgressRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) @@ -131,6 +133,7 @@ void JSONCompactWithProgressRowOutputFormat::finalizeImpl() JSONUtils::writeException(exception_message, *ostr, settings, 0); JSONUtils::writeCompactObjectEnd(*ostr); } + writeCString("\n", *ostr); ostr->next(); } From f3710e9a29e4b866bfd55886c18ebad954a48e42 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Tue, 23 Jul 2024 15:04:05 +0100 Subject: [PATCH 0364/1722] Add stateless test for JSONCompactWithProgress output format --- ...03174_json_compact_with_progress.reference | 14 +++++++++++++ .../03174_json_compact_with_progress.sh | 20 +++++++++++++++++++ 2 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/03174_json_compact_with_progress.reference create mode 100755 tests/queries/0_stateless/03174_json_compact_with_progress.sh diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.reference b/tests/queries/0_stateless/03174_json_compact_with_progress.reference new file mode 100644 index 00000000000..6e83db33565 --- /dev/null +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.reference @@ -0,0 +1,14 @@ +1 +{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}]}} +{"data":[1, "a"]} +{"data":[2, "b"]} +{"data":[3, "c"]} +{"progress":{"read_rows":"3","read_bytes":"33","written_rows":"0","written_bytes":"0","total_rows_to_read":"3","result_rows":"0","result_bytes":"0","elapsed_ns":"ELAPSED_NS"}} +{"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":33}} +2 +{"meta": [{"name":"name", "type":"String"}, {"name":"c", "type":"UInt64"}]}} +{"data":["a", "1"]} +{"data":["b", "1"]} +{"data":["c", "1"]} +{"totals": ["", "3"]}} +{"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":30}} diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.sh b/tests/queries/0_stateless/03174_json_compact_with_progress.sh new file mode 100755 index 00000000000..c21f7228517 --- /dev/null +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_table;" + +$CLICKHOUSE_CLIENT -q "SELECT 1;" +# Check JSONCompactWithProgress Output +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value;" +$CLICKHOUSE_CLIENT -q "INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c');" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_table FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' + +$CLICKHOUSE_CLIENT -q "SELECT 2;" +# Check Totals +$CLICKHOUSE_CLIENT -q "SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_table;" From ea15ad4ff5929106c7e3cc18204764472a360811 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 14:45:54 +0000 Subject: [PATCH 0365/1722] Fix --- src/Columns/ColumnDynamic.h | 5 +++++ src/Columns/ColumnObject.cpp | 8 ++++++-- src/Columns/ColumnObject.h | 2 ++ utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 4 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index d384dce58d0..5a915134886 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -286,11 +286,13 @@ public: void forEachSubcolumn(MutableColumnCallback callback) override { callback(variant_column); + variant_column_ptr = assert_cast(variant_column.get()); } void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override { callback(*variant_column); + variant_column_ptr = assert_cast(variant_column.get()); variant_column->forEachSubcolumnRecursively(callback); } @@ -364,6 +366,9 @@ private: void updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type); WrappedPtr variant_column; + /// Store and use pointer to ColumnVariant to avoid virtual calls. + /// ColumnDynamic is widely used inside ColumnObject for each path and + /// with hundreds of paths these virtual calls are noticeable. ColumnVariant * variant_column_ptr; /// Store the type of current variant with some additional information. VariantInfo variant_info; diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 4181c6081c7..ee0d87f500a 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -955,8 +955,11 @@ void ColumnObject::forEachSubcolumn(DB::IColumn::MutableColumnCallback callback) { for (auto & [_, column] : typed_paths) callback(column); - for (auto & [_, column] : dynamic_paths) + for (auto & [path, column] : dynamic_paths) + { callback(column); + dynamic_paths_ptrs[path] = assert_cast(column.get()); + } callback(shared_data); } @@ -967,10 +970,11 @@ void ColumnObject::forEachSubcolumnRecursively(DB::IColumn::RecursiveMutableColu callback(*column); column->forEachSubcolumnRecursively(callback); } - for (auto & [_, column] : dynamic_paths_ptrs) + for (auto & [path, column] : dynamic_paths) { callback(*column); column->forEachSubcolumnRecursively(callback); + dynamic_paths_ptrs[path] = assert_cast(column.get()); } callback(*shared_data); shared_data->forEachSubcolumnRecursively(callback); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 4a27edfc295..d274f4e857a 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -201,6 +201,8 @@ private: /// here are Dynamic columns. This set of paths can be extended /// during inerts into the column. std::unordered_map dynamic_paths; + /// Store and use pointers to ColumnDynamic to avoid virtual calls. + /// With hundreds of dynamic paths these virtual calls are noticeable. std::unordered_map dynamic_paths_ptrs; /// Shared storage for all other paths and values. It's filled /// when the number of dynamic paths reaches the limit. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3c7fe12ff08..506d89ca646 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2117,6 +2117,7 @@ natively nats nestjs netloc +newjson ngram ngramDistance ngramDistanceCaseInsensitive From 18a43be044947cdc450f1342db440f76d2fa6bbe Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 15:57:09 +0000 Subject: [PATCH 0366/1722] Revert changes in JSONAsString format --- .../Impl/JSONAsStringRowInputFormat.cpp | 63 +++++++++---------- .../Formats/Impl/JSONAsStringRowInputFormat.h | 16 ++--- 2 files changed, 37 insertions(+), 42 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index dde8a86babd..c5e36046c62 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -15,8 +15,11 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) : - JSONEachRowRowInputFormat(in_, header_, std::move(params_), format_settings_, false) +JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) + : JSONAsRowInputFormat(header_, std::make_unique(in_), params_, format_settings_) {} + +JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & format_settings_) : + JSONEachRowRowInputFormat(*buf_, header_, std::move(params_), format_settings_, false), buf(std::move(buf_)) { if (header_.columns() > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -24,6 +27,19 @@ JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & i header_.columns()); } + +void JSONAsRowInputFormat::setReadBuffer(ReadBuffer & in_) +{ + buf = std::make_unique(in_); + JSONEachRowRowInputFormat::setReadBuffer(*buf); +} + +void JSONAsRowInputFormat::resetReadBuffer() +{ + buf.reset(); + JSONEachRowRowInputFormat::resetReadBuffer(); +} + bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { assert(columns.size() == 1); @@ -32,44 +48,35 @@ bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) if (!allow_new_rows) return false; - skipWhitespaceIfAny(*in); - if (!in->eof()) + skipWhitespaceIfAny(*buf); + if (!buf->eof()) { - if (!data_in_square_brackets && *in->position() == ';') + if (!data_in_square_brackets && *buf->position() == ';') { /// ';' means the end of query, but it cannot be before ']'. return allow_new_rows = false; } - else if (data_in_square_brackets && *in->position() == ']') + else if (data_in_square_brackets && *buf->position() == ']') { /// ']' means the end of query. return allow_new_rows = false; } } - if (!in->eof()) + if (!buf->eof()) readJSONObject(*columns[0]); - skipWhitespaceIfAny(*in); - if (!in->eof() && *in->position() == ',') - ++in->position(); - skipWhitespaceIfAny(*in); + skipWhitespaceIfAny(*buf); + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); + skipWhitespaceIfAny(*buf); - return !in->eof(); + return !buf->eof(); } JSONAsStringRowInputFormat::JSONAsStringRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) : JSONAsRowInputFormat(header_, in_, params_, format_settings_) -{ -} - -JSONAsStringRowInputFormat::JSONAsStringRowInputFormat( - const DB::Block & header_, - std::unique_ptr buf_, - DB::IRowInputFormat::Params params_, - const DB::FormatSettings & format_settings_) - : JSONAsRowInputFormat(header_, *buf_, params_, format_settings_), buf(std::move(buf_)) { if (!isString(removeNullable(removeLowCardinality(header_.getByPosition(0).type)))) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -77,18 +84,6 @@ JSONAsStringRowInputFormat::JSONAsStringRowInputFormat( header_.getByPosition(0).type->getName()); } -void JSONAsStringRowInputFormat::setReadBuffer(ReadBuffer & in_) -{ - buf = std::make_unique(in_); - JSONAsRowInputFormat::setReadBuffer(*buf); -} - -void JSONAsStringRowInputFormat::resetReadBuffer() -{ - buf.reset(); - JSONAsRowInputFormat::resetReadBuffer(); -} - void JSONAsStringRowInputFormat::readJSONObject(IColumn & column) { PeekableReadBufferCheckpoint checkpoint{*buf}; @@ -180,7 +175,7 @@ JSONAsObjectRowInputFormat::JSONAsObjectRowInputFormat( void JSONAsObjectRowInputFormat::readJSONObject(IColumn & column) { - serializations[0]->deserializeTextJSON(column, *in, format_settings); + serializations[0]->deserializeTextJSON(column, *buf, format_settings); } Chunk JSONAsObjectRowInputFormat::getChunkForCount(size_t rows) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index 7c3c1fb9fb7..593c4ae6fea 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -19,11 +19,17 @@ class JSONAsRowInputFormat : public JSONEachRowRowInputFormat public: JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings); + void setReadBuffer(ReadBuffer & in_) override; + void resetReadBuffer() override; + private: + JSONAsRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & format_settings); + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; protected: virtual void readJSONObject(IColumn & column) = 0; + std::unique_ptr buf; }; /// Each JSON object is parsed as a whole to string. @@ -31,17 +37,11 @@ protected: class JSONAsStringRowInputFormat final : public JSONAsRowInputFormat { public: - JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_); + JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings); String getName() const override { return "JSONAsStringRowInputFormat"; } - void setReadBuffer(ReadBuffer & in_) override; - void resetReadBuffer() override; - private: - JSONAsStringRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & format_settings); - void readJSONObject(IColumn & column) override; - std::unique_ptr buf; }; @@ -83,4 +83,4 @@ private: FormatSettings settings; }; -} +} \ No newline at end of file From 81b5caad2ddb1cf59e0349a479d67aa9b7893256 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Jul 2024 15:59:03 +0000 Subject: [PATCH 0367/1722] Fix header --- .../Formats/Impl/JSONAsStringRowInputFormat.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index 593c4ae6fea..61bb7f810e0 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -1,12 +1,12 @@ #pragma once -#include -#include -#include +#include +#include #include #include -#include -#include +#include +#include +#include namespace DB { @@ -83,4 +83,4 @@ private: FormatSettings settings; }; -} \ No newline at end of file +} From f4138ee6c67bbdb82269a9087b1b054f33cb35a8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Jul 2024 10:20:27 +0800 Subject: [PATCH 0368/1722] fix bugs about corner cases --- src/Functions/FunctionOverlay.cpp | 89 +++++++++++++++++-------------- 1 file changed, 48 insertions(+), 41 deletions(-) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index 7d0e2e86de2..d3ee7e1df6d 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -218,6 +218,26 @@ public: private: + /// input offset is 1-based, maybe negative + /// output result is 0-based valid offset, within [0, input_size] + static size_t getValidOffset(Int64 offset, size_t input_size) + { + if (offset > 0) + { + if (static_cast(offset) > input_size + 1) [[unlikely]] + return input_size; + else + return offset - 1; + } + else + { + if (input_size < -static_cast(offset)) [[unlikely]] + return 0; + else + return input_size + offset; + } + } + template void constantConstant( size_t rows, @@ -237,13 +257,10 @@ private: return; } - Int64 offset = 0; // start from 1, maybe negative + size_t input_size = input.size; size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } + valid_offset = getValidOffset(const_offset, input_size); size_t replace_size = replace.size; Int64 length = 0; // maybe negative @@ -258,14 +275,14 @@ private: valid_length = replace_size; } + Int64 offset = 0; // start from 1, maybe negative size_t res_offset = 0; - size_t input_size = input.size; for (size_t i = 0; i < rows; ++i) { if constexpr (!offset_is_const) { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (!three_args && !length_is_const) @@ -274,7 +291,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -321,14 +338,6 @@ private: return; } - Int64 offset = 0; // start from 1, maybe negative - size_t valid_offset = 0; // start from 0, not negative - if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } - size_t replace_size = replace.size; Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative @@ -343,16 +352,22 @@ private: } size_t rows = input_offsets.size(); + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { size_t input_offset = input_offsets[i - 1]; size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; - if constexpr (!offset_is_const) + if constexpr (offset_is_const) + { + valid_offset = getValidOffset(const_offset, input_size); + } + else { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (!three_args && !length_is_const) @@ -361,7 +376,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -409,13 +424,10 @@ private: return; } - Int64 offset = 0; // start from 1, maybe negative + size_t input_size = input.size; size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } + valid_offset = getValidOffset(const_offset, input_size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative @@ -426,7 +438,7 @@ private: } size_t rows = replace_offsets.size(); - size_t input_size = input.size; + Int64 offset = 0; // start from 1, maybe negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { @@ -436,12 +448,11 @@ private: if constexpr (!offset_is_const) { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (three_args) { - // length = replace_size; valid_length = replace_size; } else if constexpr (!length_is_const) @@ -450,7 +461,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -507,15 +518,6 @@ private: return; } - - Int64 offset = 0; // start from 1, maybe negative - size_t valid_offset = 0; // start from 0, not negative - if constexpr (offset_is_const) - { - offset = const_offset; - valid_offset = offset > 0 ? (offset - 1) : (-offset); - } - Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative if constexpr (!three_args && length_is_const) @@ -525,6 +527,8 @@ private: } size_t rows = input_offsets.size(); + Int64 offset = 0; // start from 1, maybe negative + size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { @@ -533,15 +537,18 @@ private: size_t replace_offset = replace_offsets[i - 1]; size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; - if constexpr (!offset_is_const) + if constexpr (offset_is_const) + { + valid_offset = getValidOffset(const_offset, input_size); + } + else { offset = column_offset->getInt(i); - valid_offset = offset > 0 ? (offset - 1) : (-offset); + valid_offset = getValidOffset(offset, input_size); } if constexpr (three_args) { - // length = replace_size; valid_length = replace_size; } else if constexpr (!length_is_const) @@ -550,7 +557,7 @@ private: valid_length = length >= 0 ? length : replace_size; } - size_t prefix_size = valid_offset > input_size ? input_size : valid_offset; + size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator res_data.resize(new_res_size); From b42069cfa80e66ab59669bb7ccb93c2944d91170 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 04:27:11 +0200 Subject: [PATCH 0369/1722] Adjust some tests --- .../0_stateless/00111_shard_external_sort_distributed.sql | 6 +++--- .../0_stateless/00463_long_sessions_in_http_interface.sh | 2 +- tests/queries/0_stateless/00601_kill_running_query.sh | 2 +- tests/queries/0_stateless/00976_max_execution_speed.sql | 2 +- .../0_stateless/01119_optimize_trivial_insert_select.sql | 5 +++-- .../queries/0_stateless/01245_limit_infinite_sources.sql | 1 + tests/queries/0_stateless/01249_flush_interactive.sh | 4 ++-- tests/queries/0_stateless/01293_show_settings.reference | 1 + .../01301_aggregate_state_exception_memory_leak.sh | 2 +- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 2 +- .../0_stateless/02021_exponential_sum_shard.reference | 1 - tests/queries/0_stateless/02021_exponential_sum_shard.sql | 1 - tests/queries/0_stateless/02136_kill_scalar_queries.sh | 2 +- tests/queries/0_stateless/02293_ttest_large_samples.sql | 2 ++ .../02294_floating_point_second_in_settings.sh | 6 +++--- tests/queries/0_stateless/02343_aggregation_pipeline.sql | 8 +++----- .../0_stateless/02697_stop_reading_on_first_cancel.sh | 2 +- tests/queries/0_stateless/02700_s3_part_INT_MAX.sh | 4 +++- .../02896_max_execution_time_with_break_overflow_mode.sql | 2 ++ tests/queries/0_stateless/02915_sleep_large_uint.sql | 1 + ...p_virtual_columns_with_non_deterministic_functions.sql | 1 + 21 files changed, 32 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql index 112f5edae36..88a05f59111 100644 --- a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql +++ b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql @@ -1,10 +1,10 @@ -- Tags: distributed -SET max_memory_usage = 300000000; -SET max_bytes_before_external_sort = 20000000; +SET max_memory_usage = 150000000; +SET max_bytes_before_external_sort = 10000000; DROP TABLE IF EXISTS numbers10m; -CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 10000000; +CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 5000000; SELECT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20; diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh index d41d6409315..bb77a88820a 100755 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh @@ -74,7 +74,7 @@ ${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE t" echo "A session cannot be used by concurrent connections:" -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9&query_id=${CLICKHOUSE_DATABASE}_9" --data-binary "SELECT count() FROM system.numbers" >/dev/null & +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9&query_id=${CLICKHOUSE_DATABASE}_9&max_rows_to_read=0" --data-binary "SELECT count() FROM system.numbers" >/dev/null & # An infinite loop is required to make the test reliable. We will ensure that at least once the query on the line above has started before this check while true diff --git a/tests/queries/0_stateless/00601_kill_running_query.sh b/tests/queries/0_stateless/00601_kill_running_query.sh index 3163f8146d0..be0fff49129 100755 --- a/tests/queries/0_stateless/00601_kill_running_query.sh +++ b/tests/queries/0_stateless/00601_kill_running_query.sh @@ -11,7 +11,7 @@ function wait_for_query_to_start() while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done } -${CLICKHOUSE_CURL_COMMAND} -q --max-time 30 -sS "$CLICKHOUSE_URL&query_id=test_00601_$CLICKHOUSE_DATABASE" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(50000000) GROUP BY k)' > /dev/null & +${CLICKHOUSE_CURL_COMMAND} -q --max-time 30 -sS "$CLICKHOUSE_URL&query_id=test_00601_$CLICKHOUSE_DATABASE" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(50000000) GROUP BY k) SETTINGS max_rows_to_read = 0' > /dev/null & wait_for_query_to_start "test_00601_$CLICKHOUSE_DATABASE" $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = 'test_00601_$CLICKHOUSE_DATABASE'" wait diff --git a/tests/queries/0_stateless/00976_max_execution_speed.sql b/tests/queries/0_stateless/00976_max_execution_speed.sql index 52c3f05ff43..41374712724 100644 --- a/tests/queries/0_stateless/00976_max_execution_speed.sql +++ b/tests/queries/0_stateless/00976_max_execution_speed.sql @@ -1,2 +1,2 @@ -SET max_execution_speed = 1, max_execution_time = 3; +SET max_execution_speed = 1, max_execution_time = 3, max_rows_to_read = 0; SELECT count() FROM system.numbers; -- { serverError TIMEOUT_EXCEEDED } diff --git a/tests/queries/0_stateless/01119_optimize_trivial_insert_select.sql b/tests/queries/0_stateless/01119_optimize_trivial_insert_select.sql index a53b60a5ad3..2b301d7aced 100644 --- a/tests/queries/0_stateless/01119_optimize_trivial_insert_select.sql +++ b/tests/queries/0_stateless/01119_optimize_trivial_insert_select.sql @@ -1,8 +1,9 @@ drop table if exists t; create table t(n int, a Int64, s String) engine = MergeTree() order by a; -set enable_positional_arguments=0; -set optimize_trivial_insert_select=1; +set enable_positional_arguments = 0; +set optimize_trivial_insert_select = 1; +set max_rows_to_read = 0; -- due to aggregate functions, optimize_trivial_insert_select will not be applied insert into t select 1, sum(number) as c, getSetting('max_threads') from numbers_mt(100000000) settings max_insert_threads=4, max_threads=2; diff --git a/tests/queries/0_stateless/01245_limit_infinite_sources.sql b/tests/queries/0_stateless/01245_limit_infinite_sources.sql index 05680d86a33..69c93baf8a8 100644 --- a/tests/queries/0_stateless/01245_limit_infinite_sources.sql +++ b/tests/queries/0_stateless/01245_limit_infinite_sources.sql @@ -9,3 +9,4 @@ FROM ) WHERE number = 1 LIMIT 1 +SETTINGS max_rows_to_read = 0; diff --git a/tests/queries/0_stateless/01249_flush_interactive.sh b/tests/queries/0_stateless/01249_flush_interactive.sh index 551e11c8c8d..775b7825a16 100755 --- a/tests/queries/0_stateless/01249_flush_interactive.sh +++ b/tests/queries/0_stateless/01249_flush_interactive.sh @@ -14,10 +14,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test() { - timeout 5 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --query " + timeout 5 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --max_rows_to_read 0 --query " SELECT DISTINCT number % 5 FROM system.numbers" ||: echo -e '---' - timeout 5 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10" --data-binary " + timeout 5 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10&max_rows_to_read=0" --data-binary " SELECT DISTINCT number % 5 FROM system.numbers" ||: echo -e '---' } diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index 187f55697e4..9d326f16a3b 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -5,5 +5,6 @@ connect_timeout_with_failover_secure_ms Milliseconds 3000 external_storage_connect_timeout_sec UInt64 10 s3_connect_timeout_ms UInt64 1000 filesystem_prefetch_max_memory_usage UInt64 1073741824 +max_memory_usage UInt64 5000000000 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 47fe7a9c7d9..9dd800ceb09 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) start=$SECONDS # If the memory leak exists, it will lead to OOM fairly quickly. for _ in {1..1000}; do - $CLICKHOUSE_CLIENT --max_memory_usage 1G <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10"; + $CLICKHOUSE_CLIENT --max_memory_usage 1G --max_rows_to_read 0 <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10"; # NOTE: we cannot use timeout here since this will not guarantee that the query will be executed at least once. # (since graceful wait of clickhouse-client had been reverted) diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index 1cf52c0288b..ec14f637c01 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -9,7 +9,7 @@ create table t (x UInt64, s String) engine = MergeTree order by x SETTINGS index INSERT INTO t SELECT number, if(number < (8129 * 1024), arrayStringConcat(arrayMap(x -> toString(x), range(number % 128)), ' '), '') -FROM numbers_mt((8129 * 1024) * 3) settings max_insert_threads=8; +FROM numbers_mt((8129 * 1024) * 3) settings max_insert_threads=8, max_rows_to_read=0; -- optimize table t final; diff --git a/tests/queries/0_stateless/02021_exponential_sum_shard.reference b/tests/queries/0_stateless/02021_exponential_sum_shard.reference index 8453706a05a..c28e5d7a132 100644 --- a/tests/queries/0_stateless/02021_exponential_sum_shard.reference +++ b/tests/queries/0_stateless/02021_exponential_sum_shard.reference @@ -2,4 +2,3 @@ 0.009775171065493644 0.009775171065493644 0.009775171065493644 -0.009775171065493644 diff --git a/tests/queries/0_stateless/02021_exponential_sum_shard.sql b/tests/queries/0_stateless/02021_exponential_sum_shard.sql index 49fde0fe217..8e91637e41d 100644 --- a/tests/queries/0_stateless/02021_exponential_sum_shard.sql +++ b/tests/queries/0_stateless/02021_exponential_sum_shard.sql @@ -3,4 +3,3 @@ WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1) WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000)); WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(100000)); WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(1000000)); -WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000000)); diff --git a/tests/queries/0_stateless/02136_kill_scalar_queries.sh b/tests/queries/0_stateless/02136_kill_scalar_queries.sh index c8691b62360..f8bd5a42756 100755 --- a/tests/queries/0_stateless/02136_kill_scalar_queries.sh +++ b/tests/queries/0_stateless/02136_kill_scalar_queries.sh @@ -10,7 +10,7 @@ function wait_for_query_to_start() } QUERY_1_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY1" -(${CLICKHOUSE_CLIENT} --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." || echo 'FAIL') & +(${CLICKHOUSE_CLIENT} --max_rows_to_read 0 --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." || echo 'FAIL') & wait_for_query_to_start "${QUERY_1_ID}" ${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC" diff --git a/tests/queries/0_stateless/02293_ttest_large_samples.sql b/tests/queries/0_stateless/02293_ttest_large_samples.sql index 14baa3fddfe..826bd483fe9 100644 --- a/tests/queries/0_stateless/02293_ttest_large_samples.sql +++ b/tests/queries/0_stateless/02293_ttest_large_samples.sql @@ -15,6 +15,8 @@ SELECT FROM system.numbers limit 500000)); +SET max_rows_to_read = 0; + SELECT roundBankers(result.1, 5), roundBankers(result.2, 5 ) FROM ( SELECT studentTTest(sample, variant) as result diff --git a/tests/queries/0_stateless/02294_floating_point_second_in_settings.sh b/tests/queries/0_stateless/02294_floating_point_second_in_settings.sh index 7a18b8fea29..27dbd3e3de6 100755 --- a/tests/queries/0_stateless/02294_floating_point_second_in_settings.sh +++ b/tests/queries/0_stateless/02294_floating_point_second_in_settings.sh @@ -23,16 +23,16 @@ function check_output() { # TCP CLIENT echo "TCP CLIENT" -OUTPUT=$($CLICKHOUSE_CLIENT --max_execution_time $MAX_TIMEOUT -q "SELECT count() FROM system.numbers" 2>&1 || true) +OUTPUT=$($CLICKHOUSE_CLIENT --max_rows_to_read 0 --max_execution_time $MAX_TIMEOUT -q "SELECT count() FROM system.numbers" 2>&1 || true) check_output "${OUTPUT}" echo "TCP CLIENT WITH SETTINGS IN QUERY" -OUTPUT=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.numbers SETTINGS max_execution_time=$MAX_TIMEOUT" 2>&1 || true) +OUTPUT=$($CLICKHOUSE_CLIENT --max_rows_to_read 0 -q "SELECT count() FROM system.numbers SETTINGS max_execution_time=$MAX_TIMEOUT" 2>&1 || true) check_output "${OUTPUT}" # HTTP CLIENT echo "HTTP CLIENT" -OUTPUT=$(${CLICKHOUSE_CURL_COMMAND} -q -sS "$CLICKHOUSE_URL&max_execution_time=$MAX_TIMEOUT" -d \ +OUTPUT=$(${CLICKHOUSE_CURL_COMMAND} -q -sS "$CLICKHOUSE_URL&max_execution_time=${MAX_TIMEOUT}&max_rows_to_read=0" -d \ "SELECT count() FROM system.numbers" || true) check_output "${OUTPUT}" diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.sql b/tests/queries/0_stateless/02343_aggregation_pipeline.sql index 0f9dbd0247d..24d54293313 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.sql +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.sql @@ -13,11 +13,9 @@ set allow_prefetched_read_pool_for_local_filesystem = 0; -- { echoOn } -explain pipeline select * from (select * from numbers(1e8) group by number) group by number; - -explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number; - -explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number; +explain pipeline select * from (select * from numbers(1e8) group by number) group by number settings max_rows_to_read = 0; +explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number settings max_rows_to_read = 0; +explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number settings max_rows_to_read = 0; explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 1; diff --git a/tests/queries/0_stateless/02697_stop_reading_on_first_cancel.sh b/tests/queries/0_stateless/02697_stop_reading_on_first_cancel.sh index 2be13588453..5a2cec08eca 100755 --- a/tests/queries/0_stateless/02697_stop_reading_on_first_cancel.sh +++ b/tests/queries/0_stateless/02697_stop_reading_on_first_cancel.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) QUERY_ID="${CLICKHOUSE_DATABASE}_read_with_cancel" -$CLICKHOUSE_CLIENT -n --query_id="$QUERY_ID" --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true;" & +$CLICKHOUSE_CLIENT --max_rows_to_read 0 -n --query_id="$QUERY_ID" --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true;" & pid=$! for _ in {0..60} diff --git a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh index a34a480a078..c431686b594 100755 --- a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh +++ b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh @@ -10,7 +10,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # NOTE: .sh test is used over .sql because it needs $CLICKHOUSE_DATABASE to # avoid truncation, since seems that the version of MinIO that is used on CI # too slow with this. -$CLICKHOUSE_CLIENT -nm -q " +# +# Unfortunately, the test has to buffer it in memory. +$CLICKHOUSE_CLIENT --max_memory_usage 10G -nm -q " INSERT INTO FUNCTION s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv', '', '', 'TSV') SELECT repeat('a', 1024) FROM numbers((pow(2, 30) * 2) / 1024) SETTINGS s3_max_single_part_upload_size = '5Gi'; diff --git a/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql index ec86a66c7dd..3e131cad0f0 100644 --- a/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql +++ b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest +SET max_rows_to_read = 0; + -- Query stops after timeout without an error SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=2, timeout_overflow_mode='break' FORMAT Null; diff --git a/tests/queries/0_stateless/02915_sleep_large_uint.sql b/tests/queries/0_stateless/02915_sleep_large_uint.sql index f7c04ab6d1f..08b6c580a28 100644 --- a/tests/queries/0_stateless/02915_sleep_large_uint.sql +++ b/tests/queries/0_stateless/02915_sleep_large_uint.sql @@ -1,6 +1,7 @@ SELECT sleep(3.40282e+44); -- { serverError BAD_ARGUMENTS } SELECT sleep((pow(2, 64) / 1000000) - 1); -- { serverError BAD_ARGUMENTS } SELECT sleepEachRow(184467440737095516) from numbers(10000); -- { serverError BAD_ARGUMENTS } +SET max_rows_to_read = 0; SELECT sleepEachRow(pow(2, 31)) from numbers(9007199254740992) settings function_sleep_max_microseconds_per_block = 8589934592000000000; -- { serverError TOO_SLOW } -- Another corner case, but it requires lots of memory to run (huge block size) diff --git a/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql index 8ccc3cf61da..6ef8c5a8656 100644 --- a/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql +++ b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql @@ -1,3 +1,4 @@ +SET max_rows_to_read = 0; create table test (number UInt64) engine=MergeTree order by number; insert into test select * from numbers(50000000); select ignore(number) from test where RAND() > 4292390314 limit 10; From 016888e29a828870d5cdb50a0eb5e1514bafc97c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 04:32:20 +0200 Subject: [PATCH 0370/1722] Adjust some tests --- tests/queries/1_stateful/00067_union_all.sql | 3 ++- .../00088_global_in_one_shard_and_rows_before_limit.sql | 2 +- .../queries/1_stateful/00147_global_in_aggregate_function.sql | 1 + .../queries/1_stateful/00149_quantiles_timing_distributed.sql | 1 + tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 1 + .../1_stateful/00171_grouping_aggregated_transform_bug.sql | 1 + .../1_stateful/00182_simple_squashing_transform_bug.sql | 1 + 7 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00067_union_all.sql b/tests/queries/1_stateful/00067_union_all.sql index 2a1d00e975d..9ee14b36b03 100644 --- a/tests/queries/1_stateful/00067_union_all.sql +++ b/tests/queries/1_stateful/00067_union_all.sql @@ -10,4 +10,5 @@ UNION ALL ORDER BY id DESC LIMIT 10 ) -ORDER BY id, event; +ORDER BY id, event +SETTINGS max_rows_to_read = 40_000_000; diff --git a/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql b/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql index 52f9c46997f..443808e7bed 100644 --- a/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql +++ b/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql @@ -1,4 +1,4 @@ -- Tags: shard -SET output_format_write_statistics = 0; +SET output_format_write_statistics = 0, max_rows_to_read = 20_000_000; SELECT EventDate, count() FROM remote('127.0.0.1', test.hits) WHERE UserID GLOBAL IN (SELECT UserID FROM test.hits) GROUP BY EventDate ORDER BY EventDate LIMIT 5 FORMAT JSONCompact; diff --git a/tests/queries/1_stateful/00147_global_in_aggregate_function.sql b/tests/queries/1_stateful/00147_global_in_aggregate_function.sql index 075c01530c6..c156f073573 100644 --- a/tests/queries/1_stateful/00147_global_in_aggregate_function.sql +++ b/tests/queries/1_stateful/00147_global_in_aggregate_function.sql @@ -1,4 +1,5 @@ -- Tags: global +SET max_rows_to_read = 40_000_000; SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{1,2}', test.hits))) FROM remote('127.0.0.{1,2}', test.hits); SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.hits)) FROM remote('127.0.0.{1,2}', test.hits); diff --git a/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql b/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql index 6f910646fb7..16b565985ea 100644 --- a/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql +++ b/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql @@ -1,4 +1,5 @@ -- Tags: distributed +SET max_rows_to_read = 100_000_000; SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID); SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS optimize_aggregation_in_order = 1; diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index 7b3f50f8141..1a98a531067 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -1,5 +1,6 @@ -- Tags: no-random-settings +SET max_memory_usage = '10G' SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40; -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. diff --git a/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql b/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql index 7068780a1b1..b3e4d749328 100644 --- a/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql +++ b/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql @@ -1,4 +1,5 @@ -- Tags: distributed +SET max_rows_to_read = '100M'; SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS max_block_size = 63169; SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS optimize_aggregation_in_order = 1, max_block_size = 63169; diff --git a/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql b/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql index e73de4b33fb..85bad651090 100644 --- a/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql +++ b/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql @@ -1,6 +1,7 @@ -- Tags: global set allow_prefetched_read_pool_for_remote_filesystem=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0, max_threads=2, max_block_size=65387; +set max_rows_to_read = '20M'; SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{1,2}', test.hits))) FROM remote('127.0.0.{1,2}', test.hits); SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.hits)) FROM remote('127.0.0.{1,2}', test.hits); From e569a305ba07225ee641ae4af07ba9c88e4608d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 04:46:31 +0200 Subject: [PATCH 0371/1722] Adjust some tests --- .../00002_log_and_exception_messages_formatting.sql | 2 ++ .../0_stateless/00375_shard_group_uniq_array_of_string.sql | 2 +- .../00376_shard_group_uniq_array_of_int_array.sql | 2 +- .../00377_shard_group_uniq_array_of_string_array.sql | 2 +- tests/queries/0_stateless/00600_replace_running_query.sh | 6 +++--- .../00834_cancel_http_readonly_queries_on_client_close.sh | 2 +- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 2 +- tests/queries/0_stateless/01304_direct_io_long.sh | 4 ++-- tests/queries/0_stateless/02021_exponential_sum.reference | 1 - tests/queries/0_stateless/02021_exponential_sum.sql | 1 - tests/queries/0_stateless/02234_cast_to_ip_address.sql | 2 +- .../0_stateless/02450_kill_distributed_query_deadlock.sh | 2 +- tests/queries/0_stateless/02585_query_status_deadlock.sh | 3 +-- tests/queries/0_stateless/02786_max_execution_time_leaf.sql | 1 + .../0_stateless/02844_subquery_timeout_with_break.sql | 2 +- tests/queries/0_stateless/02916_glogal_in_cancel.sql | 2 +- 16 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 07c42d6d039..c158406c0da 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -7,6 +7,8 @@ system flush logs; drop table if exists logs; create view logs as select * from system.text_log where now() - toIntervalMinute(120) < event_time; +SET max_rows_to_read = 0; + -- Check that we don't have too many messages formatted with fmt::runtime or strings concatenation. -- 0.001 threshold should be always enough, the value was about 0.00025 WITH 0.001 AS threshold diff --git a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql index 8a310cb8fc9..f32a64cd30f 100644 --- a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql +++ b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql @@ -7,7 +7,7 @@ INSERT INTO group_uniq_str SELECT 2 as id, toString(number % 100) as v FROM syst INSERT INTO group_uniq_str SELECT 5 as id, toString(number % 100) as v FROM system.numbers LIMIT 10000000; SELECT length(groupUniqArray(v)) FROM group_uniq_str GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_str') GROUP BY id ORDER BY id; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '50M'; SELECT length(groupUniqArray(10)(v)) FROM group_uniq_str GROUP BY id ORDER BY id; SELECT length(groupUniqArray(10000)(v)) FROM group_uniq_str GROUP BY id ORDER BY id; diff --git a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql index abd0e6e6a45..43066880102 100644 --- a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql +++ b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql @@ -6,7 +6,7 @@ CREATE TABLE group_uniq_arr_int ENGINE = Memory AS (SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000); SELECT length(groupUniqArray(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '50M'; SELECT length(groupUniqArray(10)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; SELECT length(groupUniqArray(100000)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; diff --git a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql index e9cfff211f8..1c4376ad577 100644 --- a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql +++ b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql @@ -6,6 +6,6 @@ CREATE TABLE group_uniq_arr_str ENGINE = Memory AS (SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000); SELECT length(groupUniqArray(v)) FROM group_uniq_arr_str GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '50M'; DROP TABLE IF EXISTS group_uniq_arr_str; diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index 7a71d17f19b..e7022875086 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -17,7 +17,7 @@ function wait_for_query_to_start() } -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d 'SELECT 1, count() FROM system.numbers' > /dev/null 2>&1 & +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1&max_rows_to_read=0" -d 'SELECT 1, count() FROM system.numbers' > /dev/null 2>&1 & wait_for_query_to_start 'hello' # Replace it @@ -26,7 +26,7 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d # Wait for it to be replaced wait -${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & +${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --max_rows_to_read=0 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' # Trying to run another query with the same query_id @@ -38,7 +38,7 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=42&replace_running_query=1" -d 'S $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" > /dev/null wait -${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & +${CLICKHOUSE_CLIENT} --query_id=42 --max_rows_to_read=0 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null wait diff --git a/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh b/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh index 5c21c70e06a..dd3735f27b1 100755 --- a/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh +++ b/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} --max-time 1 -sS "${CLICKHOUSE_URL}&query_id=cancel_http_readonly_queries_on_client_close&cancel_http_readonly_queries_on_client_close=1&query=SELECT+count()+FROM+system.numbers" 2>&1 | grep -cF 'curl: (28)' +${CLICKHOUSE_CURL} --max-time 1 -sS "${CLICKHOUSE_URL}&query_id=cancel_http_readonly_queries_on_client_close&cancel_http_readonly_queries_on_client_close=1&max_rows_to_read=0&query=SELECT+count()+FROM+system.numbers" 2>&1 | grep -cF 'curl: (28)' i=0 retries=300 while [[ $i -lt $retries ]]; do diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index 55eacd0db44..15a53841761 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,5 +1,5 @@ drop table if exists lc_00906; create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -insert into lc_00906 select '0123456789' from numbers(100000000); +insert into lc_00906 select '0123456789' from numbers(100000000) SETTINGS max_rows_to_read = '100M'; select count(), b from lc_00906 group by b; drop table if exists lc_00906; diff --git a/tests/queries/0_stateless/01304_direct_io_long.sh b/tests/queries/0_stateless/01304_direct_io_long.sh index 2e27c2f7728..a66239058ab 100755 --- a/tests/queries/0_stateless/01304_direct_io_long.sh +++ b/tests/queries/0_stateless/01304_direct_io_long.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --max_rows_to_read 50M --multiquery " DROP TABLE IF EXISTS bug; CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi', merge_max_block_size = 8192; @@ -18,5 +18,5 @@ cat "$LOG" | grep Loaded rm "$LOG" -$CLICKHOUSE_CLIENT --multiquery --query " +$CLICKHOUSE_CLIENT --multiquery " DROP TABLE bug;" diff --git a/tests/queries/0_stateless/02021_exponential_sum.reference b/tests/queries/0_stateless/02021_exponential_sum.reference index 5bd77479cf7..c9dcee51173 100644 --- a/tests/queries/0_stateless/02021_exponential_sum.reference +++ b/tests/queries/0_stateless/02021_exponential_sum.reference @@ -5,4 +5,3 @@ 0.0009775171065493646 0.0009775171065493646 0.0009775171065493646 -0.0009775171065493646 diff --git a/tests/queries/0_stateless/02021_exponential_sum.sql b/tests/queries/0_stateless/02021_exponential_sum.sql index 8ab7638029c..62ec7dcf9f1 100644 --- a/tests/queries/0_stateless/02021_exponential_sum.sql +++ b/tests/queries/0_stateless/02021_exponential_sum.sql @@ -6,4 +6,3 @@ WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1) WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(100000); WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(1000000); WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(10000000); -WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(100000000); diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.sql b/tests/queries/0_stateless/02234_cast_to_ip_address.sql index 28f1afff57f..51e953da905 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.sql +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.sql @@ -67,7 +67,7 @@ SELECT toIPv6('::.1.2.3'); --{serverError CANNOT_PARSE_IPV6} SELECT toIPv6OrDefault('::.1.2.3'); SELECT toIPv6OrNull('::.1.2.3'); -SELECT count() FROM numbers_mt(100000000) WHERE NOT ignore(toIPv6OrZero(randomString(8))); +SELECT count() FROM numbers_mt(20000000) WHERE NOT ignore(toIPv6OrZero(randomString(8))); SELECT '--'; diff --git a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh index 0cd520d8d5d..445f907bcc5 100755 --- a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh +++ b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # this can trigger a hung/deadlock in ProcessorList. for i in {1..50}; do query_id="$CLICKHOUSE_TEST_UNIQUE_NAME-$i" - $CLICKHOUSE_CLIENT --format Null --query_id "$query_id" -q "select * from remote('127.{1|2|3|4|5|6}', numbers(1e12))" 2>/dev/null & + $CLICKHOUSE_CLIENT --format Null --query_id "$query_id" --max_rows_to_read 0 -q "select * from remote('127.{1|2|3|4|5|6}', numbers(1e12))" 2>/dev/null & while :; do killed_queries="$($CLICKHOUSE_CLIENT -q "kill query where query_id = '$query_id' sync" | wc -l)" if [[ "$killed_queries" -ge 1 ]]; then diff --git a/tests/queries/0_stateless/02585_query_status_deadlock.sh b/tests/queries/0_stateless/02585_query_status_deadlock.sh index e3e34109cdb..6321ac0064a 100755 --- a/tests/queries/0_stateless/02585_query_status_deadlock.sh +++ b/tests/queries/0_stateless/02585_query_status_deadlock.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) QUERY_ID="${CLICKHOUSE_DATABASE}_test_02585_query_to_kill_id_1" -$CLICKHOUSE_CLIENT --query_id="$QUERY_ID" -n -q " +$CLICKHOUSE_CLIENT --query_id="$QUERY_ID" --max_rows_to_read 0 -n -q " create temporary table tmp as select * from numbers(500000000); select * from remote('127.0.0.2', 'system.numbers_mt') where number in (select * from tmp);" &> /dev/null & @@ -23,4 +23,3 @@ do done $CLICKHOUSE_CLIENT -q "kill query where query_id = '$QUERY_ID' sync" &> /dev/null - diff --git a/tests/queries/0_stateless/02786_max_execution_time_leaf.sql b/tests/queries/0_stateless/02786_max_execution_time_leaf.sql index f678c913b46..2e4623f4ac6 100644 --- a/tests/queries/0_stateless/02786_max_execution_time_leaf.sql +++ b/tests/queries/0_stateless/02786_max_execution_time_leaf.sql @@ -1,4 +1,5 @@ -- Tags: no-fasttest +SET max_rows_to_read = 0; SELECT count() FROM cluster('test_cluster_two_shards', view( SELECT * FROM numbers(100000000000) )) SETTINGS max_execution_time_leaf = 1; -- { serverError TIMEOUT_EXCEEDED } -- Can return partial result SELECT count() FROM cluster('test_cluster_two_shards', view( SELECT * FROM numbers(100000000000) )) FORMAT Null SETTINGS max_execution_time_leaf = 1, timeout_overflow_mode_leaf = 'break'; diff --git a/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql b/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql index 511ed0c59de..00b527a9378 100644 --- a/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql +++ b/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql @@ -4,7 +4,7 @@ CREATE TABLE t (key UInt64, value UInt64, INDEX value_idx value TYPE bloom_filte INSERT INTO t SELECT number, rand()%1000 FROM numbers(10000); SET timeout_overflow_mode='break'; -SET max_execution_time=0.1; +SET max_execution_time=0.1, max_rows_to_read=0; SELECT * FROM t WHERE value IN (SELECT number FROM numbers(1000000000)); DROP TABLE t; diff --git a/tests/queries/0_stateless/02916_glogal_in_cancel.sql b/tests/queries/0_stateless/02916_glogal_in_cancel.sql index ad54f1ecdec..dd61795947a 100644 --- a/tests/queries/0_stateless/02916_glogal_in_cancel.sql +++ b/tests/queries/0_stateless/02916_glogal_in_cancel.sql @@ -1,2 +1,2 @@ -set max_execution_time = 0.5, timeout_overflow_mode = 'break'; +set max_execution_time = 0.5, timeout_overflow_mode = 'break', max_rows_to_read = 0; SELECT number FROM remote('127.0.0.{3|2}', numbers(1)) WHERE number GLOBAL IN (SELECT number FROM numbers(10000000000.)) format Null; From 26650dcb2e39b0d28cae4029b6adde2b6c01fda2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 04:59:13 +0200 Subject: [PATCH 0372/1722] More limits --- tests/config/install.sh | 2 +- tests/config/users.d/limits.xml | 8 ----- tests/config/users.d/limits.yaml | 57 ++++++++++++++++++++++++++++++++ 3 files changed, 58 insertions(+), 9 deletions(-) delete mode 100644 tests/config/users.d/limits.xml create mode 100644 tests/config/users.d/limits.yaml diff --git a/tests/config/install.sh b/tests/config/install.sh index 265b9248f4a..c5fb3cc92c7 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -93,7 +93,7 @@ ln -sf $SRC_PATH/users.d/prefetch_settings.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/nonconst_timezone.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/allow_introspection_functions.yaml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/replicated_ddl_entry.xml $DEST_SERVER_PATH/users.d/ -ln -sf $SRC_PATH/users.d/limits.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/limits.yaml $DEST_SERVER_PATH/users.d/ if [[ -n "$USE_OLD_ANALYZER" ]] && [[ "$USE_OLD_ANALYZER" -eq 1 ]]; then ln -sf $SRC_PATH/users.d/analyzer.xml $DEST_SERVER_PATH/users.d/ diff --git a/tests/config/users.d/limits.xml b/tests/config/users.d/limits.xml deleted file mode 100644 index f44c73241ab..00000000000 --- a/tests/config/users.d/limits.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - 5G - 20000000 - - - diff --git a/tests/config/users.d/limits.yaml b/tests/config/users.d/limits.yaml new file mode 100644 index 00000000000..4f3f439a997 --- /dev/null +++ b/tests/config/users.d/limits.yaml @@ -0,0 +1,57 @@ +profiles: + default: + max_memory_usage: 5G + max_rows_to_read: 20000000 + + # Also set every other limit to a high value, so it will not limit anything, but we will test that code around it. + s3_max_get_rps: 1000000 + s3_max_get_burst: 2000000 + s3_max_put_rps: 1000000 + s3_max_put_burst: 2000000 + max_remote_read_network_bandwidth: 1T + max_remote_write_network_bandwidth: 1T + max_local_read_bandwidth: 1T + max_local_write_bandwidth: 1T + use_index_for_in_with_subqueries_max_values: 1G + max_bytes_to_read: 1T + max_bytes_to_read_leaf: 1T + max_rows_to_group_by: 10G + max_bytes_before_external_group_by: 10G + max_rows_to_sort: 10G + max_bytes_to_sort: 10G + max_bytes_before_external_sort: 10G + max_result_rows: 1G + max_result_bytes: 1G + max_execution_time: 600 + max_execution_time_leaf: 600 + max_execution_speed: 100G + max_execution_speed_bytes: 10T + max_estimated_execution_time: 600 + max_columns_to_read: 10K + max_temporary_columns: 10K + max_temporary_non_const_columns: 10K + max_sessions_for_user: 1K + max_rows_in_set: 10G + max_bytes_in_set: 10G + max_rows_in_join: 10G + max_bytes_in_join: 10G + max_rows_in_set_to_optimize_join: 1G + max_rows_to_transfer: 1G + max_bytes_to_transfer: 1G + max_rows_in_distinct: 10G + max_bytes_in_distinct: 10G + max_memory_usage_for_user: 10G + max_network_bandwidth: 100G + max_network_bytes: 1T + max_network_bandwidth_for_user: 100G + max_network_bandwidth_for_all_users: 100G + max_temporary_data_on_disk_size_for_user: 100G + max_temporary_data_on_disk_size_for_query: 100G + max_backup_bandwidth: 100G + max_hyperscan_regexp_length: 1M + max_hyperscan_regexp_total_length: 10M + query_cache_max_size_in_bytes: 10M + query_cache_max_entries: 100K + external_storage_max_read_rows: 10G + external_storage_max_read_bytes: 10G + max_streams_for_merge_tree_reading: 1000 From fd3f0cf92b7800b171c5723541a329748a0dad1b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Jul 2024 14:17:58 +0800 Subject: [PATCH 0373/1722] support overlayUTF8 --- src/Functions/FunctionOverlay.cpp | 281 +++++++++++++----- .../0_stateless/03205_overlay.reference | 168 +++++++++++ tests/queries/0_stateless/03205_overlay.sql | 60 ++++ .../0_stateless/03206_overlay_utf8.reference | 168 +++++++++++ .../0_stateless/03206_overlay_utf8.sql | 60 ++++ 5 files changed, 665 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/03205_overlay.reference create mode 100644 tests/queries/0_stateless/03205_overlay.sql create mode 100644 tests/queries/0_stateless/03206_overlay_utf8.reference create mode 100644 tests/queries/0_stateless/03206_overlay_utf8.sql diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/FunctionOverlay.cpp index d3ee7e1df6d..61d2df88ab1 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/FunctionOverlay.cpp @@ -3,8 +3,10 @@ #include #include #include +#include #include #include +#include namespace DB { @@ -15,6 +17,8 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +using namespace GatherUtils; + namespace { @@ -24,7 +28,7 @@ template class FunctionOverlay : public IFunction { public: - static constexpr auto name = is_utf8 ? "OverlayUTF8" : "Overlay"; + static constexpr auto name = is_utf8 ? "overlayUTF8" : "overlay"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -238,6 +242,15 @@ private: } } + /// get character count of a slice [data, data+bytes) + static size_t getSliceSize(const UInt8 * data, size_t bytes) + { + if constexpr (is_utf8) + return UTF8::countCodePoints(data, bytes); + else + return bytes; + } + template void constantConstant( size_t rows, @@ -257,13 +270,12 @@ private: return; } - size_t input_size = input.size; + size_t input_size = getSliceSize(reinterpret_cast(input.data), input.size); size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) valid_offset = getValidOffset(const_offset, input_size); - size_t replace_size = replace.size; - Int64 length = 0; // maybe negative + size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); size_t valid_length = 0; // not negative if constexpr (!three_args && length_is_const) { @@ -276,6 +288,9 @@ private: } Int64 offset = 0; // start from 1, maybe negative + Int64 length = 0; // maybe negative + const UInt8 * input_begin = reinterpret_cast(input.data); + const UInt8 * input_end = reinterpret_cast(input.data + input.size); size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { @@ -293,28 +308,57 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; + + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + + size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input_begin, prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace.size); + res_offset += replace.size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero. + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } @@ -338,7 +382,7 @@ private: return; } - size_t replace_size = replace.size; + size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative if constexpr (!three_args && length_is_const) @@ -358,7 +402,8 @@ private: for (size_t i = 0; i < rows; ++i) { size_t input_offset = input_offsets[i - 1]; - size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_bytes = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_size = getSliceSize(&input_data[input_offset], input_bytes); if constexpr (offset_is_const) { @@ -378,29 +423,59 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15( - &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * input_begin = &input_data[input_offset]; + const auto * input_end = &input_data[input_offset + input_bytes]; + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + + size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], replace.data, replace.size); + res_offset += replace.size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero. + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } @@ -424,7 +499,7 @@ private: return; } - size_t input_size = input.size; + size_t input_size = getSliceSize(reinterpret_cast(input.data), input.size); size_t valid_offset = 0; // start from 0, not negative if constexpr (offset_is_const) valid_offset = getValidOffset(const_offset, input_size); @@ -438,12 +513,15 @@ private: } size_t rows = replace_offsets.size(); + const auto * input_begin = reinterpret_cast(input.data); + const auto * input_end = reinterpret_cast(input.data + input.size); Int64 offset = 0; // start from 1, maybe negative size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { size_t replace_offset = replace_offsets[i - 1]; - size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_bytes = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_size = getSliceSize(&replace_data[replace_offset], replace_bytes); if constexpr (!offset_is_const) { @@ -463,28 +541,55 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data, prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data + prefix_size + valid_length, suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input_begin, prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_bytes); + res_offset += replace_bytes; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } @@ -533,9 +638,12 @@ private: for (size_t i = 0; i < rows; ++i) { size_t input_offset = input_offsets[i - 1]; - size_t input_size = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_bytes = input_offsets[i] - input_offsets[i - 1] - 1; + size_t input_size = getSliceSize(&input_data[input_offset], input_bytes); + size_t replace_offset = replace_offsets[i - 1]; - size_t replace_size = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_bytes = replace_offsets[i] - replace_offsets[i - 1] - 1; + size_t replace_size = getSliceSize(&replace_data[replace_offset], replace_bytes); if constexpr (offset_is_const) { @@ -559,29 +667,58 @@ private: size_t prefix_size = valid_offset; size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; - size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator - res_data.resize(new_res_size); - /// copy prefix before replaced region - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); - res_offset += prefix_size; - - /// copy replace - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); - res_offset += replace_size; - - /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. - if (suffix_size) + if constexpr (!is_utf8) { - memcpySmallAllowReadWriteOverflow15( - &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); - res_offset += suffix_size; + size_t new_res_size = res_data.size() + prefix_size + replace_size + suffix_size + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &input_data[input_offset], prefix_size); + res_offset += prefix_size; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_size); + res_offset += replace_size; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_size is zero. + if (suffix_size) + { + memcpySmallAllowReadWriteOverflow15( + &res_data[res_offset], &input_data[input_offset + prefix_size + valid_length], suffix_size); + res_offset += suffix_size; + } + } + else + { + const auto * input_begin = &input_data[input_offset]; + const auto * input_end = &input_data[input_offset + input_bytes]; + const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; + const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + size_t suffix_bytes = input_end - suffix_begin; + size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator + res_data.resize(new_res_size); + + /// copy prefix before replaced region + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input_begin, prefix_bytes); + res_offset += prefix_bytes; + + /// copy replace + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &replace_data[replace_offset], replace_bytes); + res_offset += replace_bytes; + + /// copy suffix after replaced region. It is not necessary to copy if suffix_bytes is zero. + if (suffix_bytes) + { + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], suffix_begin, suffix_bytes); + res_offset += suffix_bytes; + } } /// add zero terminator res_data[res_offset] = 0; ++res_offset; - res_offsets[i] = res_offset; } } diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference new file mode 100644 index 00000000000..9e79db2e131 --- /dev/null +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -0,0 +1,168 @@ +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark_SQL +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark CORE +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Spark ANSI SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL +Structured SQL diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql new file mode 100644 index 00000000000..b131312c934 --- /dev/null +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -0,0 +1,60 @@ +SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlay('Spark SQL', '_', 6) from numbers(3); +SELECT overlay(materialize('Spark SQL'), '_', 6) from numbers(3); +SELECT overlay('Spark SQL', materialize('_'), 6) from numbers(3); +SELECT overlay('Spark SQL', '_', materialize(6)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('_'), 6) from numbers(3); +SELECT overlay(materialize('Spark SQL'), '_', materialize(6)) from numbers(3); +SELECT overlay('Spark SQL', materialize('_'), materialize(6)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)) from numbers(3); + +SELECT overlay('Spark SQL', 'CORE', 7) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'CORE', 7) from numbers(3); +SELECT overlay('Spark SQL', materialize('CORE'), 7) from numbers(3); +SELECT overlay('Spark SQL', 'CORE', materialize(7)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)) from numbers(3); +SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)) from numbers(3); + +SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlay('Spark SQL', 'tructured', 2, 4) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4) from numbers(3); +SELECT overlay('Spark SQL', materialize('tructured'), 2, 4) from numbers(3); +SELECT overlay('Spark SQL', 'tructured', materialize(2), 4) from numbers(3); +SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4) from numbers(3); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4) from numbers(3); +SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)) from numbers(3); +SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)) from numbers(3); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); diff --git a/tests/queries/0_stateless/03206_overlay_utf8.reference b/tests/queries/0_stateless/03206_overlay_utf8.reference new file mode 100644 index 00000000000..19878c97184 --- /dev/null +++ b/tests/queries/0_stateless/03206_overlay_utf8.reference @@ -0,0 +1,168 @@ +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark_SQLå’ŒCH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark CORECH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Spark ANSI SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH +Structured SQLå’ŒCH diff --git a/tests/queries/0_stateless/03206_overlay_utf8.sql b/tests/queries/0_stateless/03206_overlay_utf8.sql new file mode 100644 index 00000000000..00b756c8b5b --- /dev/null +++ b/tests/queries/0_stateless/03206_overlay_utf8.sql @@ -0,0 +1,60 @@ +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlayUTF8('Spark SQLå’ŒCH', '_', 6) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), '_', 6) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('_'), 6) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', '_', materialize(6)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('_'), 6) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), '_', materialize(6)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('_'), materialize(6)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('_'), materialize(6)) from numbers(3); + +SELECT overlayUTF8('Spark SQLå’ŒCH', 'CORE', 7) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'CORE', 7) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('CORE'), 7) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'CORE', materialize(7)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('CORE'), 7) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'CORE', materialize(7)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('CORE'), materialize(7)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('CORE'), materialize(7)) from numbers(3); + +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), 0) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), 0) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, materialize(0)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); + +SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', 2, 4) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', 2, 4) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), 2, 4) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', materialize(2), 4) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('tructured'), 2, 4) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', materialize(2), 4) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', 2, materialize(4)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), materialize(2), 4) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), 2, materialize(4)) from numbers(3); +SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', materialize(2), materialize(4)) from numbers(3); +SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); From c09c22b17575396e38fb45cb385dcc8a49f9a183 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Jul 2024 14:45:47 +0800 Subject: [PATCH 0374/1722] finish doc --- .../functions/string-replace-functions.md | 72 +++++++++++++++++++ ...new_functions_must_be_documented.reference | 2 + 2 files changed, 74 insertions(+) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 8793ebdd1a3..4e1f89fd974 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -223,3 +223,75 @@ SELECT translateUTF8('Münchener Straße', 'üß', 'us') AS res; │ Munchener Strase │ └──────────────────┘ ``` + +## overlay + +Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. + +**Syntax** + +```sql +overlay(s, replace, position[, length]) +``` + +**Parameters** + +- `s`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `position`: An integer type [Int](../data-types/int.md). +- `length`: Optional. An integer type [Int](../data-types/int.md). + +**Returned value** + +- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. + +**Example** + +```sql +SELECT overlay('Spark SQL', 'CORE', 7) AS res; +``` + +Result: + +```text + ┌─res────────┠+ │ Spark CORE │ + └────────────┘ +``` + +## overlayUTF8 + +Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +overlayUTF8(s, replace, position[, length]) +``` + +**Parameters** + +- `s`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `position`: An integer type [Int](../data-types/int.md). +- `length`: Optional. An integer type [Int](../data-types/int.md). + +**Returned value** + +- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. + +**Example** + +```sql +SELECT overlayUTF8('ClickHouse是一款OLAPæ•°æ®åº“', 'å¼€æº', 12, 2) AS res; +``` + +Result: + +```text +┌─res────────────────────────┠+│ ClickHouse是开æºOLAPæ•°æ®åº“ │ +└────────────────────────────┘ +``` diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index a152066a460..ba9d3fb7a83 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -512,6 +512,8 @@ nullIf nullIn nullInIgnoreSet or +overlay +overlayUTF8 parseDateTime parseDateTime32BestEffort parseDateTime32BestEffortOrNull From 63e586d17af1e8a92fc4d2e8af71f1dba4996fea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 09:57:56 +0200 Subject: [PATCH 0375/1722] Adjust tests --- tests/config/users.d/limits.yaml | 6 +-- ..._shard_external_sort_distributed.reference | 40 +++++++++---------- .../00111_shard_external_sort_distributed.sql | 2 +- ...00375_shard_group_uniq_array_of_string.sql | 2 +- ...76_shard_group_uniq_array_of_int_array.sql | 2 +- ...shard_group_uniq_array_of_string_array.sql | 2 +- .../00600_replace_running_query.sh | 4 +- .../00601_kill_running_query.reference | 2 +- .../00906_low_cardinality_cache.sql | 3 +- .../01091_query_profiler_does_not_hang.sql | 2 +- .../0_stateless/01293_show_settings.reference | 1 + .../0_stateless/01485_256_bit_multiply.sql | 2 + .../01603_read_with_backoff_bug.sql | 1 + .../01961_roaring_memory_tracking.sql | 2 +- .../02003_memory_limit_in_client.sh | 6 +-- .../02161_addressToLineWithInlines.sql | 2 +- .../02226_analyzer_or_like_combine.sql | 2 + .../02234_cast_to_ip_address.reference | 8 ++-- .../02343_aggregation_pipeline.reference | 6 +-- .../02353_simdjson_buffer_overflow.sql | 1 + .../0_stateless/02372_now_in_block.sql | 1 + .../02536_delta_gorilla_corruption.sql | 2 +- 22 files changed, 54 insertions(+), 45 deletions(-) diff --git a/tests/config/users.d/limits.yaml b/tests/config/users.d/limits.yaml index 4f3f439a997..1c9fff9f4c8 100644 --- a/tests/config/users.d/limits.yaml +++ b/tests/config/users.d/limits.yaml @@ -27,9 +27,9 @@ profiles: max_execution_speed: 100G max_execution_speed_bytes: 10T max_estimated_execution_time: 600 - max_columns_to_read: 10K - max_temporary_columns: 10K - max_temporary_non_const_columns: 10K + max_columns_to_read: 20K + max_temporary_columns: 20K + max_temporary_non_const_columns: 20K max_sessions_for_user: 1K max_rows_in_set: 10G max_bytes_in_set: 10G diff --git a/tests/queries/0_stateless/00111_shard_external_sort_distributed.reference b/tests/queries/0_stateless/00111_shard_external_sort_distributed.reference index df5aa77af60..7534c12a0d8 100644 --- a/tests/queries/0_stateless/00111_shard_external_sort_distributed.reference +++ b/tests/queries/0_stateless/00111_shard_external_sort_distributed.reference @@ -1,20 +1,20 @@ -7040546 -7040546 -4327029 -4327029 -1613512 -1613512 -8947307 -8947307 -6233790 -6233790 -3520273 -3520273 -806756 -806756 -8140551 -8140551 -5427034 -5427034 -2713517 -2713517 +4437158 +4437158 +1723641 +1723641 +3630402 +3630402 +916885 +916885 +2823646 +2823646 +110129 +110129 +4730407 +4730407 +2016890 +2016890 +3923651 +3923651 +1210134 +1210134 diff --git a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql index 88a05f59111..ef9c0f9f9d0 100644 --- a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql +++ b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql @@ -6,6 +6,6 @@ SET max_bytes_before_external_sort = 10000000; DROP TABLE IF EXISTS numbers10m; CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 5000000; -SELECT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20; +SELECT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers10m) ORDER BY number * 1234567890123456789 LIMIT 4999980, 20; DROP TABLE numbers10m; diff --git a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql index f32a64cd30f..445ffe66f64 100644 --- a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql +++ b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql @@ -7,7 +7,7 @@ INSERT INTO group_uniq_str SELECT 2 as id, toString(number % 100) as v FROM syst INSERT INTO group_uniq_str SELECT 5 as id, toString(number % 100) as v FROM system.numbers LIMIT 10000000; SELECT length(groupUniqArray(v)) FROM group_uniq_str GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '50M'; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '60M'; SELECT length(groupUniqArray(10)(v)) FROM group_uniq_str GROUP BY id ORDER BY id; SELECT length(groupUniqArray(10000)(v)) FROM group_uniq_str GROUP BY id ORDER BY id; diff --git a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql index 43066880102..7593e1e1580 100644 --- a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql +++ b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql @@ -6,7 +6,7 @@ CREATE TABLE group_uniq_arr_int ENGINE = Memory AS (SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000); SELECT length(groupUniqArray(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '50M'; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '55M'; SELECT length(groupUniqArray(10)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; SELECT length(groupUniqArray(100000)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; diff --git a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql index 1c4376ad577..8b48ee673f3 100644 --- a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql +++ b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql @@ -6,6 +6,6 @@ CREATE TABLE group_uniq_arr_str ENGINE = Memory AS (SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000); SELECT length(groupUniqArray(v)) FROM group_uniq_arr_str GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '50M'; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '55M'; DROP TABLE IF EXISTS group_uniq_arr_str; diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index e7022875086..8f21443d589 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) TEST_PREFIX=$RANDOM ${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600${TEST_PREFIX}" -${CLICKHOUSE_CLIENT} -q "create user u_00600${TEST_PREFIX} settings max_execution_time=60, readonly=1" +${CLICKHOUSE_CLIENT} -q "create user u_00600${TEST_PREFIX} settings max_execution_time=60, readonly=1, max_rows_to_read=0" ${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600${TEST_PREFIX}" function wait_for_query_to_start() @@ -26,7 +26,7 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d # Wait for it to be replaced wait -${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --max_rows_to_read=0 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & +${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' # Trying to run another query with the same query_id diff --git a/tests/queries/0_stateless/00601_kill_running_query.reference b/tests/queries/0_stateless/00601_kill_running_query.reference index 3917ff89482..7824d5804bc 100644 --- a/tests/queries/0_stateless/00601_kill_running_query.reference +++ b/tests/queries/0_stateless/00601_kill_running_query.reference @@ -1 +1 @@ -waiting test_00601_default default SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(50000000) GROUP BY k) +waiting test_00601_default default SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(50000000) GROUP BY k) SETTINGS max_rows_to_read = 0 diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index 15a53841761..efd96746dc4 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,5 +1,6 @@ +SET max_rows_to_read = '100M' drop table if exists lc_00906; create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -insert into lc_00906 select '0123456789' from numbers(100000000) SETTINGS max_rows_to_read = '100M'; +insert into lc_00906 select '0123456789' from numbers(100000000); select count(), b from lc_00906 group by b; drop table if exists lc_00906; diff --git a/tests/queries/0_stateless/01091_query_profiler_does_not_hang.sql b/tests/queries/0_stateless/01091_query_profiler_does_not_hang.sql index 21a84bdd691..45f1a00ae23 100644 --- a/tests/queries/0_stateless/01091_query_profiler_does_not_hang.sql +++ b/tests/queries/0_stateless/01091_query_profiler_does_not_hang.sql @@ -1,4 +1,4 @@ -- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug -SET query_profiler_cpu_time_period_ns = 1; +SET query_profiler_cpu_time_period_ns = 1, max_rows_to_read = 0; SELECT count() FROM numbers_mt(1000000000); diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index 9d326f16a3b..8b383813c9f 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -6,5 +6,6 @@ external_storage_connect_timeout_sec UInt64 10 s3_connect_timeout_ms UInt64 1000 filesystem_prefetch_max_memory_usage UInt64 1073741824 max_memory_usage UInt64 5000000000 +max_memory_usage_for_user UInt64 10000000000 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 diff --git a/tests/queries/0_stateless/01485_256_bit_multiply.sql b/tests/queries/0_stateless/01485_256_bit_multiply.sql index 5c8c47c9127..18be2b11599 100644 --- a/tests/queries/0_stateless/01485_256_bit_multiply.sql +++ b/tests/queries/0_stateless/01485_256_bit_multiply.sql @@ -1,5 +1,7 @@ -- Tags: no-random-settings, no-asan, no-msan, no-tsan, no-ubsan, no-debug +SET max_rows_to_read = '100M' + select count() from ( select toInt128(number) * number x, toInt256(number) * number y from numbers_mt(100000000) where x != y diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index ec14f637c01..b68d15a2200 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -3,6 +3,7 @@ set enable_filesystem_cache=0; set enable_filesystem_cache_on_write_operations=0; +set max_rows_to_read = '30M'; drop table if exists t; create table t (x UInt64, s String) engine = MergeTree order by x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql index 485c8192f69..79c722bd629 100644 --- a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql +++ b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql @@ -2,5 +2,5 @@ SET max_bytes_before_external_group_by = 0; -SET max_memory_usage = '100M'; +SET max_memory_usage = '100M', max_rows_to_read = '1B'; SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(200000000) GROUP BY n FORMAT Null; -- { serverError MEMORY_LIMIT_EXCEEDED } diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 96028f4847a..94eba8f25be 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -4,11 +4,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_result_bytes 0 --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" $CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_result_bytes 0 --max_memory_usage_in_client='5K' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_result_bytes 0 --max_memory_usage_in_client='5k' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" $CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" $CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" $CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" diff --git a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql index cf400ed34c5..d7ce133f38c 100644 --- a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql +++ b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql @@ -6,7 +6,7 @@ SELECT addressToLineWithInlines(1); -- { serverError FUNCTION_NOT_ALLOWED } SET allow_introspection_functions = 1; SET query_profiler_real_time_period_ns = 0; SET query_profiler_cpu_time_period_ns = 1000000; -SET log_queries = 1; +SET log_queries = 1, max_rows_to_read = 0; SELECT count() FROM numbers_mt(10000000000) SETTINGS log_comment='02161_test_case'; SET log_queries = 0; SET query_profiler_cpu_time_period_ns = 0; diff --git a/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql b/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql index fbebfc6d281..1cd6b8a4e4d 100644 --- a/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql +++ b/tests/queries/0_stateless/02226_analyzer_or_like_combine.sql @@ -1,3 +1,5 @@ +SET allow_hyperscan = 1, max_hyperscan_regexp_length = 0, max_hyperscan_regexp_total_length = 0; + EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 0; EXPLAIN QUERY TREE run_passes=1 SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 0, allow_experimental_analyzer = 1; EXPLAIN SYNTAX SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1; diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.reference b/tests/queries/0_stateless/02234_cast_to_ip_address.reference index fa9c6bd0f94..3dd306477b9 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.reference +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.reference @@ -26,9 +26,9 @@ IPv4 functions IPv6 functions \0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0 \N -\0\0\0\0\0\0\0\0\0\0ÿÿ\0\0 -\0\0\0\0\0\0\0\0\0\0ÿÿ\0\0 -\0\0\0\0\0\0\0\0\0\0ÿÿ\0\0 +\0\0\0\0\0\0\0\0\0\0��\0\0 +\0\0\0\0\0\0\0\0\0\0��\0\0 +\0\0\0\0\0\0\0\0\0\0��\0\0 -- :: \N @@ -37,7 +37,7 @@ IPv6 functions ::ffff:127.0.0.1 :: \N -100000000 +20000000 -- ::ffff:127.0.0.1 -- diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.reference b/tests/queries/0_stateless/02343_aggregation_pipeline.reference index bf61eb6da0a..eb013200a17 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.reference +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.reference @@ -1,6 +1,6 @@ -- { echoOn } -explain pipeline select * from (select * from numbers(1e8) group by number) group by number; +explain pipeline select * from (select * from numbers(1e8) group by number) group by number settings max_rows_to_read = 0; (Expression) ExpressionTransform × 16 (Aggregating) @@ -16,7 +16,7 @@ ExpressionTransform × 16 ExpressionTransform (ReadFromSystemNumbers) NumbersRange 0 → 1 -explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number; +explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number settings max_rows_to_read = 0; (Expression) ExpressionTransform × 16 (Aggregating) @@ -32,7 +32,7 @@ ExpressionTransform × 16 ExpressionTransform × 16 (ReadFromSystemNumbers) NumbersRange × 16 0 → 1 -explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number; +explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number settings max_rows_to_read = 0; (Expression) ExpressionTransform (Sorting) diff --git a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql index b324f834053..e7c6c272102 100644 --- a/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql +++ b/tests/queries/0_stateless/02353_simdjson_buffer_overflow.sql @@ -2,5 +2,6 @@ SET max_execution_time = 3; SET timeout_overflow_mode = 'break'; +SET max_rows_to_read = 0, max_bytes_to_read = 0; SELECT count() FROM system.numbers_mt WHERE NOT ignore(JSONExtract('{' || repeat('"a":"b",', rand() % 10) || '"c":"d"}', 'a', 'String')) FORMAT Null; diff --git a/tests/queries/0_stateless/02372_now_in_block.sql b/tests/queries/0_stateless/02372_now_in_block.sql index aee4572ce8d..d0aec471801 100644 --- a/tests/queries/0_stateless/02372_now_in_block.sql +++ b/tests/queries/0_stateless/02372_now_in_block.sql @@ -1,3 +1,4 @@ +SET max_rows_to_read = 0, max_bytes_to_read = 0; SELECT count() FROM (SELECT DISTINCT nowInBlock(), nowInBlock('Pacific/Pitcairn') FROM system.numbers LIMIT 2); SELECT nowInBlock(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT nowInBlock(NULL) IS NULL; diff --git a/tests/queries/0_stateless/02536_delta_gorilla_corruption.sql b/tests/queries/0_stateless/02536_delta_gorilla_corruption.sql index a4e0965e329..3accc726d08 100644 --- a/tests/queries/0_stateless/02536_delta_gorilla_corruption.sql +++ b/tests/queries/0_stateless/02536_delta_gorilla_corruption.sql @@ -12,7 +12,7 @@ create table bug_delta_gorilla (value_bug UInt64 codec (Delta, Gorilla)) engine = MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi' -as (select 0 from numbers(30000000)); +as (select 0 from numbers(20000000)); select count(*) from bug_delta_gorilla From 4712b79960d9da21741aa504cd508a5902e569fe Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 16:41:37 +0800 Subject: [PATCH 0376/1722] Add min_max as statistics type --- src/Storages/Statistics/Statistics.cpp | 8 +- src/Storages/Statistics/StatisticsMinMax.cpp | 91 +++++++++++++++++++ src/Storages/Statistics/StatisticsMinMax.h | 31 +++++++ src/Storages/StatisticsDescription.cpp | 8 +- src/Storages/StatisticsDescription.h | 1 + ...2864_statistics_count_min_sketch.reference | 14 --- .../02864_statistics_estimation.reference | 25 +++++ ...ch.sql => 02864_statistics_estimation.sql} | 41 ++++++++- 8 files changed, 198 insertions(+), 21 deletions(-) create mode 100644 src/Storages/Statistics/StatisticsMinMax.cpp create mode 100644 src/Storages/Statistics/StatisticsMinMax.h delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference create mode 100644 tests/queries/0_stateless/02864_statistics_estimation.reference rename tests/queries/0_stateless/{02864_statistics_count_min_sketch.sql => 02864_statistics_estimation.sql} (57%) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index ade3326288a..5227f3d235e 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -103,6 +104,8 @@ Float64 ColumnStatistics::estimateLess(const Field & val) const { if (stats.contains(StatisticsType::TDigest)) return stats.at(StatisticsType::TDigest)->estimateLess(val); + if (stats.contains(StatisticsType::MinMax)) + return stats.at(StatisticsType::MinMax)->estimateLess(val); return rows * ConditionSelectivityEstimator::default_normal_cond_factor; } @@ -204,6 +207,9 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { + registerValidator(StatisticsType::MinMax, minMaxValidator); + registerCreator(StatisticsType::MinMax, minMaxCreator); + registerValidator(StatisticsType::TDigest, tdigestValidator); registerCreator(StatisticsType::TDigest, tdigestCreator); @@ -240,7 +246,7 @@ ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescri { auto it = creators.find(type); if (it == creators.end()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'min_max', 'tdigest' 'uniq' and 'count_min'", type); auto stat_ptr = (it->second)(desc, stats.data_type); column_stat->stats[type] = stat_ptr; } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp new file mode 100644 index 00000000000..8138ea9fa87 --- /dev/null +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLEGAL_STATISTICS; +} + +StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_) + : IStatistics(stat_) + , min(std::numeric_limits::max()) + , max(std::numeric_limits::min()) + , row_count(0) + , data_type(data_type_) +{ +} + +Float64 StatisticsMinMax::estimateLess(const Field & val) const +{ + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + + if (val_float < min) + return 0; + + if (val_float > max) + return row_count; + + if (max == min) + return row_count; + + return ((val_float - min) / (max - min)) * row_count; +} + +void StatisticsMinMax::update(const ColumnPtr & column) +{ + for (size_t row = 0; row < column->size(); ++row) + { + if (column->isNullAt(row)) + continue; + + auto data = column->getFloat64(row); + min = std::min(data, min); + max = std::max(data, max); + } + row_count += column->size(); +} + +void StatisticsMinMax::serialize(WriteBuffer & buf) +{ + writeIntBinary(row_count, buf); + writeFloatBinary(min, buf); + writeFloatBinary(max, buf); +} + +void StatisticsMinMax::deserialize(ReadBuffer & buf) +{ + readIntBinary(row_count, buf); + readFloatBinary(min, buf); + readFloatBinary(max, buf); +} + + +void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +{ + data_type = removeNullable(data_type); + data_type = removeLowCardinalityAndNullable(data_type); + if (!data_type->isValueRepresentedByNumber()) + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'min_max' do not support type {}", data_type->getName()); +} + +StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +{ + return std::make_shared(stat, data_type); +} + +} diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h new file mode 100644 index 00000000000..913dcccb798 --- /dev/null +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -0,0 +1,31 @@ +#include + + +namespace DB +{ + +class StatisticsMinMax : public IStatistics +{ +public: + StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_); + ~StatisticsMinMax() override = default; + + Float64 estimateLess(const Field & val) const override; + + void update(const ColumnPtr & column) override; + + void serialize(WriteBuffer & buf) override; + void deserialize(ReadBuffer & buf) override; + +private: + Float64 min; + Float64 max; + Float64 row_count; + + DataTypePtr data_type; +}; + +void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); + +} diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 9c5fd3604b2..0d99c36259f 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -51,7 +51,9 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::Uniq; if (type == "count_min") return StatisticsType::CountMinSketch; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type); + if (type == "min_max") + return StatisticsType::MinMax; + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); } String SingleStatisticsDescription::getTypeName() const @@ -64,8 +66,10 @@ String SingleStatisticsDescription::getTypeName() const return "Uniq"; case StatisticsType::CountMinSketch: return "count_min"; + case StatisticsType::MinMax: + return "min_max"; default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); } } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 03b8fb0d583..1aa51ed03dd 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -14,6 +14,7 @@ enum class StatisticsType : UInt8 TDigest = 0, Uniq = 1, CountMinSketch = 2, + MinMax = 3, Max = 63, }; diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference deleted file mode 100644 index 02c41656a36..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference new file mode 100644 index 00000000000..a1db6b8dc96 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -0,0 +1,25 @@ +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime64(3),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics min_max: + Prewhere info + Prewhere filter + Prewhere filter column: and(greater(d, _CAST(9998_DateTime(\'UTC\'), \'DateTime(\'UTC\')\')), less(c, -1), greater(b, 0)) (removed) +Test statistics count_min: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics multi-types: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) +Test statistics min_max and tdigest: + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) +Test LowCardinality and Nullable data type: +tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql similarity index 57% rename from tests/queries/0_stateless/02864_statistics_count_min_sketch.sql rename to tests/queries/0_stateless/02864_statistics_estimation.sql index c730aa7b4a7..f3d085b837b 100644 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -12,13 +12,28 @@ CREATE TABLE tab a String, b UInt64, c Int64, + d DateTime64, pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; SHOW CREATE TABLE tab; -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'Test statistics min_max:'; + +ALTER TABLE tab ADD STATISTICS b TYPE min_max; +ALTER TABLE tab ADD STATISTICS c TYPE min_max; +ALTER TABLE tab ADD STATISTICS d TYPE min_max; +ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > toDateTime(9998, 'UTC')/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS b, c, d; + SELECT 'Test statistics count_min:'; @@ -28,7 +43,7 @@ ALTER TABLE tab ADD STATISTICS c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c; @@ -39,7 +54,8 @@ SELECT 'Test statistics multi-types:'; ALTER TABLE tab ADD STATISTICS a TYPE count_min; ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; +ALTER TABLE tab ADD STATISTICS d TYPE count_min, uniq, tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) @@ -49,7 +65,24 @@ SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS a, b, c; +ALTER TABLE tab DROP STATISTICS a, b, c, d; + + +SELECT 'Test statistics min_max and tdigest:'; + +ALTER TABLE tab ADD STATISTICS b TYPE min_max; +ALTER TABLE tab MATERIALIZE STATISTICS b; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab ADD STATISTICS b TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS b; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS b; + DROP TABLE IF EXISTS tab SYNC; From c2238c57231fe86883eb9b9a7042a72d28d31eae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:45:33 +0200 Subject: [PATCH 0377/1722] Fix tests --- .../0_stateless/00375_shard_group_uniq_array_of_string.sql | 2 +- .../00376_shard_group_uniq_array_of_int_array.sql | 4 +++- .../00377_shard_group_uniq_array_of_string_array.sql | 3 ++- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 2 +- tests/queries/0_stateless/01485_256_bit_multiply.sql | 2 +- tests/queries/0_stateless/01961_roaring_memory_tracking.sql | 2 +- .../queries/0_stateless/02234_cast_to_ip_address.reference | 6 +++--- tests/queries/0_stateless/02234_cast_to_ip_address.sql | 2 +- 8 files changed, 13 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql index 445ffe66f64..8db91904a6a 100644 --- a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql +++ b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql @@ -7,7 +7,7 @@ INSERT INTO group_uniq_str SELECT 2 as id, toString(number % 100) as v FROM syst INSERT INTO group_uniq_str SELECT 5 as id, toString(number % 100) as v FROM system.numbers LIMIT 10000000; SELECT length(groupUniqArray(v)) FROM group_uniq_str GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '60M'; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '100M'; SELECT length(groupUniqArray(10)(v)) FROM group_uniq_str GROUP BY id ORDER BY id; SELECT length(groupUniqArray(10000)(v)) FROM group_uniq_str GROUP BY id ORDER BY id; diff --git a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql index 7593e1e1580..24b7f1c30a6 100644 --- a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql +++ b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql @@ -1,12 +1,14 @@ -- Tags: shard +SET max_rows_to_read = '55M'; + DROP TABLE IF EXISTS group_uniq_arr_int; CREATE TABLE group_uniq_arr_int ENGINE = Memory AS SELECT g as id, if(c == 0, [v], if(c == 1, emptyArrayInt64(), [v, v])) as v FROM (SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000); SELECT length(groupUniqArray(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '55M'; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id; SELECT length(groupUniqArray(10)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; SELECT length(groupUniqArray(100000)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id; diff --git a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql index 8b48ee673f3..180a6a04861 100644 --- a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql +++ b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql @@ -1,4 +1,5 @@ -- Tags: shard +SET max_rows_to_read = '55M'; DROP TABLE IF EXISTS group_uniq_arr_str; CREATE TABLE group_uniq_arr_str ENGINE = Memory AS @@ -6,6 +7,6 @@ CREATE TABLE group_uniq_arr_str ENGINE = Memory AS (SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000); SELECT length(groupUniqArray(v)) FROM group_uniq_arr_str GROUP BY id ORDER BY id; -SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id SETTINGS max_rows_to_read = '55M'; +SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id; DROP TABLE IF EXISTS group_uniq_arr_str; diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index efd96746dc4..9c1abe1b6df 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,4 +1,4 @@ -SET max_rows_to_read = '100M' +SET max_rows_to_read = '100M'; drop table if exists lc_00906; create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_00906 select '0123456789' from numbers(100000000); diff --git a/tests/queries/0_stateless/01485_256_bit_multiply.sql b/tests/queries/0_stateless/01485_256_bit_multiply.sql index 18be2b11599..a4e99d51970 100644 --- a/tests/queries/0_stateless/01485_256_bit_multiply.sql +++ b/tests/queries/0_stateless/01485_256_bit_multiply.sql @@ -1,6 +1,6 @@ -- Tags: no-random-settings, no-asan, no-msan, no-tsan, no-ubsan, no-debug -SET max_rows_to_read = '100M' +SET max_rows_to_read = '100M'; select count() from ( diff --git a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql index 79c722bd629..22eb8e887f2 100644 --- a/tests/queries/0_stateless/01961_roaring_memory_tracking.sql +++ b/tests/queries/0_stateless/01961_roaring_memory_tracking.sql @@ -2,5 +2,5 @@ SET max_bytes_before_external_group_by = 0; -SET max_memory_usage = '100M', max_rows_to_read = '1B'; +SET max_memory_usage = '100M', max_rows_to_read = '1G'; SELECT cityHash64(rand() % 1000) as n, groupBitmapState(number) FROM numbers_mt(200000000) GROUP BY n FORMAT Null; -- { serverError MEMORY_LIMIT_EXCEEDED } diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.reference b/tests/queries/0_stateless/02234_cast_to_ip_address.reference index 3dd306477b9..b9f0a49ec4d 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.reference +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.reference @@ -26,9 +26,9 @@ IPv4 functions IPv6 functions \0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0 \N -\0\0\0\0\0\0\0\0\0\0��\0\0 -\0\0\0\0\0\0\0\0\0\0��\0\0 -\0\0\0\0\0\0\0\0\0\0��\0\0 +\0\0\0\0\0\0\0\0\0\0ÿÿ\0\0 +\0\0\0\0\0\0\0\0\0\0ÿÿ\0\0 +\0\0\0\0\0\0\0\0\0\0ÿÿ\0\0 -- :: \N diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.sql b/tests/queries/0_stateless/02234_cast_to_ip_address.sql index 51e953da905..c851cfde927 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.sql +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.sql @@ -71,7 +71,7 @@ SELECT count() FROM numbers_mt(20000000) WHERE NOT ignore(toIPv6OrZero(randomStr SELECT '--'; -SELECT cast('test' , 'IPv6'); --{serverError CANNOT_PARSE_IPV6} +SELECT cast('test' , 'IPv6'); -- { serverError CANNOT_PARSE_IPV6 } SELECT cast('::ffff:127.0.0.1', 'IPv6'); SELECT '--'; From 2106d4769ac4fca6604dec3b66831aef4b12e943 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 10:53:28 +0200 Subject: [PATCH 0378/1722] Fix tests --- tests/config/users.d/limits.yaml | 1 - tests/queries/0_stateless/02346_additional_filters.sql | 2 ++ 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/config/users.d/limits.yaml b/tests/config/users.d/limits.yaml index 1c9fff9f4c8..23aaccf9298 100644 --- a/tests/config/users.d/limits.yaml +++ b/tests/config/users.d/limits.yaml @@ -30,7 +30,6 @@ profiles: max_columns_to_read: 20K max_temporary_columns: 20K max_temporary_non_const_columns: 20K - max_sessions_for_user: 1K max_rows_in_set: 10G max_bytes_in_set: 10G max_rows_in_join: 10G diff --git a/tests/queries/0_stateless/02346_additional_filters.sql b/tests/queries/0_stateless/02346_additional_filters.sql index f6b665713ec..5a799e1c8c1 100644 --- a/tests/queries/0_stateless/02346_additional_filters.sql +++ b/tests/queries/0_stateless/02346_additional_filters.sql @@ -4,6 +4,8 @@ drop table if exists table_2; drop table if exists v_numbers; drop table if exists mv_table; +SET max_rows_to_read = 0; + create table table_1 (x UInt32, y String) engine = MergeTree order by x; insert into table_1 values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); From 40fc0ca5743a6b1dcc464268a107ba206ca375ab Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 17:43:27 +0800 Subject: [PATCH 0379/1722] Uniq statistics supports more datatypes and refactor logical when there is no statistics. --- .../ConditionSelectivityEstimator.cpp | 41 ++++++------------ .../ConditionSelectivityEstimator.h | 10 ++--- src/Storages/Statistics/Statistics.cpp | 26 ++++++------ src/Storages/Statistics/Statistics.h | 11 +++-- src/Storages/Statistics/StatisticsMinMax.cpp | 8 ++-- src/Storages/Statistics/StatisticsTDigest.cpp | 42 ++++++++++--------- src/Storages/Statistics/StatisticsTDigest.h | 9 ++-- src/Storages/Statistics/StatisticsUniq.cpp | 2 +- src/Storages/Statistics/StatisticsUniq.h | 4 +- .../02864_statistics_estimation.reference | 9 +++- .../02864_statistics_estimation.sql | 21 ++++++++-- 11 files changed, 94 insertions(+), 89 deletions(-) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 57dff958b9a..86970a3ac46 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -19,7 +19,7 @@ void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String par Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(const Field & val, Float64 rows) const { if (part_statistics.empty()) - return default_normal_cond_factor * rows; + return default_cond_range_factor * rows; Float64 result = 0; Float64 part_rows = 0; for (const auto & [key, estimator] : part_statistics) @@ -38,15 +38,8 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreat Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(const Field & val, Float64 rows) const { if (part_statistics.empty()) - { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val) - return default_unknown_cond_factor * rows; - else if (float_val.value() < - threshold || float_val.value() > threshold) - return default_normal_cond_factor * rows; - else - return default_good_cond_factor * rows; - } + return default_cond_equal_factor * rows; + Float64 result = 0; Float64 partial_cnt = 0; for (const auto & [key, estimator] : part_statistics) @@ -149,30 +142,22 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode auto [op, val] = extractBinaryOp(node, col); + if (dummy) + { + if (op == "equals") + return default_cond_equal_factor * total_rows; + else if (op == "less" || op == "lessOrEquals" || op == "greater" || op == "greaterOrEquals") + return default_cond_range_factor * total_rows; + else + return default_unknown_cond_factor * total_rows; + } + if (op == "equals") - { - if (dummy) - { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val || (float_val < - threshold || float_val > threshold)) - return default_normal_cond_factor * total_rows; - else - return default_good_cond_factor * total_rows; - } return estimator.estimateEqual(val, total_rows); - } else if (op == "less" || op == "lessOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateLess(val, total_rows); - } else if (op == "greater" || op == "greaterOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateGreater(val, total_rows); - } else return default_unknown_cond_factor * total_rows; } diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index ce7fdd12e92..269ee9ac6cb 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -38,12 +38,10 @@ private: std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; - static constexpr auto default_good_cond_factor = 0.1; - static constexpr auto default_normal_cond_factor = 0.5; - static constexpr auto default_unknown_cond_factor = 1.0; - /// Conditions like "x = N" are considered good if abs(N) > threshold. - /// This is used to assume that condition is likely to have good selectivity. - static constexpr auto threshold = 2; + /// Used to estimate the selectivity of a condition when there is no statistics. + static constexpr auto default_cond_range_factor = 0.5; + static constexpr auto default_cond_equal_factor = 0.01; + static constexpr auto default_unknown_cond_factor = 1; UInt64 total_rows = 0; std::map column_estimators; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 5227f3d235e..3513638ea9e 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -51,13 +51,6 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) } } -std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) -{ - if (field.getType() == Field::Types::String) - return field.get(); - return {}; -} - IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) { @@ -106,7 +99,7 @@ Float64 ColumnStatistics::estimateLess(const Field & val) const return stats.at(StatisticsType::TDigest)->estimateLess(val); if (stats.contains(StatisticsType::MinMax)) return stats.at(StatisticsType::MinMax)->estimateLess(val); - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; + return rows * ConditionSelectivityEstimator::default_cond_range_factor; } Float64 ColumnStatistics::estimateGreater(const Field & val) const @@ -116,8 +109,7 @@ Float64 ColumnStatistics::estimateGreater(const Field & val) const Float64 ColumnStatistics::estimateEqual(const Field & val) const { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (float_val.has_value() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) + if (stats_desc.data_type->isValueRepresentedByNumber() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) { /// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) for every bucket. if (stats.at(StatisticsType::Uniq)->estimateCardinality() < 2048) @@ -127,10 +119,16 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (stats.contains(StatisticsType::CountMinSketch)) return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val); #endif - if (!float_val.has_value() && (float_val < - ConditionSelectivityEstimator::threshold || float_val > ConditionSelectivityEstimator::threshold)) - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; - else - return rows * ConditionSelectivityEstimator::default_good_cond_factor; + if (stats.contains(StatisticsType::Uniq)) + { + auto cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); + if (cardinality == 0) + return 0; + /// Assume that the value is uniformly distributed among the unique values. + return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality(); + } + + return rows * ConditionSelectivityEstimator::default_cond_equal_factor; } /// ------------------------------------- diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 16f0c67eabd..f47a707663c 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -19,7 +19,6 @@ struct StatisticsUtils { /// Returns std::nullopt if input Field cannot be converted to a concrete value static std::optional tryConvertToFloat64(const Field & field); - static std::optional tryConvertToString(const Field & field); }; /// Statistics describe properties of the values in the column, @@ -32,11 +31,6 @@ public: explicit IStatistics(const SingleStatisticsDescription & stat_); virtual ~IStatistics() = default; - virtual void update(const ColumnPtr & column) = 0; - - virtual void serialize(WriteBuffer & buf) = 0; - virtual void deserialize(ReadBuffer & buf) = 0; - /// Estimate the cardinality of the column. /// Throws if the statistics object is not able to do a meaningful estimation. virtual UInt64 estimateCardinality() const; @@ -46,6 +40,11 @@ public: virtual Float64 estimateEqual(const Field & val) const; /// cardinality of val in the column virtual Float64 estimateLess(const Field & val) const; /// summarized cardinality of values < val in the column + virtual void update(const ColumnPtr & column) = 0; + + virtual void serialize(WriteBuffer & buf) = 0; + virtual void deserialize(ReadBuffer & buf) = 0; + protected: SingleStatisticsDescription stat; }; diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 8138ea9fa87..083e8634841 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -32,18 +32,18 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const if (val_converted.isNull()) return 0; - auto val_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - if (val_float < min) + if (val_as_float < min) return 0; - if (val_float > max) + if (val_as_float > max) return row_count; if (max == min) return row_count; - return ((val_float - min) / (max - min)) * row_count; + return ((val_as_float - min) / (max - min)) * row_count; } void StatisticsMinMax::update(const ColumnPtr & column) diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 66150e00fdb..3544f5cdea3 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include namespace DB { @@ -10,24 +12,20 @@ extern const int ILLEGAL_STATISTICS; extern const int LOGICAL_ERROR; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_) - : IStatistics(stat_) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) + : IStatistics(stat_), data_type(data_type_) { } void StatisticsTDigest::update(const ColumnPtr & column) { - size_t rows = column->size(); - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < column->size(); ++row) { - Field field; - column->get(row, field); - - if (field.isNull()) + if (column->isNullAt(row)) continue; - if (auto field_as_float = StatisticsUtils::tryConvertToFloat64(field)) - t_digest.add(*field_as_float, 1); + auto data = column->getFloat64(row); + t_digest.add(data, 1); } } @@ -43,18 +41,22 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountLessThan(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + return t_digest.getCountLessThan(val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountEqual(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + return t_digest.getCountEqual(val_as_float); } void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) @@ -65,9 +67,9 @@ void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { - return std::make_shared(stat); + return std::make_shared(stat, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 614973e5d8b..f4de6cc41e0 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -9,18 +9,19 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_); + explicit StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + + Float64 estimateLess(const Field & val) const override; + Float64 estimateEqual(const Field & val) const override; void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; - Float64 estimateLess(const Field & val) const override; - Float64 estimateEqual(const Field & val) const override; - private: QuantileTDigest t_digest; + DataTypePtr data_type; }; void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 8f60ffcf0b5..aace45a14b1 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -56,7 +56,7 @@ void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) + if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index faabde8d47c..81162b0a5b9 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -13,13 +13,13 @@ public: StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); ~StatisticsUniq() override; + UInt64 estimateCardinality() const override; + void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; - UInt64 estimateCardinality() const override; - private: std::unique_ptr arena; AggregateFunctionPtr collector; diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index a1db6b8dc96..64f264017d8 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -14,12 +14,19 @@ Test statistics multi-types: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test statistics min_max and tdigest: +Test estimating range condition: Prewhere info Prewhere filter Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) +Test estimating equals condition: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) Test LowCardinality and Nullable data type: tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index f3d085b837b..0ea9eb38e60 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -68,7 +68,7 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; -SELECT 'Test statistics min_max and tdigest:'; +SELECT 'Test estimating range condition:'; ALTER TABLE tab ADD STATISTICS b TYPE min_max; ALTER TABLE tab MATERIALIZE STATISTICS b; @@ -84,6 +84,21 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS b; +SELECT 'Test estimating equals condition:'; + +ALTER TABLE tab ADD STATISTICS a TYPE uniq; +ALTER TABLE tab MATERIALIZE STATISTICS a; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS a; + DROP TABLE IF EXISTS tab SYNC; @@ -93,8 +108,8 @@ SET allow_suspicious_low_cardinality_types=1; CREATE TABLE tab2 ( a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(count_min), + b Nullable(Int64) STATISTICS(min_max, count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(min_max, count_min), pk String, ) Engine = MergeTree() ORDER BY pk; From 878a340317863e94aec61476e105342aef997c7b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 11:47:07 +0200 Subject: [PATCH 0380/1722] Fix tests --- docker/test/stateful/run.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 304bfd7b533..5532df40fdf 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -200,7 +200,8 @@ else clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" fi clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" + # AWS S3 is very inefficient, so increase memory even further: + clickhouse-client --max_memory_usage 20G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" fi clickhouse-client --query "SHOW TABLES FROM test" From 5ae356e6df457e24220f0f63f4a4984334471202 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 17:54:48 +0800 Subject: [PATCH 0381/1722] Add document for min_max statistics --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7ffbd9a5bae..8e8d5e55772 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -991,6 +991,9 @@ They can be used for prewhere optimization only if we enable `set allow_statisti #### Available Types of Column Statistics {#available-types-of-column-statistics} +- `min_max` + + min_max statistics allows to estimate selectivity of range condition for numeric columns. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. From 3f7cd09d168ea13e4ec281c8f3943c0aebe9f920 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 18:02:26 +0800 Subject: [PATCH 0382/1722] Remove useless function tryConvertToFloat64 --- src/Storages/Statistics/Statistics.cpp | 22 ---------------------- src/Storages/Statistics/Statistics.h | 6 ------ 2 files changed, 28 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 3513638ea9e..0f0f1de4552 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -28,28 +28,6 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) -{ - switch (field.getType()) - { - case Field::Types::Int64: - return field.get(); - case Field::Types::UInt64: - return field.get(); - case Field::Types::Float64: - return field.get(); - case Field::Types::Int128: - return field.get(); - case Field::Types::UInt128: - return field.get(); - case Field::Types::Int256: - return field.get(); - case Field::Types::UInt256: - return field.get(); - default: - return {}; - } -} IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index f47a707663c..3eeb5cf92a8 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -15,12 +15,6 @@ constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; -struct StatisticsUtils -{ - /// Returns std::nullopt if input Field cannot be converted to a concrete value - static std::optional tryConvertToFloat64(const Field & field); -}; - /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, /// what are the N most frequent values, From 036485a657a35d764ad33f912d0d10b37d05e59b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 12:15:48 +0200 Subject: [PATCH 0383/1722] Fix error --- docker/test/stateful/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 5532df40fdf..bd2e38ff00f 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -201,7 +201,7 @@ else fi clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" # AWS S3 is very inefficient, so increase memory even further: - clickhouse-client --max_memory_usage 20G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" + clickhouse-client --max_memory_usage 20G --max_memory_usage_for_user 20G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" fi clickhouse-client --query "SHOW TABLES FROM test" From 118e329e23b51673e7d62bb3ffa8a34a8ecdb330 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Wed, 24 Jul 2024 13:09:47 +0100 Subject: [PATCH 0384/1722] Fix flaky test --- ...03174_json_compact_with_progress.reference | 19 ++++++++++--------- .../03174_json_compact_with_progress.sh | 4 ++-- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.reference b/tests/queries/0_stateless/03174_json_compact_with_progress.reference index 6e83db33565..b735c871fbd 100644 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.reference +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.reference @@ -1,14 +1,15 @@ 1 -{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}]}} -{"data":[1, "a"]} -{"data":[2, "b"]} -{"data":[3, "c"]} +{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}, {"name":"sleep(0.1)", "type":"UInt8"}]}} {"progress":{"read_rows":"3","read_bytes":"33","written_rows":"0","written_bytes":"0","total_rows_to_read":"3","result_rows":"0","result_bytes":"0","elapsed_ns":"ELAPSED_NS"}} +{"data":[1, "a", 0]} +{"data":[2, "b", 0]} +{"data":[3, "c", 0]} {"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":33}} 2 -{"meta": [{"name":"name", "type":"String"}, {"name":"c", "type":"UInt64"}]}} -{"data":["a", "1"]} -{"data":["b", "1"]} -{"data":["c", "1"]} -{"totals": ["", "3"]}} +{"meta": [{"name":"name", "type":"String"}, {"name":"count()", "type":"UInt64"}, {"name":"c", "type":"UInt8"}]}} +{"progress":{"read_rows":"3","read_bytes":"30","written_rows":"0","written_bytes":"0","total_rows_to_read":"3","result_rows":"0","result_bytes":"0","elapsed_ns":"ELAPSED_NS"}} +{"data":["a", "1", 0]} +{"data":["b", "1", 0]} +{"data":["c", "1", 0]} +{"totals": ["", "3", 0]}} {"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":30}} diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.sh b/tests/queries/0_stateless/03174_json_compact_with_progress.sh index c21f7228517..383668de858 100755 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.sh +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.sh @@ -11,10 +11,10 @@ $CLICKHOUSE_CLIENT -q "SELECT 1;" # Check JSONCompactWithProgress Output $CLICKHOUSE_CLIENT -q "CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value;" $CLICKHOUSE_CLIENT -q "INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c');" -$CLICKHOUSE_CLIENT -q "SELECT * FROM test_table FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' +$CLICKHOUSE_CLIENT -q "SELECT *, sleep(0.1) FROM test_table FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' $CLICKHOUSE_CLIENT -q "SELECT 2;" # Check Totals -$CLICKHOUSE_CLIENT -q "SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' +$CLICKHOUSE_CLIENT -q "SELECT name, count(), sleep(0.1) AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_table;" From e2c78844a005aa5c04e454df6ff7e0508c967d18 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 24 Jul 2024 14:53:09 +0200 Subject: [PATCH 0385/1722] Fix tests --- docker/test/stateful/run.sh | 2 +- tests/queries/0_stateless/02177_issue_31009.sql | 2 ++ .../00088_global_in_one_shard_and_rows_before_limit.sql | 2 +- tests/queries/1_stateful/00147_global_in_aggregate_function.sql | 2 +- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 2 +- .../queries/1_stateful/00182_simple_squashing_transform_bug.sql | 2 +- 6 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index bd2e38ff00f..fde8b8ae529 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -201,7 +201,7 @@ else fi clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" # AWS S3 is very inefficient, so increase memory even further: - clickhouse-client --max_memory_usage 20G --max_memory_usage_for_user 20G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" + clickhouse-client --max_memory_usage 30G --max_memory_usage_for_user 30G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" fi clickhouse-client --query "SHOW TABLES FROM test" diff --git a/tests/queries/0_stateless/02177_issue_31009.sql b/tests/queries/0_stateless/02177_issue_31009.sql index f25df59f4b4..5c62b5a9c2f 100644 --- a/tests/queries/0_stateless/02177_issue_31009.sql +++ b/tests/queries/0_stateless/02177_issue_31009.sql @@ -8,6 +8,8 @@ DROP TABLE IF EXISTS right; CREATE TABLE left ( key UInt32, value String ) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; CREATE TABLE right ( key UInt32, value String ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; +SET max_rows_to_read = '50M'; + INSERT INTO left SELECT number, toString(number) FROM numbers(25367182); INSERT INTO right SELECT number, toString(number) FROM numbers(23124707); diff --git a/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql b/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql index 443808e7bed..8f18f3740e4 100644 --- a/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql +++ b/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql @@ -1,4 +1,4 @@ -- Tags: shard -SET output_format_write_statistics = 0, max_rows_to_read = 20_000_000; +SET output_format_write_statistics = 0, max_rows_to_read = 50_000_000; SELECT EventDate, count() FROM remote('127.0.0.1', test.hits) WHERE UserID GLOBAL IN (SELECT UserID FROM test.hits) GROUP BY EventDate ORDER BY EventDate LIMIT 5 FORMAT JSONCompact; diff --git a/tests/queries/1_stateful/00147_global_in_aggregate_function.sql b/tests/queries/1_stateful/00147_global_in_aggregate_function.sql index c156f073573..f0b249e9af4 100644 --- a/tests/queries/1_stateful/00147_global_in_aggregate_function.sql +++ b/tests/queries/1_stateful/00147_global_in_aggregate_function.sql @@ -1,5 +1,5 @@ -- Tags: global -SET max_rows_to_read = 40_000_000; +SET max_rows_to_read = 100_000_000; SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{1,2}', test.hits))) FROM remote('127.0.0.{1,2}', test.hits); SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.hits)) FROM remote('127.0.0.{1,2}', test.hits); diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index 1a98a531067..184a8edcbcb 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -1,6 +1,6 @@ -- Tags: no-random-settings -SET max_memory_usage = '10G' +SET max_memory_usage = '10G'; SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40; -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. diff --git a/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql b/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql index 85bad651090..26e112cff04 100644 --- a/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql +++ b/tests/queries/1_stateful/00182_simple_squashing_transform_bug.sql @@ -1,7 +1,7 @@ -- Tags: global set allow_prefetched_read_pool_for_remote_filesystem=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0, max_threads=2, max_block_size=65387; -set max_rows_to_read = '20M'; +set max_rows_to_read = '100M'; SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{1,2}', test.hits))) FROM remote('127.0.0.{1,2}', test.hits); SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.hits)) FROM remote('127.0.0.{1,2}', test.hits); From f37f228af9d70a04166178bac24127693d8b8c35 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 Jul 2024 13:06:29 +0000 Subject: [PATCH 0386/1722] Improve columns squashing for String/Array/Map/Variant/Dynamic types --- src/Columns/ColumnArray.cpp | 16 +++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnDynamic.cpp | 110 ++++++++++++++++++ src/Columns/ColumnDynamic.h | 2 + src/Columns/ColumnMap.cpp | 9 ++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 16 +++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.cpp | 15 +++ src/Columns/ColumnString.h | 1 + src/Columns/ColumnTuple.cpp | 13 +++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 21 +++- src/Columns/ColumnVariant.h | 1 + src/Columns/IColumn.h | 9 ++ src/Interpreters/Squashing.cpp | 27 +++-- tests/performance/insert_select_squashing.xml | 23 ++++ .../03210_dynamic_squashing.reference | 8 ++ .../0_stateless/03210_dynamic_squashing.sql | 20 ++++ 19 files changed, 285 insertions(+), 10 deletions(-) create mode 100644 tests/performance/insert_select_squashing.xml create mode 100644 tests/queries/0_stateless/03210_dynamic_squashing.reference create mode 100644 tests/queries/0_stateless/03210_dynamic_squashing.sql diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 19cce678cc7..9244d75a04d 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -452,6 +452,22 @@ void ColumnArray::reserve(size_t n) getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1. } +void ColumnArray::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + Columns source_data_columns; + source_data_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + { + const auto & source_array_column = assert_cast(*source_column); + new_size += source_array_column.size(); + source_data_columns.push_back(source_array_column.getDataPtr()); + } + + getOffsets().reserve_exact(new_size); + data->prepareForSquashing(source_data_columns); +} + void ColumnArray::shrinkToFit() { getOffsets().shrink_to_fit(); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 63affb86d9d..d6f71b72940 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -118,6 +118,7 @@ public: void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index a92d54dd675..74b7ef69d8d 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -643,6 +643,116 @@ ColumnPtr ColumnDynamic::compress() const }); } +void ColumnDynamic::prepareForSquashing(const Columns & source_columns) +{ + if (source_columns.empty()) + return; + + /// Internal variants of source dynamic columns may differ. + /// We want to preallocate memory for all variants we will have after squashing. + /// It may happen that the total number of variants in source columns will + /// exceed the limit, in this case we will choose the most frequent variants. + + /// First, preallocate memory for variant discriminators and offsets. + size_t new_size = size(); + for (const auto & source_column : source_columns) + new_size += source_column->size(); + auto & variant_col = getVariantColumn(); + variant_col.getLocalDiscriminators().reserve_exact(new_size); + variant_col.getOffsets().reserve_exact(new_size); + + /// Second, collect all variants and their total sizes. + std::unordered_map total_variant_sizes; + DataTypes all_variants; + + auto add_variants = [&](const ColumnDynamic & source_dynamic) + { + const auto & source_variant_column = source_dynamic.getVariantColumn(); + const auto & source_variant_info = source_dynamic.getVariantInfo(); + const auto & source_variants = assert_cast(*source_variant_info.variant_type).getVariants(); + + for (size_t i = 0; i != source_variants.size(); ++i) + { + const auto & variant_name = source_variant_info.variant_names[i]; + auto it = total_variant_sizes.find(variant_name); + /// Add this variant to the list of all variants if we didn't see it yet. + if (it == total_variant_sizes.end()) + { + all_variants.push_back(source_variants[i]); + it = total_variant_sizes.emplace(variant_name, 0).first; + } + + it->second += source_variant_column.getVariantByGlobalDiscriminator(i).size(); + } + }; + + for (const auto & source_column : source_columns) + add_variants(assert_cast(*source_column)); + + /// Add variants from this dynamic column. + add_variants(*this); + + DataTypePtr result_variant_type; + /// Check if the number of all variants exceeds the limit. + if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_variant_sizes.contains("String"))) + { + /// We want to keep the most frequent variants in the resulting dynamic column. + DataTypes result_variants; + result_variants.reserve(max_dynamic_types); + /// Add variants from current variant column as we will not rewrite it. + for (const auto & variant : assert_cast(*variant_info.variant_type).getVariants()) + result_variants.push_back(variant); + /// Add String variant in advance (if we didn't add it yet) as we must have it across variants when we reach the limit. + if (!variant_info.variant_name_to_discriminator.contains("String")) + result_variants.push_back(std::make_shared()); + + /// Create list of remaining variants with their sizes and sort it. + std::vector> variants_with_sizes; + variants_with_sizes.reserve(all_variants.size() - variant_info.variant_names.size()); + for (const auto & variant : all_variants) + { + /// Add variant to the list only of we didn't add it yet. + auto variant_name = variant->getName(); + if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name)) + variants_with_sizes.emplace_back(total_variant_sizes[variant->getName()], variant); + } + + std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); + /// Add the most frequent variants until we reach max_dynamic_types. + size_t num_new_variants = max_dynamic_types - result_variants.size(); + for (size_t i = 0; i != num_new_variants; ++i) + result_variants.push_back(variants_with_sizes[i].second); + + result_variant_type = std::make_shared(result_variants); + } + else + { + result_variant_type = std::make_shared(all_variants); + } + + if (!result_variant_type->equals(*variant_info.variant_type)) + updateVariantInfoAndExpandVariantColumn(result_variant_type); + + /// Now current dynamic column has all resulting variants and we can call + /// prepareForSquashing on them to preallocate the memory. + for (size_t i = 0; i != variant_info.variant_names.size(); ++i) + { + Columns source_variant_columns; + source_variant_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + { + const auto & source_dynamic_column = assert_cast(*source_column); + const auto & source_variant_info = source_dynamic_column.getVariantInfo(); + /// Try to find this variant in the current source column. + auto it = source_variant_info.variant_name_to_discriminator.find(variant_info.variant_names[i]); + if (it != source_variant_info.variant_name_to_discriminator.end()) + source_variant_columns.push_back(source_dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(it->second)); + } + + variant_col.getVariantByGlobalDiscriminator(i).prepareForSquashing(source_variant_columns); + } +} + void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) { if (!empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index e92cabd3db9..cb3a896d2cb 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -254,6 +254,8 @@ public: variant_column->reserve(n); } + void prepareForSquashing(const Columns & source_columns) override; + void ensureOwnership() override { variant_column->ensureOwnership(); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 1025b4e77b9..3bab20dfbf2 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -249,6 +249,15 @@ void ColumnMap::reserve(size_t n) nested->reserve(n); } +void ColumnMap::prepareForSquashing(const Columns & source_columns) +{ + Columns nested_source_columns; + nested_source_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + nested_source_columns.push_back(assert_cast(*source_column).getNestedColumnPtr()); + nested->prepareForSquashing(nested_source_columns); +} + void ColumnMap::shrinkToFit() { nested->shrinkToFit(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 3eaaa0ad562..191476839f1 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -94,6 +94,7 @@ public: void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 6529f0b78db..2a25cac6461 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -706,6 +706,22 @@ void ColumnNullable::reserve(size_t n) getNullMapData().reserve(n); } +void ColumnNullable::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + Columns nested_source_columns; + nested_source_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + { + const auto & source_nullable_column = assert_cast(*source_column); + new_size += source_nullable_column.size(); + nested_source_columns.push_back(source_nullable_column.getNestedColumnPtr()); + } + + nested_column->prepareForSquashing(nested_source_columns); + getNullMapData().reserve(new_size); +} + void ColumnNullable::shrinkToFit() { getNestedColumn().shrinkToFit(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index fe9f5b6dcc2..2c32e0fe5a0 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -125,6 +125,7 @@ public: size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 7cfa2571f5a..9ed2c7e3d4d 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -557,6 +557,21 @@ void ColumnString::reserve(size_t n) offsets.reserve_exact(n); } +void ColumnString::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + size_t new_chars_size = chars.size(); + for (const auto & source_column : source_columns) + { + const auto & source_string_column = assert_cast(*source_column); + new_size += source_string_column.size(); + new_chars_size += source_string_column.chars.size(); + } + + offsets.reserve_exact(new_size); + chars.reserve_exact(new_chars_size); +} + void ColumnString::shrinkToFit() { chars.shrink_to_fit(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c1012e1e55e..20cd950fe9b 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -283,6 +283,7 @@ public: ColumnPtr compress() const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 4fc3f88a87c..c6ee7d775ae 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -595,6 +595,19 @@ void ColumnTuple::reserve(size_t n) getColumn(i).reserve(n); } +void ColumnTuple::prepareForSquashing(const Columns & source_columns) +{ + const size_t tuple_size = columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + { + Columns nested_columns; + nested_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + nested_columns.push_back(assert_cast(*source_column).getColumnPtr(i)); + getColumn(i).prepareForSquashing(nested_columns); + } +} + void ColumnTuple::shrinkToFit() { const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 16b47a993f6..ef396d6a130 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -110,6 +110,7 @@ public: void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index de7efb41d19..68e19861c38 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1247,8 +1247,25 @@ void ColumnVariant::updatePermutation(IColumn::PermutationSortDirection directio void ColumnVariant::reserve(size_t n) { - local_discriminators->reserve(n); - offsets->reserve(n); + getLocalDiscriminators().reserve_exact(n); + getOffsets().reserve_exact(n); +} + +void ColumnVariant::prepareForSquashing(const Columns & source_columns) +{ + size_t new_size = size(); + for (const auto & source_column : source_columns) + new_size += source_column->size(); + reserve(new_size); + + for (size_t i = 0; i != variants.size(); ++i) + { + Columns source_variant_columns; + source_variant_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + source_variant_columns.push_back(assert_cast(*source_column).getVariantPtrByGlobalDiscriminator(i)); + getVariantByGlobalDiscriminator(i).prepareForSquashing(source_variant_columns); + } } void ColumnVariant::ensureOwnership() diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 34c24b5428d..737eb27abfe 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -237,6 +237,7 @@ public: size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override; + void prepareForSquashing(const Columns & source_columns) override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index f9c1a3e7034..edcb9f0bc30 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -475,6 +475,15 @@ public: /// It affects performance only (not correctness). virtual void reserve(size_t /*n*/) {} + /// Reserve memory before squashing all specified source columns into this column. + virtual void prepareForSquashing(const std::vector & source_columns) + { + size_t new_size = size(); + for (const auto & source_column : source_columns) + new_size += source_column->size(); + reserve(new_size); + } + /// Requests the removal of unused capacity. /// It is a non-binding request to reduce the capacity of the underlying container to its size. virtual void shrinkToFit() {} diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 488177c3b4f..5cd40974c45 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -5,7 +5,6 @@ #include #include - namespace DB { @@ -114,20 +113,32 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl { auto & first_chunk = input_chunks[0]; Columns columns = first_chunk.detachColumns(); + mutable_columns.reserve(columns.size()); for (auto & column : columns) - { mutable_columns.push_back(IColumn::mutate(std::move(column))); - mutable_columns.back()->reserve(rows); - } } + size_t num_columns = mutable_columns.size(); + /// Collect the list of source columns for each column. + std::vector source_columns_list(num_columns, Columns{}); + for (size_t i = 0; i != num_columns; ++i) + source_columns_list[i].reserve(input_chunks.size() - 1); + for (size_t i = 1; i < input_chunks.size(); ++i) // We've already processed the first chunk above { - Columns columns = input_chunks[i].detachColumns(); - for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) + auto columns = input_chunks[i].detachColumns(); + for (size_t j = 0; j != num_columns; ++j) + source_columns_list[j].emplace_back(std::move(columns[j])); + } + + for (size_t i = 0; i != num_columns; ++i) + { + /// We know all the data we will insert in advance and can make all necessary pre-allocations. + mutable_columns[i]->prepareForSquashing(source_columns_list[i]); + for (auto & source_column : source_columns_list[i]) { - const auto source_column = columns[j]; - mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); + auto column = std::move(source_column); + mutable_columns[i]->insertRangeFrom(*column, 0, column->size()); } } diff --git a/tests/performance/insert_select_squashing.xml b/tests/performance/insert_select_squashing.xml new file mode 100644 index 00000000000..4c2c88f3d22 --- /dev/null +++ b/tests/performance/insert_select_squashing.xml @@ -0,0 +1,23 @@ + + + 1000 + + + +CREATE TABLE squash_performance +( + s1 String, + s2 Nullable(String), + a1 Array(Array(String)), + a2 Array(Array(UInt32)), + m1 Map(String, Array(String)), + m2 Map(String, Array(UInt64)), + t Tuple(String, Array(String), Map(String, String)) +) +ENGINE = Null; + + + INSERT INTO squash_performance SELECT * FROM generateRandom(42) LIMIT 500000 + + DROP TABLE IF EXISTS squash_performance + diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.reference b/tests/queries/0_stateless/03210_dynamic_squashing.reference new file mode 100644 index 00000000000..4f5b5ba098c --- /dev/null +++ b/tests/queries/0_stateless/03210_dynamic_squashing.reference @@ -0,0 +1,8 @@ +Array(UInt8) +None +UInt64 +None +String +UInt64 +String +UInt64 diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql new file mode 100644 index 00000000000..23b47184e33 --- /dev/null +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -0,0 +1,20 @@ +set allow_experimental_dynamic_type = 1; +set max_block_size = 1000; + +drop table if exists test; + +create table test (d Dynamic) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); +select distinct dynamicType(d) as type from test order by type; + +drop table test; +create table test (d Dynamic(max_types=2)) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); +select distinct dynamicType(d) as type from test order by type; + +truncate table test; +insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); +select distinct dynamicType(d) as type from test order by type; + +drop table test; + From b0ec8c92fcc09857345acb4e4f627827556d17cd Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Wed, 24 Jul 2024 17:49:33 +0100 Subject: [PATCH 0387/1722] Fix totals json object --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index ddf071324c6..0814e5f45d6 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -68,7 +68,7 @@ void JSONCompactWithProgressRowOutputFormat::writeAfterTotals() { JSONUtils::writeCompactArrayEnd(*ostr); JSONUtils::writeCompactObjectEnd(*ostr); - writeCString("}\n", *ostr); + writeCString("\n", *ostr); } void JSONCompactWithProgressRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) From caaf14a0a91bd4779cc292dd674761a47148d24d Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Wed, 24 Jul 2024 17:56:14 +0100 Subject: [PATCH 0388/1722] Fix flaky test by ignoring progress lines when comparing output --- ...03174_json_compact_with_progress.reference | 20 +++++++++---------- .../03174_json_compact_with_progress.sh | 4 ++-- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.reference b/tests/queries/0_stateless/03174_json_compact_with_progress.reference index b735c871fbd..b45f296e5c0 100644 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.reference +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.reference @@ -1,15 +1,13 @@ 1 -{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}, {"name":"sleep(0.1)", "type":"UInt8"}]}} -{"progress":{"read_rows":"3","read_bytes":"33","written_rows":"0","written_bytes":"0","total_rows_to_read":"3","result_rows":"0","result_bytes":"0","elapsed_ns":"ELAPSED_NS"}} -{"data":[1, "a", 0]} -{"data":[2, "b", 0]} -{"data":[3, "c", 0]} +{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}]}} +{"data":[1, "a"]} +{"data":[2, "b"]} +{"data":[3, "c"]} {"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":33}} 2 -{"meta": [{"name":"name", "type":"String"}, {"name":"count()", "type":"UInt64"}, {"name":"c", "type":"UInt8"}]}} -{"progress":{"read_rows":"3","read_bytes":"30","written_rows":"0","written_bytes":"0","total_rows_to_read":"3","result_rows":"0","result_bytes":"0","elapsed_ns":"ELAPSED_NS"}} -{"data":["a", "1", 0]} -{"data":["b", "1", 0]} -{"data":["c", "1", 0]} -{"totals": ["", "3", 0]}} +{"meta": [{"name":"name", "type":"String"}, {"name":"c", "type":"UInt64"}]}} +{"data":["a", "1"]} +{"data":["b", "1"]} +{"data":["c", "1"]} +{"totals": ["", "3"]} {"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":30}} diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.sh b/tests/queries/0_stateless/03174_json_compact_with_progress.sh index 383668de858..b440dbe2891 100755 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.sh +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.sh @@ -11,10 +11,10 @@ $CLICKHOUSE_CLIENT -q "SELECT 1;" # Check JSONCompactWithProgress Output $CLICKHOUSE_CLIENT -q "CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value;" $CLICKHOUSE_CLIENT -q "INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c');" -$CLICKHOUSE_CLIENT -q "SELECT *, sleep(0.1) FROM test_table FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_table FORMAT JSONCompactWithProgress settings max_block_size=2;" | grep -v --text "progress" | sed -E 's/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' $CLICKHOUSE_CLIENT -q "SELECT 2;" # Check Totals -$CLICKHOUSE_CLIENT -q "SELECT name, count(), sleep(0.1) AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactWithProgress settings max_block_size=2;" | sed -E 's/"elapsed_ns":"[0-9]+"/"elapsed_ns":"ELAPSED_NS"/g; s/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' +$CLICKHOUSE_CLIENT -q "SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactWithProgress settings max_block_size=2;" | grep -v --text "progress" | sed -E 's/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_table;" From 2381c3dbca98ff289f71c0fdd699ed3b31f146db Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 Jul 2024 19:11:51 +0000 Subject: [PATCH 0389/1722] Fix and rewrite tests --- src/Columns/ColumnObject.cpp | 16 ++ src/Storages/ColumnsDescription.cpp | 1 + tests/performance/json_type.xml | 6 +- tests/performance/new_json_type.xml | 41 +++++ .../01825_new_type_json_in_array.reference | 2 + .../01825_new_type_json_in_array.sql | 2 + .../01825_new_type_json_nbagames.sh | 2 +- ...columns_1_compact_merge_tree.reference.j2} | 5 - ...on_read_subcolumns_1_compact_merge_tree.sh | 107 ------------- ...ead_subcolumns_1_compact_merge_tree.sql.j2 | 93 ++++++++++++ ...subcolumns_1_wide_merge_tree.reference.j2} | 5 - ..._json_read_subcolumns_1_wide_merge_tree.sh | 107 ------------- ...n_read_subcolumns_1_wide_merge_tree.sql.j2 | 93 ++++++++++++ ...columns_2_compact_merge_tree.reference.j2} | 5 - ...on_read_subcolumns_2_compact_merge_tree.sh | 142 ------------------ ...ead_subcolumns_2_compact_merge_tree.sql.j2 | 128 ++++++++++++++++ ...07_json_read_subcolumns_2_memory.reference | 2 - .../03207_json_read_subcolumns_2_memory.sh | 137 ----------------- .../03207_json_read_subcolumns_2_memory.sql | 123 +++++++++++++++ ...subcolumns_2_wide_merge_tree.reference.j2} | 5 - ..._json_read_subcolumns_2_wide_merge_tree.sh | 142 ------------------ ...n_read_subcolumns_2_wide_merge_tree.sql.j2 | 128 ++++++++++++++++ ...ay_of_json_read_subcolumns_1.reference.j2} | 15 -- .../03208_array_of_json_read_subcolumns_1.sh | 71 --------- ...208_array_of_json_read_subcolumns_1.sql.j2 | 41 +++++ ...columns_2_compact_merge_tree.reference.j2} | 5 - ...on_read_subcolumns_2_compact_merge_tree.sh | 73 --------- ...ead_subcolumns_2_compact_merge_tree.sql.j2 | 57 +++++++ ...of_json_read_subcolumns_2_memory.reference | 2 - ..._array_of_json_read_subcolumns_2_memory.sh | 68 --------- ...array_of_json_read_subcolumns_2_memory.sql | 52 +++++++ ...subcolumns_2_wide_merge_tree.reference.j2} | 5 - ..._json_read_subcolumns_2_wide_merge_tree.sh | 73 --------- ...n_read_subcolumns_2_wide_merge_tree.sql.j2 | 57 +++++++ ..._json_type_horizontal_merges.reference.j2} | 4 - .../03209_json_type_horizontal_merges.sh | 70 --------- .../03209_json_type_horizontal_merges.sql.j2 | 59 ++++++++ ...09_json_type_vertical_merges.reference.j2} | 4 - .../03209_json_type_vertical_merges.sh | 70 --------- .../03209_json_type_vertical_merges.sql.j2 | 59 ++++++++ ...0_json_type_alter_add_column.reference.j2} | 3 - .../03210_json_type_alter_add_column.sh | 47 ------ .../03210_json_type_alter_add_column.sql.j2 | 33 ++++ ... => 03211_nested_json_merges.reference.j2} | 8 - .../0_stateless/03211_nested_json_merges.sh | 65 -------- .../03211_nested_json_merges.sql.j2 | 46 ++++++ 46 files changed, 1035 insertions(+), 1244 deletions(-) create mode 100644 tests/performance/new_json_type.xml rename tests/queries/0_stateless/{03207_json_read_subcolumns_1_compact_merge_tree.reference => 03207_json_read_subcolumns_1_compact_merge_tree.reference.j2} (99%) delete mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sql.j2 rename tests/queries/0_stateless/{03207_json_read_subcolumns_1_wide_merge_tree.reference => 03207_json_read_subcolumns_1_wide_merge_tree.reference.j2} (99%) delete mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sql.j2 rename tests/queries/0_stateless/{03207_json_read_subcolumns_2_compact_merge_tree.reference => 03207_json_read_subcolumns_2_compact_merge_tree.reference.j2} (95%) delete mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 delete mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql rename tests/queries/0_stateless/{03207_json_read_subcolumns_2_wide_merge_tree.reference => 03207_json_read_subcolumns_2_wide_merge_tree.reference.j2} (95%) delete mode 100755 tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh create mode 100644 tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 rename tests/queries/0_stateless/{03208_array_of_json_read_subcolumns_1.reference => 03208_array_of_json_read_subcolumns_1.reference.j2} (99%) delete mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sql.j2 rename tests/queries/0_stateless/{03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference => 03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference.j2} (96%) delete mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 delete mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql rename tests/queries/0_stateless/{03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference => 03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference.j2} (96%) delete mode 100755 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh create mode 100644 tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 rename tests/queries/0_stateless/{03209_json_type_horizontal_merges.reference => 03209_json_type_horizontal_merges.reference.j2} (95%) delete mode 100755 tests/queries/0_stateless/03209_json_type_horizontal_merges.sh create mode 100644 tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 rename tests/queries/0_stateless/{03209_json_type_vertical_merges.reference => 03209_json_type_vertical_merges.reference.j2} (95%) delete mode 100755 tests/queries/0_stateless/03209_json_type_vertical_merges.sh create mode 100644 tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 rename tests/queries/0_stateless/{03210_json_type_alter_add_column.reference => 03210_json_type_alter_add_column.reference.j2} (97%) delete mode 100755 tests/queries/0_stateless/03210_json_type_alter_add_column.sh create mode 100644 tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 rename tests/queries/0_stateless/{03211_nested_json_merges.reference => 03211_nested_json_merges.reference.j2} (85%) delete mode 100755 tests/queries/0_stateless/03211_nested_json_merges.sh create mode 100644 tests/queries/0_stateless/03211_nested_json_merges.sql.j2 diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index ee0d87f500a..98cffdf32bb 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -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> 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; } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 68c11060d88..da749812167 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -711,6 +711,7 @@ std::optional ColumnsDescription::tryGetColumn(const GetColumns return NameAndTypePair(ordinary_column_name, dynamic_subcolumn_name, it->type, dynamic_subcolumn_type); } } + return {}; } diff --git a/tests/performance/json_type.xml b/tests/performance/json_type.xml index b6406f52579..db3fd844f89 100644 --- a/tests/performance/json_type.xml +++ b/tests/performance/json_type.xml @@ -27,9 +27,9 @@ - CREATE TABLE t_json_1(data JSON) ENGINE = MergeTree ORDER BY tuple() - CREATE TABLE t_json_2(data JSON) ENGINE = MergeTree ORDER BY tuple() - CREATE TABLE t_json_3(data JSON) ENGINE = MergeTree ORDER BY tuple() + CREATE TABLE t_json_1(data Object('json')) ENGINE = MergeTree ORDER BY tuple() + CREATE TABLE t_json_2(data Object('json')) ENGINE = MergeTree ORDER BY tuple() + CREATE TABLE t_json_3(data Object('json')) ENGINE = MergeTree ORDER BY tuple() INSERT INTO t_json_1 SELECT materialize({json1}) FROM numbers(200000) INSERT INTO t_json_2 SELECT {json2} FROM numbers(100000) diff --git a/tests/performance/new_json_type.xml b/tests/performance/new_json_type.xml new file mode 100644 index 00000000000..1ad21850c6c --- /dev/null +++ b/tests/performance/new_json_type.xml @@ -0,0 +1,41 @@ + + + 1 + + + + + + + json1 + + '{"k1":1, "k2": "some"}' + + + + json2 + + '{"col' || toString(number % 100) || '":' || toString(number) || '}' + + + + json3 + + '{"k1":[{"k2":"aaa","k3":[{"k4":"bbb"},{"k4":"ccc"}]},{"k2":"ddd","k3":[{"k4":"eee"},{"k4":"fff"}]}]}' + + + + + CREATE TABLE t_json_1(data JSON) ENGINE = MergeTree ORDER BY tuple() + CREATE TABLE t_json_2(data JSON) ENGINE = MergeTree ORDER BY tuple() + CREATE TABLE t_json_3(data JSON) ENGINE = MergeTree ORDER BY tuple() + + INSERT INTO t_json_1 SELECT materialize({json1}) FROM numbers(200000) + INSERT INTO t_json_2 SELECT {json2} FROM numbers(100000) + INSERT INTO t_json_3 SELECT materialize({json3}) FROM numbers_mt(100000) + + DROP TABLE IF EXISTS t_json_1 + DROP TABLE IF EXISTS t_json_2 + DROP TABLE IF EXISTS t_json_3 + diff --git a/tests/queries/0_stateless/01825_new_type_json_in_array.reference b/tests/queries/0_stateless/01825_new_type_json_in_array.reference index 0ca02385389..aa33d9a7413 100644 --- a/tests/queries/0_stateless/01825_new_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_new_type_json_in_array.reference @@ -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}]} diff --git a/tests/queries/0_stateless/01825_new_type_json_in_array.sql b/tests/queries/0_stateless/01825_new_type_json_in_array.sql index 7a8314ee3a8..fbf47e8607d 100644 --- a/tests/queries/0_stateless/01825_new_type_json_in_array.sql +++ b/tests/queries/0_stateless/01825_new_type_json_in_array.sql @@ -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; diff --git a/tests/queries/0_stateless/01825_new_type_json_nbagames.sh b/tests/queries/0_stateless/01825_new_type_json_nbagames.sh index 2a02aa9adc9..20eba88eda4 100755 --- a/tests/queries/0_stateless/01825_new_type_json_nbagames.sh +++ b/tests/queries/0_stateless/01825_new_type_json_nbagames.sh @@ -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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 similarity index 99% rename from tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference rename to tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 index 16150ee7b45..972cfd9c37f 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 @@ -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)') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh deleted file mode 100755 index 75b13f89a06..00000000000 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sql.j2 new file mode 100644 index 00000000000..0ec1a86372b --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.sql.j2 @@ -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; \ No newline at end of file diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 similarity index 99% rename from tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference rename to tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 index 16150ee7b45..972cfd9c37f 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 @@ -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)') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh deleted file mode 100755 index 144a88d8ec6..00000000000 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sql.j2 new file mode 100644 index 00000000000..f571d2417f4 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.sql.j2 @@ -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; \ No newline at end of file diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 similarity index 95% rename from tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference rename to tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 index db7180cec75..13343b21a8c 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 @@ -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)') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh deleted file mode 100755 index 6df95ad4ad6..00000000000 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 new file mode 100644 index 00000000000..942489ec8fc --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 @@ -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; \ No newline at end of file diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference index 1bbd3926bdc..6c455b1bb0d 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference @@ -1,5 +1,3 @@ -insert -test ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') ('a.b.d','DateTime64(9)') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh deleted file mode 100755 index 154c1aa5f9c..00000000000 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql new file mode 100644 index 00000000000..3c791118c9c --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql @@ -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; diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 similarity index 95% rename from tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference rename to tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 index db7180cec75..13343b21a8c 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 @@ -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)') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh deleted file mode 100755 index d20d9a4d79d..00000000000 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 new file mode 100644 index 00000000000..325ce2dcbf5 --- /dev/null +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 @@ -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; \ No newline at end of file diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference.j2 similarity index 99% rename from tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference rename to tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference.j2 index ec8d1407bdb..0228ae1e7df 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.reference.j2 @@ -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') diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh deleted file mode 100755 index 9d8d02de7aa..00000000000 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sql.j2 b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sql.j2 new file mode 100644 index 00000000000..1353980cd35 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_1.sql.j2 @@ -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 -%} diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference.j2 similarity index 96% rename from tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference rename to tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference.j2 index 9c092278ec7..2fd3437e3d2 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.reference.j2 @@ -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') diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh deleted file mode 100755 index c39411f7b09..00000000000 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 new file mode 100644 index 00000000000..e1e0c07a5df --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 @@ -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; diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference index 63d10b1315f..34557cf60bb 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.reference @@ -1,5 +1,3 @@ -insert -test ('a.a1','String') ('a.a2','String') ('a.a3','String') diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh deleted file mode 100755 index b179baeadef..00000000000 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql new file mode 100644 index 00000000000..1cd5e2b8d47 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql @@ -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; diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference.j2 similarity index 96% rename from tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference rename to tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference.j2 index 9c092278ec7..2fd3437e3d2 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.reference.j2 @@ -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') diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh deleted file mode 100755 index 6686179e801..00000000000 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 new file mode 100644 index 00000000000..89223316aa3 --- /dev/null +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 @@ -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; diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 similarity index 95% rename from tests/queries/0_stateless/03209_json_type_horizontal_merges.reference rename to tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 index 203b59521f4..31ba25b7732 100644 --- a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 @@ -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 diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sh b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sh deleted file mode 100755 index 7b30827e7a0..00000000000 --- a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sh +++ /dev/null @@ -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;" - diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 new file mode 100644 index 00000000000..6ae9f438432 --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 @@ -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 -%} diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 similarity index 95% rename from tests/queries/0_stateless/03209_json_type_vertical_merges.reference rename to tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 index 203b59521f4..31ba25b7732 100644 --- a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 @@ -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 diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.sh b/tests/queries/0_stateless/03209_json_type_vertical_merges.sh deleted file mode 100755 index 66b05ea0394..00000000000 --- a/tests/queries/0_stateless/03209_json_type_vertical_merges.sh +++ /dev/null @@ -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;" - diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 b/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 new file mode 100644 index 00000000000..aef36452bb8 --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 @@ -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 -%} diff --git a/tests/queries/0_stateless/03210_json_type_alter_add_column.reference b/tests/queries/0_stateless/03210_json_type_alter_add_column.reference.j2 similarity index 97% rename from tests/queries/0_stateless/03210_json_type_alter_add_column.reference rename to tests/queries/0_stateless/03210_json_type_alter_add_column.reference.j2 index 907cde709f3..37b6854938a 100644 --- a/tests/queries/0_stateless/03210_json_type_alter_add_column.reference +++ b/tests/queries/0_stateless/03210_json_type_alter_add_column.reference.j2 @@ -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 diff --git a/tests/queries/0_stateless/03210_json_type_alter_add_column.sh b/tests/queries/0_stateless/03210_json_type_alter_add_column.sh deleted file mode 100755 index dfa5de9f091..00000000000 --- a/tests/queries/0_stateless/03210_json_type_alter_add_column.sh +++ /dev/null @@ -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;" - diff --git a/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 b/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 new file mode 100644 index 00000000000..8ab9f5181e3 --- /dev/null +++ b/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 @@ -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 -%} diff --git a/tests/queries/0_stateless/03211_nested_json_merges.reference b/tests/queries/0_stateless/03211_nested_json_merges.reference.j2 similarity index 85% rename from tests/queries/0_stateless/03211_nested_json_merges.reference rename to tests/queries/0_stateless/03211_nested_json_merges.reference.j2 index 221fcefba66..e05e9ced0b1 100644 --- a/tests/queries/0_stateless/03211_nested_json_merges.reference +++ b/tests/queries/0_stateless/03211_nested_json_merges.reference.j2 @@ -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 diff --git a/tests/queries/0_stateless/03211_nested_json_merges.sh b/tests/queries/0_stateless/03211_nested_json_merges.sh deleted file mode 100755 index be0ee696694..00000000000 --- a/tests/queries/0_stateless/03211_nested_json_merges.sh +++ /dev/null @@ -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;" diff --git a/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 b/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 new file mode 100644 index 00000000000..4ab5a5da0ef --- /dev/null +++ b/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 @@ -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 -%} From c418ea4e60bdfcab416951f41928d3c72c490188 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 Jul 2024 21:42:17 +0000 Subject: [PATCH 0390/1722] Fix build --- src/Functions/FunctionsJSON.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 848856c500f..ea485241aab 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -739,7 +739,7 @@ public: { NumberType value; - if (!tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, error)) + if (!tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, true, error)) return false; auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); From c837541a7783f14780a7d2535dd6fa2cbf5effd5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Jul 2024 10:11:53 +0800 Subject: [PATCH 0391/1722] fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 943caf918d6..fa26cc0ff1f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2166,6 +2166,7 @@ outfile overcommit overcommitted overfitting +overlayUTF overparallelization packetpool packetsize From b9703ad9242f7492de6f6d10e13ac89a74231210 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 25 Jul 2024 15:14:12 +0800 Subject: [PATCH 0392/1722] Fix logical error when estimating --- src/Storages/Statistics/Statistics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 0f0f1de4552..17ff8deb720 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -103,7 +103,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (cardinality == 0) return 0; /// Assume that the value is uniformly distributed among the unique values. - return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality(); + return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality() * rows; } return rows * ConditionSelectivityEstimator::default_cond_equal_factor; From 97ceca4b92289352eb71b6cfd0f5280162334609 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jul 2024 12:20:39 +0000 Subject: [PATCH 0393/1722] fix reading from Merge tables --- .../QueryPlan/Optimizations/projectionsCommon.cpp | 11 +++++++++-- src/Storages/IStorage.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 10 ++++++++-- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMerge.cpp | 6 ++++-- 5 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 487367b8c30..e7df58e1a86 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -41,12 +41,19 @@ bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) if (reading->readsInOrder()) return false; + const auto & query_settings = reading->getContext()->getSettingsRef(); + // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + if (query_settings.allow_experimental_query_deduplication) return false; // Currently projection don't support settings which implicitly modify aggregate functions. - if (reading->getContext()->getSettingsRef().aggregate_functions_null_for_empty) + if (query_settings.aggregate_functions_null_for_empty) + return false; + + /// Don't use projections if have mutations to apply + /// because we need to apply them on original data. + if (query_settings.apply_mutations_on_fly && reading->getMutationsSnapshot()->hasDataMutations()) return false; return true; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6217470780d..92de82934e3 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -271,6 +271,7 @@ public: /// Return true if the trivial count query could be optimized without reading the data at all /// in totalRows() or totalRowsByPartitionPredicate() methods or with optimized reading in read() method. + /// 'storage_snapshot' may be nullptr. virtual bool supportsTrivialCountOptimization(const StorageSnapshotPtr & /*storage_snapshot*/, ContextPtr /*query_context*/) const { return false; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b32a3f5eda3..f1054355b07 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8444,10 +8444,16 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart DB::updateObjectColumns(object_columns, columns, part->getColumns()); } -bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const +bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const { + if (hasLightweightDeletedMask()) + return false; + + if (!storage_snapshot) + return !query_context->getSettingsRef().apply_mutations_on_fly; + const auto & snapshot_data = assert_cast(*storage_snapshot->data); - return !hasLightweightDeletedMask() && !snapshot_data.mutations_snapshot->hasDataMutations(); + return !snapshot_data.mutations_snapshot->hasDataMutations(); } Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d5cfddb70af..588b31db7b9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -443,7 +443,7 @@ public: bool areAsynchronousInsertsEnabled() const override; - bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const override; + bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const override; /// A snapshot of pending mutations that weren't applied to some of the parts yet /// and should be applied on the fly (i.e. when reading from the part). diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f5bc183931f..0cc369c87fa 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1617,9 +1617,11 @@ std::tuple StorageMerge::evaluateDatabaseName(cons return {false, ast}; } -bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr ctx) const +bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr ctx) const { - return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(storage_snapshot, ctx); }) == nullptr; + /// Here we actually need storage snapshot of all nested tables. + /// But to avoid complexity pass nullptr to make more lightweight check in MergeTreeData. + return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr; } std::optional StorageMerge::totalRows(const Settings & settings) const From 63ccbcbdfc7aa012e1879b11f9f6f4a23f9a1a0c Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Thu, 25 Jul 2024 13:22:47 +0100 Subject: [PATCH 0394/1722] Remove test tags --- tests/queries/0_stateless/03174_json_compact_with_progress.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.sh b/tests/queries/0_stateless/03174_json_compact_with_progress.sh index b440dbe2891..8f524cdff76 100755 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.sh +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From cad120987c0847b14c407b6548c1862d8ed34f6d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:33:45 +0000 Subject: [PATCH 0395/1722] Fix flaky tests --- src/Processors/Formats/IRowInputFormat.cpp | 4 ---- src/Storages/VirtualColumnUtils.cpp | 1 - .../0_stateless/01825_new_type_json_ghdata_insert_select.sh | 2 +- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 31d7fdbf67b..0b6c81923db 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -124,10 +124,6 @@ Chunk IRowInputFormat::read() return getChunkForCount(num_rows); } - /// Reserve params.max_block_size rows in advance. - for (auto & column : columns) - column->reserve(params.max_block_size); - RowReadExtension info; bool continue_reading = true; for (size_t rows = 0; (rows < params.max_block_size || num_rows == 0) && continue_reading; ++rows) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index dd1bcb7446c..151079154b1 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -290,7 +290,6 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( else if (!allow_non_deterministic_functions) return nullptr; - if (node_copy.children.empty()) return nullptr; diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh index ad428603da2..ef87034ff89 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 713b337681dbf562ef1a354a30f2aec2a6fdbd00 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:38:07 +0000 Subject: [PATCH 0396/1722] Add comment about splitSubcolumnName in DataTypeDynamic --- src/DataTypes/DataTypeDynamic.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index 88871f28f06..da2299f53e6 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -90,6 +90,10 @@ void registerDataTypeDynamic(DataTypeFactory & factory) namespace { +/// Split Dynamic subcolumn name into 2 parts: type name and subcolumn of this type. +/// We cannot simply split by '.' because type name can also contain dots. For example: Tuple(`a.b` UInt32). +/// But in all such cases this '.' will be inside back quotes. To split subcolumn name correctly +/// we search for the first '.' that is not inside back quotes. std::pair splitSubcolumnName(std::string_view subcolumn_name) { bool inside_quotes = false; From f20ca25a79333a82c8b191c8718fa1a88fdc014a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:40:21 +0000 Subject: [PATCH 0397/1722] Fix creating serialization for Rapid and Dummy json parsers --- src/DataTypes/DataTypeObject.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index eeda6476fd0..e2a2caa9f21 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -112,9 +112,17 @@ SerializationPtr DataTypeObject::doGetDefaultSerialization() const path_regexps_to_skip, buildJSONExtractTree(getPtr(), "JSON serialization")); #elif USE_RAPIDJSON - return std::make_shared>(std::move(typed_path_serializations), paths_to_skip, buildJSONExtractTree(getPtr(), "JSON serialization")); + return std::make_shared>( + std::move(typed_path_serializations), + paths_to_skip, + path_regexps_to_skip, + buildJSONExtractTree(getPtr(), "JSON serialization")); #else - return std::make_shared>(std::move(typed_path_serializations), paths_to_skip, buildJSONExtractTree(getPtr(), "JSON serialization")); + return std::make_shared>( + std::move(typed_path_serializations), + paths_to_skip, + path_regexps_to_skip, + buildJSONExtractTree(getPtr(), "JSON serialization")); #endif } } From 701d99a9ad56243933ce5d85796f47c26f574b8e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 13:56:17 +0000 Subject: [PATCH 0398/1722] Small clean up --- src/DataTypes/Serializations/ISerialization.h | 2 +- src/Formats/JSONExtractTree.cpp | 4 ++-- src/Formats/SchemaInferenceUtils.cpp | 2 +- src/IO/WriteHelpers.h | 6 ------ src/Interpreters/InterpreterShowColumnsQuery.cpp | 2 +- 5 files changed, 5 insertions(+), 11 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 5d6ac2707ed..54234189528 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -448,7 +448,7 @@ public: static bool hasSubcolumnForPath(const SubstreamPath & path, size_t prefix_len); static SubstreamData createFromPath(const SubstreamPath & path, size_t prefix_len); - /// Returns true if subcolumn doesn't actually stores any data in column and doen'st require a separate stream + /// Returns true if subcolumn doesn't actually stores any data in column and doesn't require a separate stream /// for writing/reading data. For example, it's a null-map subcolumn of Variant type (it's always constructed from discriminators);. static bool isFictitiousSubcolumn(const SubstreamPath & path, size_t prefix_len); diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index f5b5a7bb984..8203ccb5862 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1404,7 +1404,7 @@ public: String & error) const override { auto & column_dynamic = assert_cast(column); - /// First, check if element is NULL. + /// Check if element is NULL. if (element.isNull()) { column_dynamic.insertDefault(); @@ -1414,7 +1414,7 @@ public: auto & variant_column = column_dynamic.getVariantColumn(); const auto & variant_info = column_dynamic.getVariantInfo(); - /// First, try to insert element into current variants but with no types conversion. + /// Try to insert element into current variants but with no types conversion. /// We want to avoid inferring the type on each row, so if we can insert this element into /// any existing variant with no types conversion (like Integer -> String, Double -> Integer, etc) /// we will do it and won't try to infer the type. diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 28d60bf078b..5d6267cd1bb 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -295,7 +295,7 @@ namespace WhichDataType which(type); if (which.isInt64() || which.isUInt64()) { - const auto & new_type = std::make_shared(); + auto new_type = std::make_shared(); if (json_info && json_info->numbers_parsed_from_json_strings.erase(type.get())) json_info->numbers_parsed_from_json_strings.insert(new_type.get()); type = new_type; diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 306635272f5..6b0de441e94 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -90,12 +90,6 @@ inline void writeUUIDBinary(const UUID & x, WriteBuffer & buf) writePODBinary(uuid.items[1], buf); } -template -void setValue(char * data, const T & value) -{ - memcpy(data, reinterpret_cast(&value), sizeof(T)); -} - template inline void writeIntBinary(const T & x, WriteBuffer & buf) { diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index b39ce25ab57..472cdedf3ae 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -68,7 +68,7 @@ WITH map( 'Map', 'JSON', 'Tuple', 'JSON', 'Object', 'JSON', - 'JSON', 'JSON', + 'JSON', 'JSON', 'String', '{}', 'FixedString', '{}') AS native_to_mysql_mapping, )", From 6725546b312ef52675f8fbd5f41d0ef5327a3e8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 19:35:21 +0200 Subject: [PATCH 0399/1722] Update some tests --- .../02884_parallel_window_functions.sql | 18 ++++++++++-------- .../03143_asof_join_ddb_long.reference | 4 ++-- .../0_stateless/03143_asof_join_ddb_long.sql | 4 ++-- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql index c5ab013a198..ea1cd458c65 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.sql +++ b/tests/queries/0_stateless/02884_parallel_window_functions.sql @@ -1,6 +1,6 @@ -- Tags: long, no-tsan, no-asan, no-ubsan, no-msan, no-debug -CREATE TABLE window_funtion_threading +CREATE TABLE window_function_threading Engine = MergeTree ORDER BY (ac, nw) AS SELECT @@ -20,7 +20,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading GROUP BY ac, nw ) GROUP BY nw @@ -40,7 +40,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading GROUP BY ac, nw ) GROUP BY nw @@ -58,7 +58,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading GROUP BY ac, nw ) GROUP BY nw @@ -66,6 +66,8 @@ ORDER BY nw ASC, R DESC LIMIT 10 SETTINGS max_threads = 1; +SET max_rows_to_read = 30000000; + SELECT nw, sum(WR) AS R, @@ -77,7 +79,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 0 GROUP BY ac, @@ -88,7 +90,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 1 GROUP BY ac, @@ -99,7 +101,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 2 GROUP BY ac, @@ -110,7 +112,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 3 GROUP BY ac, diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.reference b/tests/queries/0_stateless/03143_asof_join_ddb_long.reference index 2850a8aba98..ae7f7c805f2 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.reference +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.reference @@ -1,2 +1,2 @@ -49999983751397 10000032 -49999983751397 10000032 +7999995751397 4000032 +7999995751397 4000032 diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index 17a67511030..a635fd2e86a 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -11,7 +11,7 @@ AS toDateTime('1990-03-21 13:00:00') + INTERVAL number MINUTE AS begin, number % 4 AS key, number AS value - FROM numbers(0, 10000000); + FROM numbers(0, 4000000); CREATE TABLE skewed_probe ENGINE = MergeTree ORDER BY (key, begin) AS @@ -33,7 +33,7 @@ AS SELECT toDateTime('1990-03-21 13:00:01') + INTERVAL number MINUTE AS begin, 3 AS key - FROM numbers(0, 10000000); + FROM numbers(0, 4000000); SELECT SUM(value), COUNT(*) From f00928afa14864cd8efbc186f63538c8cd99f643 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 18:41:43 +0000 Subject: [PATCH 0400/1722] Improve castOrDefault from Variant/Dynamic columns --- src/Functions/FunctionsConversion.cpp | 52 +++++++++++++++++-- ..._variant_dynamic_cast_or_default.reference | 32 ++++++++++++ .../03212_variant_dynamic_cast_or_default.sql | 9 ++++ 3 files changed, 89 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference create mode 100644 tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..e516d1dbe54 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4086,9 +4086,30 @@ private: /// Create conversion wrapper for each variant. for (const auto & variant_type : variant_types) - variant_wrappers.push_back(prepareUnpackDictionaries(variant_type, to_type)); + { + WrapperType wrapper; + if (cast_type == CastType::accurateOrNull) + { + /// With accurateOrNull cast type we should insert default values on variants that cannot be casted. + /// We can avoid try/catch here if we will implement check that 2 types can be casted, but it + /// requires quite a lot of work. By now let's simply use try/catch. + try + { + wrapper = prepareUnpackDictionaries(variant_type, to_type); + } + catch (...) + { + /// Leave wrapper empty and check it later. + } + } + else + { + wrapper = prepareUnpackDictionaries(variant_type, to_type); + } + variant_wrappers.push_back(wrapper); + } - return [variant_wrappers, variant_types, to_type] + return [variant_wrappers, variant_types, to_type, cast_type_ = this->cast_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr { const auto & column_variant = assert_cast(*arguments.front().column.get()); @@ -4101,7 +4122,30 @@ private: auto variant_col = column_variant.getVariantPtrByGlobalDiscriminator(i); ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }}; const auto & variant_wrapper = variant_wrappers[i]; - casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + ColumnPtr casted_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + { + if (cast_type_ == CastType::accurateOrNull) + { + /// With accurateOrNull cast type wrapper should throw an exception + /// only when the cast between types is not supported. + /// In this case we will insert default values on rows with this variant. + try + { + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + } + catch (...) + { + /// Do nothing. + } + } + else + { + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + } + } + casted_variant_columns.push_back(std::move(casted_variant)); } /// Second, construct resulting column from casted variant columns according to discriminators. @@ -4111,7 +4155,7 @@ private: for (size_t i = 0; i != input_rows_count; ++i) { auto global_discr = column_variant.globalDiscriminatorByLocal(local_discriminators[i]); - if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !casted_variant_columns[global_discr]) res->insertDefault(); else res->insertFrom(*casted_variant_columns[global_discr], column_variant.offsetAt(i)); diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference new file mode 100644 index 00000000000..8b1a342181c --- /dev/null +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference @@ -0,0 +1,32 @@ +0 \N +1 1 +0 str_2 +0 [0,1,2] +0 \N +5 5 +0 str_6 +0 [0,1,2,3,4,5,6] +\N \N +1 1 +\N str_2 +\N [0,1,2] +\N \N +5 5 +\N str_6 +\N [0,1,2,3,4,5,6] +0 \N +1 1 +0 str_2 +0 [0,1,2] +0 \N +5 5 +0 str_6 +0 [0,1,2,3,4,5,6] +\N \N +1 1 +\N str_2 +\N [0,1,2] +\N \N +5 5 +\N str_6 +\N [0,1,2,3,4,5,6] diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql new file mode 100644 index 00000000000..1e71e36780c --- /dev/null +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql @@ -0,0 +1,9 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +select accurateCastOrDefault(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); +select accurateCastOrNull(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); + +select accurateCastOrDefault(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); +select accurateCastOrNull(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); From eb300f4f782bf9a6b216624bddd6e6deffd55d0f Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 18:59:06 +0000 Subject: [PATCH 0401/1722] Better implementation --- src/Functions/FunctionsConversion.cpp | 56 ++++++++++++--------------- 1 file changed, 24 insertions(+), 32 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index e516d1dbe54..c14fa3187d8 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4078,6 +4078,26 @@ private: }; } + /// Create wrapper only if we support this conversion. + WrapperType createWrapperIfCanConvert(const DataTypePtr & from, const DataTypePtr & to) const + { + try + { + /// We can avoid try/catch here if we will implement check that 2 types can be casted, but it + /// requires quite a lot of work. By now let's simply use try/catch. + /// First, check that we can create a wrapper. + WrapperType wrapper = prepareUnpackDictionaries(from, to); + /// Second, check if we can perform a conversion on empty columns. + ColumnsWithTypeAndName column_from = {{from->createColumn(), from, "" }}; + wrapper(column_from, to, nullptr, 0); + return wrapper; + } + catch (...) + { + return {}; + } + } + WrapperType createVariantToColumnWrapper(const DataTypeVariant & from_variant, const DataTypePtr & to_type) const { const auto & variant_types = from_variant.getVariants(); @@ -4090,17 +4110,8 @@ private: WrapperType wrapper; if (cast_type == CastType::accurateOrNull) { - /// With accurateOrNull cast type we should insert default values on variants that cannot be casted. - /// We can avoid try/catch here if we will implement check that 2 types can be casted, but it - /// requires quite a lot of work. By now let's simply use try/catch. - try - { - wrapper = prepareUnpackDictionaries(variant_type, to_type); - } - catch (...) - { - /// Leave wrapper empty and check it later. - } + /// Create wrapper only if we support conversion from variant to the resulting type. + wrapper = createWrapperIfCanConvert(variant_type, to_type); } else { @@ -4109,7 +4120,7 @@ private: variant_wrappers.push_back(wrapper); } - return [variant_wrappers, variant_types, to_type, cast_type_ = this->cast_type] + return [variant_wrappers, variant_types, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr { const auto & column_variant = assert_cast(*arguments.front().column.get()); @@ -4125,26 +4136,7 @@ private: ColumnPtr casted_variant; /// Check if we have wrapper for this variant. if (variant_wrapper) - { - if (cast_type_ == CastType::accurateOrNull) - { - /// With accurateOrNull cast type wrapper should throw an exception - /// only when the cast between types is not supported. - /// In this case we will insert default values on rows with this variant. - try - { - casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); - } - catch (...) - { - /// Do nothing. - } - } - else - { - casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); - } - } + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); casted_variant_columns.push_back(std::move(casted_variant)); } From a7441669aa87b1551d2211ec4c3e550aaaa86b41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 21:43:03 +0200 Subject: [PATCH 0402/1722] Update some tests --- ...675_profile_events_from_query_log_and_client.sh | 2 +- .../02884_parallel_window_functions.sql | 2 +- ...967_parallel_replicas_join_algo_and_analyzer.sh | 14 +++++++------- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index e346d9893a7..1cf65ed8120 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "INSERT TO S3" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | sort +" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | grep -v 'RequestThrottlerCount' | sort echo "CHECK WITH query_log" $CLICKHOUSE_CLIENT -nq " diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql index ea1cd458c65..2207c90a4ee 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.sql +++ b/tests/queries/0_stateless/02884_parallel_window_functions.sql @@ -66,7 +66,7 @@ ORDER BY nw ASC, R DESC LIMIT 10 SETTINGS max_threads = 1; -SET max_rows_to_read = 30000000; +SET max_rows_to_read = 40000000; SELECT nw, diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh index 2840482da6d..8cefa873940 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh @@ -90,7 +90,7 @@ $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -128,7 +128,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -154,7 +154,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -179,7 +179,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -205,7 +205,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -230,7 +230,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -255,7 +255,7 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | From ba5a07bcc7b78673e58d0501e85b59e929215bac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 21:47:41 +0200 Subject: [PATCH 0403/1722] Better tests --- .../01301_aggregate_state_exception_memory_leak.reference | 2 +- .../0_stateless/01301_aggregate_state_exception_memory_leak.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference index b20e7415f52..6282bf366d0 100644 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference @@ -1,2 +1,2 @@ -Memory limit (for query) exceeded +Memory limit exceeded Ok diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 9dd800ceb09..266518d11d4 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -16,5 +16,5 @@ for _ in {1..1000}; do if [[ $elapsed -gt 30 ]]; then break fi -done 2>&1 | grep -o -F 'Memory limit (for query) exceeded' | uniq +done 2>&1 | grep -o -P 'Memory limit .+ exceeded' | sed -r -e 's/(Memory limit)(.+)( exceeded)/\1\3/' | uniq echo 'Ok' From f81e8aa345d64f5fbcae103f92cdc649f0d82d24 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jul 2024 22:08:32 +0200 Subject: [PATCH 0404/1722] Update tests --- .../queries/0_stateless/01010_pmj_right_table_memory_limits.sql | 2 ++ tests/queries/0_stateless/01304_direct_io_long.sh | 2 +- .../01730_distributed_group_by_no_merge_order_by_long.sql | 2 +- tests/queries/0_stateless/02151_lc_prefetch.sql | 1 + .../queries/0_stateless/02344_insert_profile_events_stress.sql | 1 + ...02354_distributed_with_external_aggregation_memory_usage.sql | 2 ++ .../02481_parquet_list_monotonically_increasing_offsets.sh | 2 +- tests/queries/0_stateless/02497_remote_disk_fat_column.sql | 2 +- .../02896_max_execution_time_with_break_overflow_mode.sql | 2 +- tests/queries/0_stateless/03143_asof_join_ddb_long.sql | 1 + 10 files changed, 12 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql b/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql index a090be85221..b8f2596f3d5 100644 --- a/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql +++ b/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel, no-fasttest, no-random-settings +SET max_bytes_in_join = 0; +SET max_rows_in_join = 0; SET max_memory_usage = 32000000; SET join_on_disk_max_files_to_merge = 4; diff --git a/tests/queries/0_stateless/01304_direct_io_long.sh b/tests/queries/0_stateless/01304_direct_io_long.sh index a66239058ab..35d1440bcb5 100755 --- a/tests/queries/0_stateless/01304_direct_io_long.sh +++ b/tests/queries/0_stateless/01304_direct_io_long.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT --max_rows_to_read 50M --multiquery " INSERT INTO bug SELECT rand64(), '2020-06-07' FROM numbers(50000000); OPTIMIZE TABLE bug FINAL;" LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE" -$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$LOG" +$CLICKHOUSE_BENCHMARK --max_rows_to_read 51M --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$LOG" cat "$LOG" | grep Exception cat "$LOG" | grep Loaded diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index 6625ad916e8..6172afbc699 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -12,7 +12,7 @@ select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by n -- and the query with GROUP BY on remote servers will first do GROUP BY and then send the block, -- so the initiator will first receive all blocks from remotes and only after start merging, -- and will hit the memory limit. -select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='20Mi', max_block_size=4294967296; -- { serverError MEMORY_LIMIT_EXCEEDED } +select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='20Mi', max_block_size=4294967296, max_rows_to_read=0; -- { serverError MEMORY_LIMIT_EXCEEDED } -- with optimize_aggregation_in_order=1 remote servers will produce blocks more frequently, -- since they don't need to wait until the aggregation will be finished, diff --git a/tests/queries/0_stateless/02151_lc_prefetch.sql b/tests/queries/0_stateless/02151_lc_prefetch.sql index c2b97231145..f8c76038120 100644 --- a/tests/queries/0_stateless/02151_lc_prefetch.sql +++ b/tests/queries/0_stateless/02151_lc_prefetch.sql @@ -3,5 +3,6 @@ drop table if exists tab_lc; CREATE TABLE tab_lc (x UInt64, y LowCardinality(String)) engine = MergeTree order by x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into tab_lc select number, toString(number % 10) from numbers(20000000); optimize table tab_lc; +SET max_rows_to_read = '21M'; select count() from tab_lc where y == '0' settings local_filesystem_read_prefetch=1; drop table if exists tab_lc; diff --git a/tests/queries/0_stateless/02344_insert_profile_events_stress.sql b/tests/queries/0_stateless/02344_insert_profile_events_stress.sql index e9a790bea5d..902e1da543c 100644 --- a/tests/queries/0_stateless/02344_insert_profile_events_stress.sql +++ b/tests/queries/0_stateless/02344_insert_profile_events_stress.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel, long, no-debug, no-tsan, no-msan, no-asan +SET max_rows_to_read = 0; create table data_02344 (key Int) engine=Null; -- 3e9 rows is enough to fill the socket buffer and cause INSERT hung. diff --git a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql index 105fb500461..82eb4c93e3d 100644 --- a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql +++ b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql @@ -1,5 +1,7 @@ -- Tags: long, no-tsan, no-msan, no-asan, no-ubsan, no-debug, no-object-storage +SET max_rows_to_read = '51M'; + DROP TABLE IF EXISTS t_2354_dist_with_external_aggr; create table t_2354_dist_with_external_aggr(a UInt64, b String, c FixedString(100)) engine = MergeTree order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; diff --git a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh index 55e6ac2f758..3e28e76d6da 100755 --- a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh +++ b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh @@ -11,7 +11,7 @@ echo "Parquet" DATA_FILE=$CUR_DIR/data_parquet/list_monotonically_increasing_offsets.parquet ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (list Array(Int64), json Nullable(String)) ENGINE = Memory" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} --max_memory_usage 10G -q "INSERT INTO parquet_load FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" ${CLICKHOUSE_CLIENT} --query="drop table parquet_load" diff --git a/tests/queries/0_stateless/02497_remote_disk_fat_column.sql b/tests/queries/0_stateless/02497_remote_disk_fat_column.sql index d97109b66f3..65519296602 100644 --- a/tests/queries/0_stateless/02497_remote_disk_fat_column.sql +++ b/tests/queries/0_stateless/02497_remote_disk_fat_column.sql @@ -2,7 +2,7 @@ set allow_suspicious_fixed_string_types=1; create table fat_granularity (x UInt32, fat FixedString(160000)) engine = MergeTree order by x settings storage_policy = 's3_cache'; -insert into fat_granularity select number, toString(number) || '_' from numbers(100000) settings max_block_size = 8192, max_insert_threads=8; +insert into fat_granularity select number, toString(number) || '_' from numbers(100000) settings max_block_size = 3000, max_insert_threads = 8, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; -- Too large sizes of FixedString to deserialize select x from fat_granularity prewhere fat like '256\_%' settings max_threads=2; diff --git a/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql index 3e131cad0f0..ecaad62b35a 100644 --- a/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql +++ b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql @@ -1,6 +1,6 @@ -- Tags: no-fasttest -SET max_rows_to_read = 0; +SET max_rows_to_read = 0, max_execution_time = 0, max_estimated_execution_time = 0; -- Query stops after timeout without an error SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=2, timeout_overflow_mode='break' FORMAT Null; diff --git a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql index a635fd2e86a..18d98dbdfe4 100644 --- a/tests/queries/0_stateless/03143_asof_join_ddb_long.sql +++ b/tests/queries/0_stateless/03143_asof_join_ddb_long.sql @@ -35,6 +35,7 @@ AS 3 AS key FROM numbers(0, 4000000); +SET max_rows_to_read = 0; SELECT SUM(value), COUNT(*) FROM skewed_probe From 9c7078bcf7edfc79dbea2cf6e065aa594810ccaf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 02:54:11 +0200 Subject: [PATCH 0405/1722] Update tests --- tests/queries/0_stateless/00974_query_profiler.sql | 1 + ...0_distributed_group_by_no_merge_order_by_long.sql | 3 ++- .../0_stateless/02122_join_group_by_timeout.sh | 8 ++++---- ...ibuted_with_external_aggregation_memory_usage.sql | 2 +- ..._parquet_list_monotonically_increasing_offsets.sh | 2 +- .../02884_parallel_window_functions.reference | 12 ++++++------ 6 files changed, 15 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/00974_query_profiler.sql b/tests/queries/0_stateless/00974_query_profiler.sql index 24e4241b813..71ea14c3d64 100644 --- a/tests/queries/0_stateless/00974_query_profiler.sql +++ b/tests/queries/0_stateless/00974_query_profiler.sql @@ -15,6 +15,7 @@ SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FRO SET query_profiler_real_time_period_ns = 0; SET query_profiler_cpu_time_period_ns = 1000000; SET log_queries = 1; +SET max_rows_to_read = 0; SELECT count(), ignore('test cpu time query profiler') FROM numbers_mt(10000000000); SET log_queries = 0; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index 6172afbc699..e980f367de7 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -12,7 +12,8 @@ select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by n -- and the query with GROUP BY on remote servers will first do GROUP BY and then send the block, -- so the initiator will first receive all blocks from remotes and only after start merging, -- and will hit the memory limit. -select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='20Mi', max_block_size=4294967296, max_rows_to_read=0; -- { serverError MEMORY_LIMIT_EXCEEDED } +SET max_rows_to_read = 0; +select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='20Mi', max_block_size=4294967296; -- { serverError MEMORY_LIMIT_EXCEEDED } -- with optimize_aggregation_in_order=1 remote servers will produce blocks more frequently, -- since they don't need to wait until the aggregation will be finished, diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.sh b/tests/queries/0_stateless/02122_join_group_by_timeout.sh index 59719f75d7c..79c4f01c98a 100755 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.sh +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.sh @@ -14,7 +14,7 @@ MAX_PROCESS_WAIT=5 # TCP CLIENT: As of today (02/12/21) uses PullingAsyncPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) -timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \ +timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_result_rows 0 --max_result_bytes 0 --max_execution_time 1 -q \ "SELECT * FROM ( SELECT a.name as n @@ -31,7 +31,7 @@ timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \ FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq ### Should stop pulling data and return what has been generated already (return code 0) -timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT -q \ +timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_result_rows 0 --max_result_bytes 0 -q \ "SELECT a.name as n FROM ( @@ -48,7 +48,7 @@ echo $? # HTTP CLIENT: As of today (02/12/21) uses PullingPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) -${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d \ +${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_result_rows=0&max_result_bytes=0&max_execution_time=1" -d \ "SELECT * FROM ( SELECT a.name as n @@ -66,7 +66,7 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_exec ### Should stop pulling data and return what has been generated already (return code 0) -${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \ +${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_result_rows=0&max_result_bytes=0" -d \ "SELECT a.name as n FROM ( diff --git a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql index 82eb4c93e3d..5eea6f149b5 100644 --- a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql +++ b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql @@ -1,6 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-asan, no-ubsan, no-debug, no-object-storage -SET max_rows_to_read = '51M'; +SET max_rows_to_read = '101M'; DROP TABLE IF EXISTS t_2354_dist_with_external_aggr; diff --git a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh index 3e28e76d6da..2f512697868 100755 --- a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh +++ b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh @@ -12,6 +12,6 @@ DATA_FILE=$CUR_DIR/data_parquet/list_monotonically_increasing_offsets.parquet ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (list Array(Int64), json Nullable(String)) ENGINE = Memory" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} --max_memory_usage 10G -q "INSERT INTO parquet_load FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum +${CLICKHOUSE_CLIENT} --max_result_rows 0 --max_result_bytes 0 --query="SELECT * FROM parquet_load" | md5sum ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" ${CLICKHOUSE_CLIENT} --query="drop table parquet_load" diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.reference b/tests/queries/0_stateless/02884_parallel_window_functions.reference index bac15838dc2..a2cc96dda74 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.reference +++ b/tests/queries/0_stateless/02884_parallel_window_functions.reference @@ -12,7 +12,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading GROUP BY ac, nw ) GROUP BY nw @@ -32,7 +32,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading GROUP BY ac, nw ) GROUP BY nw @@ -53,7 +53,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 0 GROUP BY ac, @@ -64,7 +64,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 1 GROUP BY ac, @@ -75,7 +75,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 2 GROUP BY ac, @@ -86,7 +86,7 @@ FROM AVG(wg) AS WR, ac, nw - FROM window_funtion_threading + FROM window_function_threading WHERE (ac % 4) = 3 GROUP BY ac, From a03f12fe76969bc2002bc76ce04f7097cb02295b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 14:17:17 +0200 Subject: [PATCH 0406/1722] Fix some tests --- ...2354_distributed_with_external_aggregation_memory_usage.sql | 2 +- .../0_stateless/02884_parallel_window_functions.reference | 1 + tests/queries/0_stateless/replication.lib | 3 ++- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql index 5eea6f149b5..f9da5b3a73c 100644 --- a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql +++ b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql @@ -25,6 +25,6 @@ select a, b, c, sum(a) as s from remote('127.0.0.{2,3}', currentDatabase(), t_2354_dist_with_external_aggr) group by a, b, c format Null -settings max_memory_usage = '5Gi'; +settings max_memory_usage = '5Gi', max_result_rows = 0, max_result_bytes = 0; DROP TABLE t_2354_dist_with_external_aggr; diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.reference b/tests/queries/0_stateless/02884_parallel_window_functions.reference index a2cc96dda74..1f5346a1484 100644 --- a/tests/queries/0_stateless/02884_parallel_window_functions.reference +++ b/tests/queries/0_stateless/02884_parallel_window_functions.reference @@ -42,6 +42,7 @@ SETTINGS max_threads = 1; 0 2 0 1 2 0 2 2 0 +SET max_rows_to_read = 40000000; SELECT nw, sum(WR) AS R, diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 05651531fba..dcac721859e 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -114,7 +114,8 @@ function check_replication_consistency() # it's important to disable prefer warmed unmerged parts because # otherwise it can read non-syncrhonized state of replicas - res=$($CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 -q \ + # also, disable the limit that is set for tests globally + res=$($CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 --max_rows_to_read=0 -q \ "SELECT if((countDistinct(data) as c) == 0, 1, c) FROM From 7f4eb59c42aa4f432c99fa7e4f8240056fa26b91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 14:33:40 +0200 Subject: [PATCH 0407/1722] Fix some tests --- .../01730_distributed_group_by_no_merge_order_by_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index e980f367de7..805e0b4fedb 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -12,7 +12,7 @@ select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by n -- and the query with GROUP BY on remote servers will first do GROUP BY and then send the block, -- so the initiator will first receive all blocks from remotes and only after start merging, -- and will hit the memory limit. -SET max_rows_to_read = 0; +SET max_rows_to_read = 0, max_result_rows = 0; select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='20Mi', max_block_size=4294967296; -- { serverError MEMORY_LIMIT_EXCEEDED } -- with optimize_aggregation_in_order=1 remote servers will produce blocks more frequently, From 80b925ec75983ea558be536d657bfd2d277f1fbc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 14:47:58 +0200 Subject: [PATCH 0408/1722] Update test --- ...arallel_replicas_join_algo_and_analyzer.sh | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh index 8cefa873940..2c5b5a2a07b 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.sh @@ -86,11 +86,11 @@ SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -123,12 +123,12 @@ SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=1" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -149,12 +149,12 @@ SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -174,12 +174,12 @@ SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -200,12 +200,12 @@ SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -225,12 +225,12 @@ SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | @@ -250,12 +250,12 @@ SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS max_rows_to_read=0, allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', +SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | From 342bbb53c65d32698d55649252e6bc29f2a557b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Jul 2024 14:49:36 +0200 Subject: [PATCH 0409/1722] It was an optimization, not a limit --- tests/config/users.d/limits.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/config/users.d/limits.yaml b/tests/config/users.d/limits.yaml index 23aaccf9298..63c4e884a9d 100644 --- a/tests/config/users.d/limits.yaml +++ b/tests/config/users.d/limits.yaml @@ -34,7 +34,6 @@ profiles: max_bytes_in_set: 10G max_rows_in_join: 10G max_bytes_in_join: 10G - max_rows_in_set_to_optimize_join: 1G max_rows_to_transfer: 1G max_bytes_to_transfer: 1G max_rows_in_distinct: 10G From d1c45662e8560177eb1e90d5ad159b58ff396f44 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 26 Jul 2024 20:24:41 +0000 Subject: [PATCH 0410/1722] Fix: missing conversion in IN operator with parallel replicas --- src/Analyzer/FunctionNode.cpp | 6 ------ ...arallel_replicas_lost_decimal_conversion.reference | 0 ...3209_parallel_replicas_lost_decimal_conversion.sql | 11 +++++++++++ 3 files changed, 11 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.reference create mode 100644 tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.sql diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index e98b04fe9a9..debed0983fd 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -239,12 +239,6 @@ ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const if (function_name == "_CAST" && !argument_nodes.empty() && argument_nodes[0]->getNodeType() == QueryTreeNodeType::CONSTANT) new_options.add_cast_for_constants = false; - /// Avoid cast for `IN tuple(...)` expression. - /// Tuples could be quite big, and adding a type may significantly increase query size. - /// It should be safe because set type for `column IN tuple` is deduced from `column` type. - if (isNameOfInFunction(function_name) && argument_nodes.size() > 1 && argument_nodes[1]->getNodeType() == QueryTreeNodeType::CONSTANT) - new_options.add_cast_for_constants = false; - const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) { diff --git a/tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.reference b/tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.sql b/tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.sql new file mode 100644 index 00000000000..bcc9dec306b --- /dev/null +++ b/tests/queries/0_stateless/03209_parallel_replicas_lost_decimal_conversion.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS t_03209 SYNC; + +CREATE TABLE t_03209 ( `a` Decimal(18, 0), `b` Decimal(18, 1), `c` Decimal(36, 0) ) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_03209', 'r1') ORDER BY tuple(); +INSERT INTO t_03209 VALUES ('33', '44.4', '35'); + +SET max_parallel_replicas = 2, cluster_for_parallel_replicas='parallel_replicas'; + +SELECT * FROM t_03209 WHERE a IN toDecimal32('33.3000', 4) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT * FROM t_03209 WHERE a IN toDecimal32('33.3000', 4) SETTINGS allow_experimental_parallel_reading_from_replicas=1; + +DROP TABLE t_03209 SYNC; From 0d8a3f13e39c9e0be87c8459db9582c7c336617f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 26 Jul 2024 21:49:41 +0100 Subject: [PATCH 0411/1722] impl --- src/Processors/Sources/ShellCommandSource.cpp | 16 ++++++++++++---- .../test_executable_dictionary/test.py | 1 + 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 55eaf67eb3b..1659287c227 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -8,13 +8,15 @@ #include #include -#include -#include -#include -#include #include +#include +#include +#include +#include + #include +#include namespace DB { @@ -137,9 +139,15 @@ public: while (!bytes_read) { + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Starting polling on descriptors ({}) with timeout {} ms", + fmt::join(std::span(pfds, pfds + num_pfds) | std::views::transform([](const auto & pollfd) { return pollfd.fd; }), ", "), + timeout_milliseconds); pfds[0].revents = 0; pfds[1].revents = 0; size_t num_events = pollWithTimeout(pfds, num_pfds, timeout_milliseconds); + LOG_TRACE(getLogger("TimeoutReadBufferFromFileDescriptor"), "Poll returned with num_events={}", num_events); if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index 22f3442bb95..a1de429a235 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -245,6 +245,7 @@ def test_executable_input_slow_python(started_cluster): ) +@pytest.mark.repeat(50) def test_executable_implicit_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( From fd5934d0ad8954a263554ce48402849deafa5341 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 01:28:38 +0200 Subject: [PATCH 0412/1722] Update test --- .../01730_distributed_group_by_no_merge_order_by_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index 805e0b4fedb..6eb55839f5e 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -1,6 +1,7 @@ -- Tags: long, distributed, no-random-settings drop table if exists data_01730; +SET max_rows_to_read = 0, max_result_rows = 0; -- does not use 127.1 due to prefer_localhost_replica @@ -12,7 +13,6 @@ select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by n -- and the query with GROUP BY on remote servers will first do GROUP BY and then send the block, -- so the initiator will first receive all blocks from remotes and only after start merging, -- and will hit the memory limit. -SET max_rows_to_read = 0, max_result_rows = 0; select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='20Mi', max_block_size=4294967296; -- { serverError MEMORY_LIMIT_EXCEEDED } -- with optimize_aggregation_in_order=1 remote servers will produce blocks more frequently, From 16e84d1e3678b56d0b32dff1377b6daadf870dd2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 02:37:54 +0200 Subject: [PATCH 0413/1722] Update test --- tests/queries/0_stateless/01651_lc_insert_tiny_log.sql | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql index d11c9120c61..bc5553ad227 100644 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql @@ -1,7 +1,7 @@ -set allow_suspicious_low_cardinality_types=1; +set allow_suspicious_low_cardinality_types = 1, max_rows_to_read = '21M'; drop table if exists perf_lc_num; -CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = TinyLog; +CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = TinyLog; INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); @@ -16,7 +16,7 @@ select sum(length(arr)), sum(num) from perf_lc_num; drop table if exists perf_lc_num; -CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = Log; +CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = Log; INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); @@ -31,7 +31,7 @@ select sum(length(arr)), sum(num) from perf_lc_num; drop table if exists perf_lc_num; -CREATE TABLE perf_lc_num(  num UInt8,  arr Array(LowCardinality(Int64)) default [num]  ) ENGINE = StripeLog; +CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = StripeLog; INSERT INTO perf_lc_num (num) SELECT toUInt8(number) FROM numbers(10000000); @@ -44,5 +44,3 @@ select sum(length(arr)) from perf_lc_num; select sum(length(arr)), sum(num) from perf_lc_num; drop table if exists perf_lc_num; - - From 59ce7ec1c6f478b3ac700a612fea46e7fc12756b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 04:26:46 +0200 Subject: [PATCH 0414/1722] Update 01651_lc_insert_tiny_log.sql --- tests/queries/0_stateless/01651_lc_insert_tiny_log.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql index bc5553ad227..b1d6a39d5c9 100644 --- a/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql +++ b/tests/queries/0_stateless/01651_lc_insert_tiny_log.sql @@ -1,4 +1,4 @@ -set allow_suspicious_low_cardinality_types = 1, max_rows_to_read = '21M'; +set allow_suspicious_low_cardinality_types = 1, max_rows_to_read = '31M'; drop table if exists perf_lc_num; CREATE TABLE perf_lc_num( num UInt8, arr Array(LowCardinality(Int64)) default [num] ) ENGINE = TinyLog; From 9a6de84559cdf927e8747e4cd536676fd3b1c513 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 10:39:55 +0200 Subject: [PATCH 0415/1722] Update tests --- .../00086_concat_nary_const_with_nonconst_segfault.sql | 2 +- .../01730_distributed_group_by_no_merge_order_by_long.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00086_concat_nary_const_with_nonconst_segfault.sql b/tests/queries/0_stateless/00086_concat_nary_const_with_nonconst_segfault.sql index 2f0ef648983..4b87b2af28d 100644 --- a/tests/queries/0_stateless/00086_concat_nary_const_with_nonconst_segfault.sql +++ b/tests/queries/0_stateless/00086_concat_nary_const_with_nonconst_segfault.sql @@ -1 +1 @@ -SELECT extract(toString(number), '10000000') FROM system.numbers_mt WHERE concat(materialize('1'), '...', toString(number)) LIKE '%10000000%' LIMIT 1 +SELECT extract(toString(number), '10000000') FROM system.numbers_mt WHERE concat(materialize('1'), '...', toString(number)) LIKE '%10000000%' LIMIT 1 SETTINGS max_rows_to_read = 0; diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index 6eb55839f5e..83a26c83005 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -1,7 +1,7 @@ -- Tags: long, distributed, no-random-settings drop table if exists data_01730; -SET max_rows_to_read = 0, max_result_rows = 0; +SET max_rows_to_read = 0, max_result_rows = 0, max_bytes_before_external_group_by = 0; -- does not use 127.1 due to prefer_localhost_replica From 3db505a1327fc5bf96c93f2a510436402be13f3b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 27 Jul 2024 12:53:38 +0200 Subject: [PATCH 0416/1722] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 1dda9e72084..dc3bf984cc6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -67,6 +67,7 @@ static std::initializer_list Date: Sat, 27 Jul 2024 16:10:15 +0200 Subject: [PATCH 0417/1722] Better limits --- tests/config/users.d/limits.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/users.d/limits.yaml b/tests/config/users.d/limits.yaml index 63c4e884a9d..46cff73142c 100644 --- a/tests/config/users.d/limits.yaml +++ b/tests/config/users.d/limits.yaml @@ -38,7 +38,7 @@ profiles: max_bytes_to_transfer: 1G max_rows_in_distinct: 10G max_bytes_in_distinct: 10G - max_memory_usage_for_user: 10G + max_memory_usage_for_user: 32G max_network_bandwidth: 100G max_network_bytes: 1T max_network_bandwidth_for_user: 100G From 4c7d93dcc87ac4056c73356c155e441e1dffba2a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 27 Jul 2024 17:03:19 +0000 Subject: [PATCH 0418/1722] Update test output --- ...imize_skip_unused_shards_rewrite_in.reference | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 28dbb9215a8..0a7a9d64208 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -24,8 +24,8 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_no%') and type = 'QueryFinish' order by query; - (0, 2) - (0, 2) + _CAST((0, 2), \'Tuple(UInt8, UInt8)\') + _CAST((0, 2), \'Tuple(UInt8, UInt8)\') -- -- w/ optimize_skip_unused_shards_rewrite_in=1 -- @@ -45,8 +45,8 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_02%') and type = 'QueryFinish' order by query; - tuple(0) - tuple(2) + _CAST(tuple(0), \'Tuple(UInt8)\') + _CAST(tuple(2), \'Tuple(UInt8)\') select 'optimize_skip_unused_shards_rewrite_in(2,)'; optimize_skip_unused_shards_rewrite_in(2,) with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2,); @@ -59,7 +59,7 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_2%') and type = 'QueryFinish' order by query; - tuple(2) + _CAST(tuple(2), \'Tuple(UInt8)\') select 'optimize_skip_unused_shards_rewrite_in(0,)'; optimize_skip_unused_shards_rewrite_in(0,) with (select currentDatabase()) as id_00 select *, ignore(id_00) from dist_01756 where dummy in (0,); @@ -73,7 +73,7 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_00%') and type = 'QueryFinish' order by query; - tuple(0) + _CAST(tuple(0), \'Tuple(UInt8)\') -- signed column select 'signed column'; signed column @@ -88,8 +88,8 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%key_signed%') and type = 'QueryFinish' order by query; - tuple(-1) - tuple(-2) + _CAST(tuple(-1), \'Tuple(Int8)\') + _CAST(tuple(-2), \'Tuple(Int8)\') -- not tuple select * from dist_01756 where dummy in (0); 0 From b1eaec0d49b326df09c46ea3107e98fa083c220d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Jul 2024 23:29:52 +0200 Subject: [PATCH 0419/1722] Update 01293_show_settings.reference --- tests/queries/0_stateless/01293_show_settings.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index 8b383813c9f..c4c3473ee18 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -6,6 +6,6 @@ external_storage_connect_timeout_sec UInt64 10 s3_connect_timeout_ms UInt64 1000 filesystem_prefetch_max_memory_usage UInt64 1073741824 max_memory_usage UInt64 5000000000 -max_memory_usage_for_user UInt64 10000000000 +max_memory_usage_for_user UInt64 32000000000 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 From cd1350c0f380d2c2754231c12aefb891c299b3fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 14:07:36 +0200 Subject: [PATCH 0420/1722] Mark some tests as long --- .../0_stateless/00111_shard_external_sort_distributed.sql | 2 +- .../00377_shard_group_uniq_array_of_string_array.sql | 2 +- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 2 ++ tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql index ef9c0f9f9d0..93efc317bfa 100644 --- a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql +++ b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql @@ -1,4 +1,4 @@ --- Tags: distributed +-- Tags: distributed, long SET max_memory_usage = 150000000; SET max_bytes_before_external_sort = 10000000; diff --git a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql index 180a6a04861..1ec91ac2396 100644 --- a/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql +++ b/tests/queries/0_stateless/00377_shard_group_uniq_array_of_string_array.sql @@ -1,4 +1,4 @@ --- Tags: shard +-- Tags: shard, long SET max_rows_to_read = '55M'; DROP TABLE IF EXISTS group_uniq_arr_str; diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index 9c1abe1b6df..337fba865fd 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,3 +1,5 @@ +-- Tags: long + SET max_rows_to_read = '100M'; drop table if exists lc_00906; create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index b68d15a2200..85be5082d92 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan +-- Tags: no-tsan, long -- Tag no-tsan: Too long for TSan set enable_filesystem_cache=0; From 1042fc68c2969ee4963268a97daf522d0e163ac5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 16:41:43 +0200 Subject: [PATCH 0421/1722] Update test --- .../queries/0_stateless/00632_get_sample_block_cache.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00632_get_sample_block_cache.sql b/tests/queries/0_stateless/00632_get_sample_block_cache.sql index ae9b6bb7b2c..6a226c4912a 100644 --- a/tests/queries/0_stateless/00632_get_sample_block_cache.sql +++ b/tests/queries/0_stateless/00632_get_sample_block_cache.sql @@ -2,6 +2,9 @@ SET joined_subquery_requires_alias = 0; +-- We are no longer interested in the old analyzer. +SET allow_experimental_analyzer = 1; + -- This test (SELECT) without cache can take tens minutes DROP TABLE IF EXISTS dict_string; DROP TABLE IF EXISTS dict_ui64; @@ -41,8 +44,6 @@ SETTINGS index_granularity = 8192; CREATE TABLE dict_string (entityIri String) ENGINE = Memory; CREATE TABLE dict_ui64 (learnerId UInt64) ENGINE = Memory; ---SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count`, `views-count`, `time-before-full-watched-average`, if (isNaN((`overall-full-watched-learners-count`/`overall-watchers-count`) * 100), 0, (`overall-full-watched-learners-count`/`overall-watchers-count`) * 100) as `overall-watched-part`, if (isNaN((`full-watched-learners-count`/`watchers-count` * 100)), 0, (`full-watched-learners-count`/`watchers-count` * 100)) as `full-watched-part`, if (isNaN((`rejects-count`/`views-count` * 100)), 0, (`rejects-count`/`views-count` * 100)) as `rejects-part` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count`, `views-count`, `time-before-full-watched-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count`, `views-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average` FROM (SELECT `entityIri`, `watchers-count` FROM (SELECT `entityIri` FROM `CloM8CwMR2`) ANY LEFT JOIN (SELECT uniq(learnerId) as `watchers-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(viewDurationSum) as `time-repeating-average`, `entityIri` FROM (SELECT sum(views.viewDuration) as viewDurationSum, `entityIri`, `learnerId` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `views`.`repeatingView` = 1 AND `learnerId` IN `tkRpHxGqM1` GROUP BY `learnerId`, `entityIri`) GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(views.viewDuration) as `reject-views-duration-average`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `views`.`reject` = 1 AND `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(viewsCount) as `repeating-views-count-average`, `entityIri` FROM (SELECT count() as viewsCount, `learnerId`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `courseId` = 1 AND `entityIri` IN `CloM8CwMR2` WHERE `views`.`repeatingView` = 1 AND `learnerId` IN `tkRpHxGqM1` GROUP BY `learnerId`, `entityIri`) GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(views.viewDuration) as `views-duration-average`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(views.watchedPart) as `watched-part-average`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT count() as `rejects-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `views`.`reject` = 1 AND `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(progressMax) as `progress-average`, `entityIri` FROM (SELECT max(views.progress) as progressMax, `entityIri`, `learnerId` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `learnerId`, `entityIri`) GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(fullWatchedViews) as `views-count-before-full-watched-average`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT any(duration) as `duration`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT uniq(learnerId) as `full-watched-learners-count`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `fullWatched` = 1 AND `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT uniq(learnerId) as `overall-watchers-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT uniq(learnerId) as `overall-full-watched-learners-count`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `fullWatched` = 1 AND `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT count() as `views-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(fullWatchedTime) as `time-before-full-watched-average`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN `CloM8CwMR2` AND `courseId` = 1 WHERE `learnerId` IN `tkRpHxGqM1` GROUP BY `entityIri`) USING `entityIri`) FORMAT JSON; - SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count`, `views-count`, `time-before-full-watched-average`, if (isNaN((`overall-full-watched-learners-count`/`overall-watchers-count`) * 100), 0, (`overall-full-watched-learners-count`/`overall-watchers-count`) * 100) as `overall-watched-part`, if (isNaN((`full-watched-learners-count`/`watchers-count` * 100)), 0, (`full-watched-learners-count`/`watchers-count` * 100)) as `full-watched-part`, if (isNaN((`rejects-count`/`views-count` * 100)), 0, (`rejects-count`/`views-count` * 100)) as `rejects-part` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count`, `views-count`, `time-before-full-watched-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count`, `views-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count`, `overall-full-watched-learners-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count`, `overall-watchers-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration`, `full-watched-learners-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average`, `duration` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average`, `views-count-before-full-watched-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count`, `progress-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average`, `rejects-count` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average`, `watched-part-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average`, `views-duration-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average`, `repeating-views-count-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average`, `reject-views-duration-average` FROM (SELECT `entityIri`, `watchers-count`, `time-repeating-average` FROM (SELECT `entityIri`, `watchers-count` FROM (SELECT `entityIri` FROM dict_string) ANY LEFT JOIN (SELECT uniq(learnerId) as `watchers-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(viewDurationSum) as `time-repeating-average`, `entityIri` FROM (SELECT sum(views.viewDuration) as viewDurationSum, `entityIri`, `learnerId` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `views`.`repeatingView` = 1 AND `learnerId` IN dict_ui64 GROUP BY `learnerId`, `entityIri`) GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(views.viewDuration) as `reject-views-duration-average`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `views`.`reject` = 1 AND `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(viewsCount) as `repeating-views-count-average`, `entityIri` FROM (SELECT count() as viewsCount, `learnerId`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `courseId` = 1 AND `entityIri` IN dict_string WHERE `views`.`repeatingView` = 1 AND `learnerId` IN dict_ui64 GROUP BY `learnerId`, `entityIri`) GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(views.viewDuration) as `views-duration-average`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(views.watchedPart) as `watched-part-average`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT count() as `rejects-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `views`.`reject` = 1 AND `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(progressMax) as `progress-average`, `entityIri` FROM (SELECT max(views.progress) as progressMax, `entityIri`, `learnerId` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `learnerId`, `entityIri`) GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(fullWatchedViews) as `views-count-before-full-watched-average`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT any(duration) as `duration`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT uniq(learnerId) as `full-watched-learners-count`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `fullWatched` = 1 AND `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT uniq(learnerId) as `overall-watchers-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT uniq(learnerId) as `overall-full-watched-learners-count`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `fullWatched` = 1 AND `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT count() as `views-count`, `entityIri` FROM `video_views` FINAL ARRAY JOIN `views` PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`) ANY LEFT JOIN (SELECT avg(fullWatchedTime) as `time-before-full-watched-average`, `entityIri` FROM `video_views` FINAL PREWHERE `entityIri` IN dict_string AND `courseId` = 1 WHERE `learnerId` IN dict_ui64 GROUP BY `entityIri`) USING `entityIri`); @@ -55,7 +56,7 @@ DROP TABLE video_views; --- Test for tsan: Ensure cache used from one thread +-- Test for tsan: Ensure cache is used from one thread SET max_threads = 32; DROP TABLE IF EXISTS sample_00632; @@ -173,7 +174,6 @@ FROM UNION ALL SELECT * FROM ( SELECT * FROM sample_00632 WHERE x > 0 ) ) GROUP BY x - --HAVING c = 1 ORDER BY x ASC ); DROP TABLE sample_00632; From 81714ce561c99710c3350a40d662966d5bb1a86a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 16:44:53 +0200 Subject: [PATCH 0422/1722] Make test simpler --- tests/queries/0_stateless/02585_query_status_deadlock.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02585_query_status_deadlock.sh b/tests/queries/0_stateless/02585_query_status_deadlock.sh index 6321ac0064a..932cf593393 100755 --- a/tests/queries/0_stateless/02585_query_status_deadlock.sh +++ b/tests/queries/0_stateless/02585_query_status_deadlock.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) QUERY_ID="${CLICKHOUSE_DATABASE}_test_02585_query_to_kill_id_1" $CLICKHOUSE_CLIENT --query_id="$QUERY_ID" --max_rows_to_read 0 -n -q " -create temporary table tmp as select * from numbers(500000000); +create temporary table tmp as select * from numbers(100000000); select * from remote('127.0.0.2', 'system.numbers_mt') where number in (select * from tmp);" &> /dev/null & $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" @@ -19,7 +19,7 @@ do if [ -n "$res" ]; then break fi - sleep 1 + sleep 1 done $CLICKHOUSE_CLIENT -q "kill query where query_id = '$QUERY_ID' sync" &> /dev/null From aec346676127abba85886a828e663ebf05cfa81e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jul 2024 23:07:28 +0200 Subject: [PATCH 0423/1722] Update test --- tests/queries/0_stateless/01161_all_system_tables.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index 739df782a39..d4a80d074dc 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -19,7 +19,7 @@ function run_selects() thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name != 'zookeeper' and name != 'models' - AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num AND name NOT LIKE '%\\_sender' AND name NOT LIKE '%\\_watcher'") + AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num AND name NOT LIKE '%\\_sender' AND name NOT LIKE '%\\_watcher' AND name != 'coverage_log'") for t in "${tables_arr[@]}" do From e7cd07510aee8769cd31c4cbfa6a86d6198d37f5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 29 Jul 2024 13:03:21 +0200 Subject: [PATCH 0424/1722] Move setting to 24.8 version --- src/Core/SettingsChangesHistory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index dc3bf984cc6..41319ac7645 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,8 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.8", {{"input_format_try_infer_variants", false, false, "Try to infer Variant type in text formats when there is more than one possible type for column/array elements"}, + }}, {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, @@ -66,7 +68,6 @@ static std::initializer_list Date: Mon, 29 Jul 2024 12:14:53 +0000 Subject: [PATCH 0425/1722] Fix first batch of review --- src/Columns/ColumnObject.cpp | 62 ++++++++++++++++++++++++++---------- src/Columns/ColumnObject.h | 29 ++++++++++++++--- 2 files changed, 70 insertions(+), 21 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 98cffdf32bb..8649e2314b9 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -93,7 +93,13 @@ ColumnObject::Ptr ColumnObject::create( for (const auto & [path, column] : dynamic_paths_) mutable_dynamic_paths[path] = dynamic_paths_.at(path)->assumeMutable(); - return ColumnObject::create(std::move(mutable_typed_paths), std::move(mutable_dynamic_paths), shared_data_->assumeMutable(), max_dynamic_paths_, max_dynamic_types_, statistics_); + return ColumnObject::create( + std::move(mutable_typed_paths), + std::move(mutable_dynamic_paths), + shared_data_->assumeMutable(), + max_dynamic_paths_, + max_dynamic_types_, + statistics_); } ColumnObject::MutablePtr ColumnObject::create( @@ -141,7 +147,13 @@ MutableColumnPtr ColumnObject::cloneEmpty() const for (const auto & [path, column] : dynamic_paths) empty_dynamic_paths[path] = column->cloneEmpty(); - return ColumnObject::create(std::move(empty_typed_paths), std::move(empty_dynamic_paths), shared_data->cloneEmpty(), max_dynamic_paths, max_dynamic_types, statistics); + return ColumnObject::create( + std::move(empty_typed_paths), + std::move(empty_dynamic_paths), + shared_data->cloneEmpty(), + max_dynamic_paths, + max_dynamic_types, + statistics); } MutableColumnPtr ColumnObject::cloneResized(size_t size) const @@ -156,7 +168,13 @@ MutableColumnPtr ColumnObject::cloneResized(size_t size) const for (const auto & [path, column] : dynamic_paths) resized_dynamic_paths[path] = column->cloneResized(size); - return ColumnObject::create(std::move(resized_typed_paths), std::move(resized_dynamic_paths), shared_data->cloneResized(size), max_dynamic_paths, max_dynamic_types, statistics); + return ColumnObject::create( + std::move(resized_typed_paths), + std::move(resized_dynamic_paths), + shared_data->cloneResized(size), + max_dynamic_paths, + max_dynamic_types, + statistics); } Field ColumnObject::operator[](size_t n) const @@ -316,6 +334,9 @@ bool ColumnObject::tryInsert(const Field & x) size_t prev_size = size(); size_t prev_paths_size = shared_data_paths->size(); size_t prev_values_size = shared_data_values->size(); + /// Save all newly added dynamic paths. In case of failure + /// we should remove them. + std::unordered_set new_dynamic_paths; auto restore_sizes = [&]() { for (auto & [_, column] : typed_paths) @@ -324,6 +345,13 @@ bool ColumnObject::tryInsert(const Field & x) column->popBack(column->size() - prev_size); } + /// Remove all newly added dynamic paths. + for (const auto & path : new_dynamic_paths) + { + dynamic_paths_ptrs.erase(path); + dynamic_paths.erase(path); + } + for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() != prev_size) @@ -422,7 +450,7 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) } /// Finally, insert paths from shared data. - insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, src_dynamic_paths_for_shared_data, n, 1); + insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, std::move(src_dynamic_paths_for_shared_data), n, 1); } #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -456,10 +484,10 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l } /// Finally, insert paths from shared data. - insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, src_dynamic_paths_for_shared_data, start, length); + insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, std::move(src_dynamic_paths_for_shared_data), start, length); } -void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::ColumnObject & src_object_column, std::vector & src_dynamic_paths_for_shared_data, size_t start, size_t length) +void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length) { /// Paths in shared data are sorted, so paths from src_dynamic_paths_for_shared_data should be inserted properly /// to keep paths sorted. Let's sort them in advance. @@ -527,7 +555,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co while (src_dynamic_paths_for_shared_data_index < src_dynamic_paths_for_shared_data.size() && src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index] < path) { - auto dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; + const auto & dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.at(dynamic_path), row); ++src_dynamic_paths_for_shared_data_index; } @@ -541,7 +569,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co /// Insert remaining dynamic paths from src_dynamic_paths_for_shared_data. for (; src_dynamic_paths_for_shared_data_index != src_dynamic_paths_for_shared_data.size(); ++src_dynamic_paths_for_shared_data_index) { - auto dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; + const auto & dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.at(dynamic_path), row); } @@ -635,7 +663,7 @@ StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const c { WriteBufferFromOwnString buf; getDynamicSerialization()->serializeBinary(*column, n, buf, getFormatSettings()); - serializePathAndValueIntoArena(arena, begin, path, buf.str(), res); + serializePathAndValueIntoArena(arena, begin, path, buf.str(), res); } /// Serialize paths and values from shared data. @@ -664,15 +692,15 @@ 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> paths_and_values_for_shared_data; + std::vector> paths_and_values_for_shared_data; auto num_paths = unalignedLoad(pos); pos += sizeof(size_t); for (size_t i = 0; i != num_paths; ++i) { auto path_size = unalignedLoad(pos); pos += sizeof(size_t); - StringRef path(pos, path_size); - String path_str = path.toString(); + std::string_view path(pos, path_size); + String path_str(path); pos += path_size; /// Check if it's a typed path. In this case we should use /// deserializeAndInsertFromArena of corresponding column. @@ -687,18 +715,18 @@ const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) auto value_size = unalignedLoad(pos); pos += sizeof(size_t); - StringRef value(pos, value_size); + std::string_view value(pos, value_size); pos += value_size; /// Check if we have this path in dynamic paths. if (auto dynamic_it = dynamic_paths.find(path_str); dynamic_it != dynamic_paths.end()) { - ReadBufferFromMemory buf(value.data, value.size); + ReadBufferFromMemory buf(value.data(), value.size()); getDynamicSerialization()->deserializeBinary(*dynamic_it->second, buf, getFormatSettings()); } /// Try to add a new dynamic path. else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path_str)) { - ReadBufferFromMemory buf(value.data, value.size); + ReadBufferFromMemory buf(value.data(), value.size()); getDynamicSerialization()->deserializeBinary(*dynamic_path_column, buf, getFormatSettings()); } /// Limit on dynamic paths is reached, add this path to shared data later. @@ -714,8 +742,8 @@ const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); for (const auto & [path, value] : paths_and_values_for_shared_data) { - shared_data_paths->insertData(path.data, path.size); - shared_data_values->insertData(value.data, value.size); + shared_data_paths->insertData(path.data(), path.size()); + shared_data_values->insertData(value.data(), value.size()); } getSharedDataOffsets().push_back(shared_data_paths->size()); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index d274f4e857a..fbb68897e08 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -36,15 +36,36 @@ private: friend class COWHelper, ColumnObject>; ColumnObject(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_); - ColumnObject(std::unordered_map typed_paths_, std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); + ColumnObject( + std::unordered_map typed_paths_, + std::unordered_map dynamic_paths_, + MutableColumnPtr shared_data_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_, + const Statistics & statistics_ = {}); public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use mutate in order to make mutable column and mutate shared nested columns. */ using Base = COWHelper, ColumnObject>; - static Ptr create(const std::unordered_map & typed_paths_, const std::unordered_map & dynamic_paths_, const ColumnPtr & shared_data_, size_t max_dynamic_paths_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); - static MutablePtr create(std::unordered_map typed_paths_, std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); + + static Ptr create( + const std::unordered_map & typed_paths_, + const std::unordered_map & dynamic_paths_, + const ColumnPtr & shared_data_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_, + const Statistics & statistics_ = {}); + + static MutablePtr create( + std::unordered_map typed_paths_, + std::unordered_map dynamic_paths_, + MutableColumnPtr shared_data_, + size_t max_dynamic_paths_, + size_t max_dynamic_types_, + const Statistics & statistics_ = {}); + static MutablePtr create(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_); std::string getName() const override; @@ -191,7 +212,7 @@ public: static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); private: - void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector & src_dynamic_paths_for_shared_data, size_t start, size_t length); + void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length); void serializePathAndValueIntoArena(Arena & arena, const char *& begin, StringRef path, StringRef value, StringRef & res) const; /// Map path -> column for paths with explicitly specified types. From d5065a43ae4ae5ba0f068e3fdf5952dd5319f561 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 17:55:01 +0200 Subject: [PATCH 0426/1722] Update StorageObjectStorage.cpp --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ca0ced8dcd3..d9c82d68791 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -39,11 +39,16 @@ String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, Con auto query_settings = configuration->getQuerySettings(context); /// We don't want to throw an exception if there are no files with specified path. query_settings.throw_on_zero_files_match = false; + + bool local_distributed_processing = distributed_processing; + if (context->getSettingsRef().use_hive_partitioning) + local_distributed_processing = false; + auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, query_settings, object_storage, - distributed_processing, + local_distributed_processing, context, {}, // predicate metadata.getColumns().getAll(), // virtual_columns From f9a5210bacc418e354ddcf8893fa8c5a291b46d4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 19:36:31 +0200 Subject: [PATCH 0427/1722] solve Alexey's review --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 87c1aecc3a7..257a77547c0 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -127,7 +127,7 @@ NameSet getVirtualNamesForFileLikeStorage() std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { - std::string pattern = "([^/]+)=([^/]+)/"; + std::string pattern = "([^/])=([^/]+)/"; re2::StringPiece input_piece(path); std::unordered_map key_values; From e517338182b79aaa70c6b3e2d15c499acccc4d88 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Jul 2024 21:16:19 +0200 Subject: [PATCH 0428/1722] Update tests --- tests/queries/0_stateless/replication.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index dcac721859e..36309cf0331 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -89,7 +89,7 @@ function check_replication_consistency() # Touch all data to check that it's readable (and trigger PartCheckThread if needed) # it's important to disable prefer warmed unmerged parts because # otherwise it can read non-syncrhonized state of replicas - while ! $CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 -q "SELECT * FROM merge(currentDatabase(), '$table_name_prefix') FORMAT Null" 2>/dev/null; do + while ! $CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 --max_result_rows 0 --max_result_bytes 0 --max_rows_to_read 0 --max_bytes_to_read 0 -q "SELECT * FROM merge(currentDatabase(), '$table_name_prefix') FORMAT Null" 2>/dev/null; do sleep 1; num_tries=$((num_tries+1)) if [ $num_tries -eq 250 ]; then From 6317979825794882905bc02b3a18dd82cfd8ec1c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 Jul 2024 20:53:11 +0100 Subject: [PATCH 0429/1722] add one more --- tests/integration/test_executable_table_function/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index 801a3c7c14a..a79616fc008 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -139,6 +139,7 @@ def test_executable_function_input_signalled_python(started_cluster): assert node.query(query.format(source="(SELECT id FROM test_data_table)")) == "" +@pytest.mark.repeat(50) def test_executable_function_input_slow_python(started_cluster): skip_test_msan(node) From d89019293e955452ede3e0abbe4b11ab2a3471bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 06:15:13 +0200 Subject: [PATCH 0430/1722] Update tests --- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 4 ++-- tests/queries/0_stateless/02700_s3_part_INT_MAX.sh | 2 +- tests/queries/1_stateful/00157_cache_dictionary.sql | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index 85be5082d92..f41b336be46 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -1,5 +1,5 @@ --- Tags: no-tsan, long --- Tag no-tsan: Too long for TSan +-- Tags: no-tsan, no-msan, long +-- too long. set enable_filesystem_cache=0; set enable_filesystem_cache_on_write_operations=0; diff --git a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh index c431686b594..cfb38c60615 100755 --- a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh +++ b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh @@ -12,7 +12,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # too slow with this. # # Unfortunately, the test has to buffer it in memory. -$CLICKHOUSE_CLIENT --max_memory_usage 10G -nm -q " +$CLICKHOUSE_CLIENT --max_memory_usage 16G -nm -q " INSERT INTO FUNCTION s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv', '', '', 'TSV') SELECT repeat('a', 1024) FROM numbers((pow(2, 30) * 2) / 1024) SETTINGS s3_max_single_part_upload_size = '5Gi'; diff --git a/tests/queries/1_stateful/00157_cache_dictionary.sql b/tests/queries/1_stateful/00157_cache_dictionary.sql index 3621ff82126..a7c6c099de6 100644 --- a/tests/queries/1_stateful/00157_cache_dictionary.sql +++ b/tests/queries/1_stateful/00157_cache_dictionary.sql @@ -9,7 +9,7 @@ ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS storage_policy = 'default'; -INSERT INTO test.hits_1m SELECT * FROM test.hits LIMIT 1000000; +INSERT INTO test.hits_1m SELECT * FROM test.hits LIMIT 1000000 SETTINGS min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; CREATE DATABASE IF NOT EXISTS db_dict; DROP DICTIONARY IF EXISTS db_dict.cache_hits; From b0e6b3e88930d3ca493dddb688235c64cec1d893 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Jul 2024 06:30:12 +0000 Subject: [PATCH 0431/1722] Kick off CI build From 56ba7c5d48cfa648f1d496d55cdfd50450da0299 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 09:33:17 +0200 Subject: [PATCH 0432/1722] Update a test --- tests/queries/0_stateless/00632_get_sample_block_cache.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00632_get_sample_block_cache.sql b/tests/queries/0_stateless/00632_get_sample_block_cache.sql index 6a226c4912a..a631cbb8b86 100644 --- a/tests/queries/0_stateless/00632_get_sample_block_cache.sql +++ b/tests/queries/0_stateless/00632_get_sample_block_cache.sql @@ -57,7 +57,7 @@ DROP TABLE video_views; -- Test for tsan: Ensure cache is used from one thread -SET max_threads = 32; +SET max_threads = 32, max_memory_usage = '10G'; DROP TABLE IF EXISTS sample_00632; From 16d7ff11be71df496a960ba833649df2a452a24b Mon Sep 17 00:00:00 2001 From: haozelong Date: Tue, 30 Jul 2024 17:16:19 +0800 Subject: [PATCH 0433/1722] Fix the problem that alter modfiy order by causes inconsistent metadata --- src/Storages/AlterCommands.cpp | 12 ++++++++++++ .../integration/test_replicated_database/test.py | 16 ++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7891042bb96..2bef3319020 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1245,6 +1245,13 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) { auto columns = metadata.columns; + auto ast_to_str = [](const ASTPtr & query) -> String + { + if (!query) + return ""; + return queryToString(query); + }; + for (size_t i = 0; i < size(); ++i) { auto & command = (*this)[i]; @@ -1277,6 +1284,11 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) if (!has_column && command.if_exists) command.ignore = true; } + else if (command.type == AlterCommand::MODIFY_ORDER_BY) + { + if (ast_to_str(command.order_by) == ast_to_str(metadata.sorting_key.definition_ast)) + command.ignore = true; + } } prepared = true; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 60a6e099b22..6344a4a6b00 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1549,3 +1549,19 @@ def test_all_groups_cluster(started_cluster): assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" ) + +def test_alter_modify_order_by(started_cluster): + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + + main_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');") + main_node.query("CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);") + main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") + snapshotting_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');") + + query = "show create table alter_modify_order_by.t1;" + expected = main_node.query(query) + assert_eq_with_retry(snapshotting_node, query, expected) + + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") From 0c9fa155d4993220c00e4b41c0354b20d3312f33 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 30 Jul 2024 11:58:35 +0200 Subject: [PATCH 0434/1722] revert last commit --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4edab01925d..f16eff7edb6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -131,7 +131,7 @@ NameSet getVirtualNamesForFileLikeStorage() std::unordered_map parseHivePartitioningKeysAndValues(const String & path, const ColumnsDescription & storage_columns) { - std::string pattern = "([^/])=([^/]+)/"; + std::string pattern = "([^/]+)=([^/]+)/"; re2::StringPiece input_piece(path); std::unordered_map key_values; From ec759699f3360df53bf21fa6149bfec1e3a08a8c Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 30 Jul 2024 21:07:59 +0800 Subject: [PATCH 0435/1722] format the tests/integration/test_replicated_database/test.py --- tests/integration/test_replicated_database/test.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 6344a4a6b00..72d45e45841 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1550,14 +1550,21 @@ def test_all_groups_cluster(started_cluster): "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" ) + def test_alter_modify_order_by(started_cluster): main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - main_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');") - main_node.query("CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);") + main_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');" + ) + main_node.query( + "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" + ) main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") - snapshotting_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');") + snapshotting_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');" + ) query = "show create table alter_modify_order_by.t1;" expected = main_node.query(query) From d3830e0a4fd96c286e5b91a78b1a9583f5eb6291 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 Jul 2024 14:54:42 +0000 Subject: [PATCH 0436/1722] Add perf tests for Dynamic and Variant --- .../insert_select_squashing_dynamic.xml | 59 +++++++++++++++++++ .../insert_select_squashing_variant.xml | 34 +++++++++++ 2 files changed, 93 insertions(+) create mode 100644 tests/performance/insert_select_squashing_dynamic.xml create mode 100644 tests/performance/insert_select_squashing_variant.xml diff --git a/tests/performance/insert_select_squashing_dynamic.xml b/tests/performance/insert_select_squashing_dynamic.xml new file mode 100644 index 00000000000..f7f600fd8bd --- /dev/null +++ b/tests/performance/insert_select_squashing_dynamic.xml @@ -0,0 +1,59 @@ + + + 1000 + 0 + 1 + + + +CREATE TABLE dynamic_squash_performance_1 +( + d Dynamic +) +ENGINE = Null; + + + +CREATE TABLE dynamic_squash_performance_2 +( + d Dynamic(max_types=6) +) +ENGINE = Null; + + + +CREATE TABLE src_dynamic_squash_performance_1 +( + d Dynamic +) +ENGINE = Memory; + + + +CREATE TABLE src_dynamic_squash_performance_2 +( + d Dynamic(max_types=6) +) +ENGINE = Memory; + + + + + + + + + + + + INSERT INTO dynamic_squash_performance_1 SELECT number::Dynamic FROM numbers(10000000) + INSERT INTO dynamic_squash_performance_1 SELECT range(number % 100)::Dynamic FROM numbers(2000000) + INSERT INTO dynamic_squash_performance_1 SELECT * FROM src_dynamic_squash_performance_1 + INSERT INTO dynamic_squash_performance_2 SELECT * FROM src_dynamic_squash_performance_2 + + DROP TABLE IF EXISTS dynamic_squash_performance_1 + DROP TABLE IF EXISTS dynamic_squash_performance_2 + DROP TABLE IF EXISTS src_dynamic_squash_performance_1 + DROP TABLE IF EXISTS src_dynamic_squash_performance_2 + + diff --git a/tests/performance/insert_select_squashing_variant.xml b/tests/performance/insert_select_squashing_variant.xml new file mode 100644 index 00000000000..5c59fc7b50f --- /dev/null +++ b/tests/performance/insert_select_squashing_variant.xml @@ -0,0 +1,34 @@ + + + 1000 + 0 + 1 + 1 + + + +CREATE TABLE variant_squash_performance +( + v Variant(Tuple(v1 Array(UInt64)), Tuple(v2 Array(UInt64)), Tuple(v3 Array(UInt64)), Tuple(v4 Array(UInt64)), Tuple(v5 Array(UInt64))) +) +ENGINE = Null; + + + +CREATE TABLE src_variant_squash_performance +( + v Variant(Tuple(v1 Array(UInt64)), Tuple(v2 Array(UInt64)), Tuple(v3 Array(UInt64)), Tuple(v4 Array(UInt64)), Tuple(v5 Array(UInt64))) +) +ENGINE = Memory; + + + + + + + INSERT INTO variant_squash_performance SELECT * FROM src_variant_squash_performance + + DROP TABLE IF EXISTS variant_squash_performance + DROP TABLE IF EXISTS src_variant_squash_performance + + From 9d0608ce001b35e17ff81c00d4965dbe4938b56b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 30 Jul 2024 16:55:37 +0200 Subject: [PATCH 0437/1722] Update Runner.cpp --- utils/keeper-bench/Runner.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index d99f2645a31..59761d827e1 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -545,8 +545,7 @@ struct ZooKeeperRequestFromLogReader file_read_buf = DB::wrapReadBufferWithCompressionMethod(std::move(file_read_buf), compression_method); DB::SingleReadBufferIterator read_buffer_iterator(std::move(file_read_buf)); - std::string sample_path; - auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, sample_path, context); + auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); DB::ColumnsWithTypeAndName columns; columns.reserve(columns_description.size()); From 08ecf6c6642f7fba6f5503f5121c832b982de945 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 30 Jul 2024 20:07:37 +0000 Subject: [PATCH 0438/1722] Reset formatter after buffer is restarted Formatter can use buffer to initialize some of its members. If buffer is restarted after those members are initialized, then the buffer restart might invalidate pointers/references/iterators hold into buffer. --- src/Storages/MessageQueueSink.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 36899011e33..d4dabd60ef8 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -60,12 +60,12 @@ void MessageQueueSink::consume(Chunk & chunk) row_format->writeRow(columns, row); } row_format->finalize(); - row_format->resetFormatter(); producer->produce(buffer->str(), i, columns, row - 1); /// Reallocate buffer if it's capacity is large then DBMS_DEFAULT_BUFFER_SIZE, /// because most likely in this case we serialized abnormally large row /// and won't need this large allocated buffer anymore. buffer->restart(DBMS_DEFAULT_BUFFER_SIZE); + row_format->resetFormatter(); } } else @@ -73,8 +73,8 @@ void MessageQueueSink::consume(Chunk & chunk) format->write(getHeader().cloneWithColumns(chunk.detachColumns())); format->finalize(); producer->produce(buffer->str(), chunk.getNumRows(), columns, chunk.getNumRows() - 1); - format->resetFormatter(); buffer->restart(); + format->resetFormatter(); } } From b337474e05a39b9d30aa8202be40a4ab71718884 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Jul 2024 20:44:19 +0000 Subject: [PATCH 0439/1722] Fix flaky test --- .../02932_refreshable_materialized_views_2.reference | 2 +- .../0_stateless/02932_refreshable_materialized_views_2.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference index eb4a0498260..cdaad32de0a 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference @@ -5,7 +5,7 @@ <23: simple refresh> 1 <24: rename during refresh> 1 <25: rename during refresh> f Running -<27: cancelled> f Scheduled +<27: cancelled> f Scheduled Cancelled <28: drop during refresh> 0 0 CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x <29: randomize> 1 1 diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh index cf0e61fb6a5..2a803114842 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh @@ -78,13 +78,13 @@ $CLICKHOUSE_CLIENT -nq " # Cancel. $CLICKHOUSE_CLIENT -nq " system cancel view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Cancelled' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Scheduled' ] do sleep 0.5 done # Check that another refresh doesn't immediately start after the cancelled one. $CLICKHOUSE_CLIENT -nq " - select '<27: cancelled>', view, status from refreshes where view = 'f'; + select '<27: cancelled>', view, status, last_refresh_result from refreshes where view = 'f'; system refresh view f;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Running' ] do From 458509909da0e6702dbb6023775112e4fa1c9a83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Jul 2024 23:21:14 +0200 Subject: [PATCH 0440/1722] Update test --- tests/queries/1_stateful/00157_cache_dictionary.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00157_cache_dictionary.sql b/tests/queries/1_stateful/00157_cache_dictionary.sql index a7c6c099de6..bb5a21d0779 100644 --- a/tests/queries/1_stateful/00157_cache_dictionary.sql +++ b/tests/queries/1_stateful/00157_cache_dictionary.sql @@ -9,7 +9,7 @@ ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS storage_policy = 'default'; -INSERT INTO test.hits_1m SELECT * FROM test.hits LIMIT 1000000 SETTINGS min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; +INSERT INTO test.hits_1m SELECT * FROM test.hits LIMIT 1000000 SETTINGS min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_block_size = 8192; CREATE DATABASE IF NOT EXISTS db_dict; DROP DICTIONARY IF EXISTS db_dict.cache_hits; From ea06447ca3b5330e09c0426574583b11c2ecaaa0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 30 Jul 2024 22:03:29 +0000 Subject: [PATCH 0441/1722] Fix: skip cast only if constant doesn't have source expression --- src/Analyzer/FunctionNode.cpp | 7 +++++++ ...ize_skip_unused_shards_rewrite_in.reference | 18 +++++++++--------- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index debed0983fd..8e4e0725a2d 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -239,6 +239,13 @@ ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const if (function_name == "_CAST" && !argument_nodes.empty() && argument_nodes[0]->getNodeType() == QueryTreeNodeType::CONSTANT) new_options.add_cast_for_constants = false; + /// Avoid cast for `IN tuple(...)` expression. + /// Tuples could be quite big, and adding a type may significantly increase query size. + /// It should be safe because set type for `column IN tuple` is deduced from `column` type. + if (isNameOfInFunction(function_name) && argument_nodes.size() > 1 && argument_nodes[1]->getNodeType() == QueryTreeNodeType::CONSTANT + && !static_cast(argument_nodes[1].get())->hasSourceExpression()) + new_options.add_cast_for_constants = false; + const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) { diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 0a7a9d64208..8d064020c1f 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -24,10 +24,10 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_no%') and type = 'QueryFinish' order by query; - _CAST((0, 2), \'Tuple(UInt8, UInt8)\') - _CAST((0, 2), \'Tuple(UInt8, UInt8)\') + (0, 2) + (0, 2) -- --- w/ optimize_skip_unused_shards_rewrite_in=1 +-- w/ otimize_skip_unused_shards_rewrite_in=1 -- set optimize_skip_unused_shards_rewrite_in=1; @@ -45,8 +45,8 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_02%') and type = 'QueryFinish' order by query; - _CAST(tuple(0), \'Tuple(UInt8)\') - _CAST(tuple(2), \'Tuple(UInt8)\') + tuple(0) + tuple(2) select 'optimize_skip_unused_shards_rewrite_in(2,)'; optimize_skip_unused_shards_rewrite_in(2,) with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2,); @@ -59,7 +59,7 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_2%') and type = 'QueryFinish' order by query; - _CAST(tuple(2), \'Tuple(UInt8)\') + tuple(2) select 'optimize_skip_unused_shards_rewrite_in(0,)'; optimize_skip_unused_shards_rewrite_in(0,) with (select currentDatabase()) as id_00 select *, ignore(id_00) from dist_01756 where dummy in (0,); @@ -73,7 +73,7 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_00%') and type = 'QueryFinish' order by query; - _CAST(tuple(0), \'Tuple(UInt8)\') + tuple(0) -- signed column select 'signed column'; signed column @@ -88,8 +88,8 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%key_signed%') and type = 'QueryFinish' order by query; - _CAST(tuple(-1), \'Tuple(Int8)\') - _CAST(tuple(-2), \'Tuple(Int8)\') + tuple(-1) + tuple(-2) -- not tuple select * from dist_01756 where dummy in (0); 0 From 956f8762fef7473804f7d82d63f076e09736f42c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 31 Jul 2024 05:11:34 +0000 Subject: [PATCH 0442/1722] fix after merge --- src/Client/ClientApplicationBase.cpp | 37 ++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 9f133616d2e..1b2ae16a479 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -158,6 +158,8 @@ void ClientApplicationBase::init(int argc, char ** argv) ("config-file,C", po::value(), "config-file path") + ("proto_caps", po::value(), "enable/disable chunked protocol: chunked_optional, notchunked, notchunked_optional, send_chunked, send_chunked_optional, send_notchunked, send_notchunked_optional, recv_chunked, recv_chunked_optional, recv_notchunked, recv_notchunked_optional") + ("query,q", po::value>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") ("multiquery,n", "Obsolete, does nothing") @@ -339,6 +341,41 @@ void ClientApplicationBase::init(int argc, char ** argv) if (options.count("server_logs_file")) server_logs_file = options["server_logs_file"].as(); + if (options.count("proto_caps")) + { + std::string proto_caps_str = options["proto_caps"].as(); + + std::vector proto_caps; + splitInto<','>(proto_caps, proto_caps_str); + + for (auto cap_str : proto_caps) + { + std::string direction; + + if (cap_str.starts_with("send_")) + { + direction = "send"; + cap_str = cap_str.substr(std::string_view("send_").size()); + } + else if (cap_str.starts_with("recv_")) + { + direction = "recv"; + cap_str = cap_str.substr(std::string_view("recv_").size()); + } + + if (cap_str != "chunked" && cap_str != "notchunked" && cap_str != "chunked_optional" && cap_str != "notchunked_optional") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "proto_caps option is incorrect ({})", proto_caps_str); + + if (direction.empty()) + { + config().setString("proto_caps.send", std::string(cap_str)); + config().setString("proto_caps.recv", std::string(cap_str)); + } + else + config().setString("proto_caps." + direction, std::string(cap_str)); + } + } + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); query_kind = parseQueryKind(options["query_kind"].as()); profile_events.print = options.count("print-profile-events"); From 3adbc4cd334a05560ca2397ac5e1a14d8c580167 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 Jul 2024 15:10:29 +0800 Subject: [PATCH 0443/1722] Fix code style --- src/Storages/Statistics/StatisticsMinMax.h | 2 ++ src/Storages/Statistics/StatisticsTDigest.cpp | 5 ++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 913dcccb798..3e08e6b5768 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -1,3 +1,5 @@ +#pragma once + #include diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 3544f5cdea3..25cee4ac8e8 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -9,7 +9,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_STATISTICS; -extern const int LOGICAL_ERROR; } StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) @@ -44,7 +43,7 @@ Float64 StatisticsTDigest::estimateLess(const Field & val) const Field val_converted = convertFieldToType(val, *data_type); if (val_converted.isNull()) return 0; - + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); return t_digest.getCountLessThan(val_as_float); } @@ -54,7 +53,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const Field val_converted = convertFieldToType(val, *data_type); if (val_converted.isNull()) return 0; - + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); return t_digest.getCountEqual(val_as_float); } From 9fb610ae10da22f521a1ab2e4442c78766d5be37 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 31 Jul 2024 11:52:36 +0200 Subject: [PATCH 0444/1722] fix tests --- .../queries/0_stateless/03203_hive_style_partitioning.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 430a3582f65..a4a2e48e046 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -118,4 +118,3 @@ Eva Schmidt Elizabeth Schmidt Samuel Schmidt Elizabeth Schmidt Eva Schmidt Elizabeth Samuel Schmidt Elizabeth -Elizabeth Gordon Elizabeth Gordon From 0cd37533a1e9873632cff7dc6debbbf802a29742 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 31 Jul 2024 10:10:44 +0000 Subject: [PATCH 0445/1722] fix after merge --- src/Client/ClientBase.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f8c2fb0d6bc..0c26b77bcec 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1039,10 +1039,10 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa connection_parameters.timeouts, query, query_parameters, - global_context->getCurrentQueryId(), + client_context->getCurrentQueryId(), query_processing_stage, - &global_context->getSettingsRef(), - &global_context->getClientInfo(), + &client_context->getSettingsRef(), + &client_context->getClientInfo(), true, [&](const Progress & progress) { onProgress(progress); }); From 7e2e24c75d705922e16fc79a257d03d5d5c1017e Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 10:29:12 +0000 Subject: [PATCH 0446/1722] Unifiing tests changes --- .../test/integration/runner/requirements.txt | 1 + .../ObjectStorage/Azure/Configuration.cpp | 10 +- .../DataLakes/registerDataLakeStorages.cpp | 2 +- .../helpers/{s3_tools.py => cloud_tools.py} | 78 +++-- .../test_iceberg_azure_storage/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 + .../configs/users.d/users.xml | 9 + .../test_iceberg_azure_storage/test.py | 291 ++++++++++++++++++ tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- .../integration/test_storage_iceberg/test.py | 214 ++++++++++--- tests/integration/test_storage_s3/test.py | 2 +- 12 files changed, 552 insertions(+), 68 deletions(-) rename tests/integration/helpers/{s3_tools.py => cloud_tools.py} (53%) create mode 100644 tests/integration/test_iceberg_azure_storage/__init__.py create mode 100644 tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml create mode 100644 tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml create mode 100644 tests/integration/test_iceberg_azure_storage/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8a77d8abf77..db10398e73d 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -64,6 +64,7 @@ minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 oauthlib==3.2.0 +pandas==2.2.1 packaging==24.0 paramiko==3.4.0 pika==1.2.0 diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index f0a0a562b92..9730391d429 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -148,10 +148,12 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, { if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage AzureBlobStorage requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " - "[account_name, account_key, format, compression, structure)])"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage AzureBlobStorage requires 3 to {} arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure)])", + (with_structure ? 8 : 7)); } for (auto & engine_arg : engine_args) diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index e75ab3201b8..f0bd51de375 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -54,7 +54,7 @@ void registerStorageIceberg(StorageFactory & factory) [&](const StorageFactory::Arguments & args) { auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), true); return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/cloud_tools.py similarity index 53% rename from tests/integration/helpers/s3_tools.py rename to tests/integration/helpers/cloud_tools.py index 0c3538c3c39..534791b8bc5 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -2,30 +2,66 @@ from minio import Minio import glob import os import json +import shutil -def upload_directory(minio_client, bucket_name, local_path, s3_path): - result_files = [] - for local_file in glob.glob(local_path + "/**"): - if os.path.isfile(local_file): +from enum import Enum + + +class CloudUploader: + def upload_directory(self, local_path, remote_blob_path): + result_files = [] + # print(f"Arguments: {local_path}, {s3_path}") + # for local_file in glob.glob(local_path + "/**"): + # print("Local file: {}", local_file) + for local_file in glob.glob(local_path + "/**"): result_local_path = os.path.join(local_path, local_file) - result_s3_path = os.path.join(s3_path, local_file) - print(f"Putting file {result_local_path} to {result_s3_path}") - minio_client.fput_object( - bucket_name=bucket_name, - object_name=result_s3_path, - file_path=result_local_path, - ) - result_files.append(result_s3_path) - else: - files = upload_directory( - minio_client, - bucket_name, - os.path.join(local_path, local_file), - os.path.join(s3_path, local_file), - ) - result_files.extend(files) - return result_files + result_remote_blob_path = os.path.join(remote_blob_path, local_file) + if os.path.isfile(local_file): + self.upload_file(result_local_path, result_remote_blob_path) + result_files.append(result_remote_blob_path) + else: + files = self.upload_directory( + result_local_path, + result_remote_blob_path, + ) + result_files.extend(files) + return result_files + + +class S3Uploader(CloudUploader): + def __init__(self, minio_client, bucket_name): + self.minio_client = minio_client + self.bucket_name = bucket_name + + def upload_file(self, local_path, remote_blob_path): + self.minio_client.fput_object( + bucket_name=self.bucket_name, + object_name=remote_blob_path, + file_path=local_path, + ) + + +class LocalUploader(CloudUploader): + def __init__(self): + pass + + def upload_file(self, local_path, remote_blob_path): + if local_path != remote_blob_path: + shutil.copyfile(local_path, remote_blob_path) + + +class AzureUploader(CloudUploader): + def __init__(self, blob_service_client, container_name): + self.blob_service_client = blob_service_client + self.container_name = container_name + + def upload_file(self, local_path, remote_blob_path): + blob_client = self.blob_service_client.get_blob_client( + container=self.container_name, blob=remote_blob_path + ) + with open(local_path, "rb") as data: + blob_client.upload_blob(data, overwrite=True) def get_file_contents(minio_client, bucket, s3_path): diff --git a/tests/integration/test_iceberg_azure_storage/__init__.py b/tests/integration/test_iceberg_azure_storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml new file mode 100644 index 00000000000..d4c54e2d13d --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/ + minio + minio123 + + + diff --git a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml new file mode 100644 index 00000000000..4b6ba057ecb --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py new file mode 100644 index 00000000000..f96f8acfaaf --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/test.py @@ -0,0 +1,291 @@ +import helpers.client +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.test_tools import TSV + +import pyspark +import logging +import os +import json +import pytest +import time +import glob +import uuid +import os + +import tempfile + +import io +import avro.schema +import avro.io +import avro.datafile +import pandas as pd + +from pyspark.sql.types import ( + StructType, + StructField, + StringType, + IntegerType, + DateType, + TimestampType, + BooleanType, + ArrayType, +) +from pyspark.sql.functions import current_timestamp +from datetime import datetime +from pyspark.sql.functions import monotonically_increasing_id, row_number +from pyspark.sql.window import Window +from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 +from minio.deleteobjects import DeleteObject + +from tests.integration.helpers.cloud_tools import ( + prepare_s3_bucket, + upload_directory, + get_file_contents, + list_s3_objects, +) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def get_spark(): + builder = ( + pyspark.sql.SparkSession.builder.appName("spark_test") + .config( + "spark.sql.catalog.spark_catalog", + "org.apache.iceberg.spark.SparkSessionCatalog", + ) + .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") + .config( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", + ) + .master("local") + ) + return builder.master("local").getOrCreate() + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__, with_spark=True) + cluster.add_instance( + "node1", + main_configs=["configs/config.d/named_collections.xml"], + user_configs=["configs/users.d/users.xml"], + with_minio=True, + stay_alive=True, + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + cluster.spark_session = get_spark() + + yield cluster + + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def write_iceberg_from_file( + spark, path, table_name, mode="overwrite", format_version="1", partition_by=None +): + if mode == "overwrite": + if partition_by is None: + spark.read.load(f"file://{path}").writeTo(table_name).tableProperty( + "format-version", format_version + ).using("iceberg").create() + else: + spark.read.load(f"file://{path}").writeTo(table_name).partitionedBy( + partition_by + ).tableProperty("format-version", format_version).using("iceberg").create() + else: + spark.read.load(f"file://{path}").writeTo(table_name).append() + + +def write_iceberg_from_df( + spark, df, table_name, mode="overwrite", format_version="1", partition_by=None +): + if mode == "overwrite": + if partition_by is None: + df.writeTo(table_name).tableProperty( + "format-version", format_version + ).using("iceberg").create() + else: + df.writeTo(table_name).tableProperty( + "format-version", format_version + ).partitionedBy(partition_by).using("iceberg").create() + else: + df.writeTo(table_name).append() + + +def generate_data(spark, start, end): + a = spark.range(start, end, 1).toDF("a") + b = spark.range(start + 1, end + 1, 1).toDF("b") + b = b.withColumn("b", b["b"].cast(StringType())) + + a = a.withColumn( + "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) + ) + b = b.withColumn( + "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) + ) + + df = a.join(b, on=["row_index"]).drop("row_index") + return df + + +def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + + +def create_initial_data_file( + cluster, node, query, table_name, compression_method="none" +): + node.query( + f""" + INSERT INTO TABLE FUNCTION + file('{table_name}.parquet') + SETTINGS + output_format_parquet_compression_method='{compression_method}', + s3_truncate_on_insert=1 {query} + FORMAT Parquet""" + ) + user_files_path = os.path.join( + SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files" + ) + result_path = f"{user_files_path}/{table_name}.parquet" + return result_path + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_single_iceberg_file(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_single_iceberg_file_" + format_version + + inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" + parquet_data_path = create_initial_data_file( + started_cluster, instance, inserted_data, TABLE_NAME + ) + + write_iceberg_from_file( + spark, parquet_data_path, TABLE_NAME, format_version=format_version + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + for bucket in minio_client.list_buckets(): + for object in minio_client.list_objects(bucket.name, recursive=True): + print("Object: ", object.object_name) + extension = object.object_name.split(".")[-1] + print("File extension: ", extension) + try: + response = minio_client.get_object( + object.bucket_name, object.object_name + ) + + if extension == "avro": + avro_bytes = response.read() + + # Use BytesIO to create a file-like object from the byte string + avro_file = io.BytesIO(avro_bytes) + + # Read the Avro data + reader = avro.datafile.DataFileReader( + avro_file, avro.io.DatumReader() + ) + records = [record for record in reader] + + # Close the reader + reader.close() + + # Now you can work with the records + for record in records: + # print(json.dumps(record, indent=4, sort_keys=True)) + print(str(record)) + # my_json = ( + # str(record) + # .replace("'", '"') + # .replace("None", "null") + # .replace('b"', '"') + # ) + # print(my_json) + # data = json.loads(my_json) + # s = json.dumps(data, indent=4, sort_keys=True) + # print(s) + elif extension == "json": + my_bytes_value = response.read() + my_json = my_bytes_value.decode("utf8").replace("'", '"') + data = json.loads(my_json) + s = json.dumps(data, indent=4, sort_keys=True) + print(s) + elif extension == "parquet": + # print("To be continued...") + # # Your byte string containing the Parquet data + # parquet_bytes = response.read() + + # # Use BytesIO to create a file-like object from the byte string + # parquet_file = io.BytesIO(parquet_bytes) + + # # Read the Parquet data into a PyArrow Table + # table = pq.read_table(parquet_file) + + # # Convert the PyArrow Table to a Pandas DataFrame + # df = table.to_pandas() + + # # Now you can work with s DataFrame + # print(df) + parquet_bytes = ( + response.read() + ) # Replace with your actual byte string + + # Create a temporary file and write the byte string to it + with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + tmp_file.write(parquet_bytes) + tmp_file_path = tmp_file.name + + # Read the Parquet file using PySpark + df = spark.read.parquet(tmp_file_path) + + # Show the DataFrame + print(df.toPandas()) + else: + print(response.read()) + + finally: + print("----------------") + response.close() + response.release_conn() + + create_iceberg_table(instance, TABLE_NAME) + + assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( + inserted_data + ) + + assert 0 == 1 diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d3dd7cfe52a..aaff1414d8d 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -28,7 +28,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from helpers.s3_tools import ( +from helpers.cloud_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 0c3fbfb3cda..750b77b29f3 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.cloud_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 7762d17b96f..7f83846bd89 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -12,6 +12,14 @@ import glob import uuid import os +import tempfile + +import io +import avro.schema +import avro.io +import avro.datafile +import pandas as pd + from pyspark.sql.types import ( StructType, StructField, @@ -29,11 +37,13 @@ from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject -from helpers.s3_tools import ( +from helpers.cloud_tools import ( prepare_s3_bucket, - upload_directory, get_file_contents, list_s3_objects, + S3Uploader, + AzureUploader, + LocalUploader, ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -67,6 +77,7 @@ def started_cluster(): main_configs=["configs/config.d/named_collections.xml"], user_configs=["configs/users.d/users.xml"], with_minio=True, + with_azurite=True, stay_alive=True, ) @@ -77,6 +88,15 @@ def started_cluster(): logging.info("S3 bucket created") cluster.spark_session = get_spark() + cluster.default_s3_uploader = S3Uploader( + cluster.minio_client, cluster.minio_bucket + ) + + container_name = "my_container" + + cluster.default_azurite_uploader = AzureUploader( + cluster.blob_service_client, container_name + ) yield cluster @@ -142,13 +162,25 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) +def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwargs): + if storage_type == "local": + pass + elif storage_type == "s3": + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + elif storage_type == "azure": + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergAzure(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + else: + raise Exception("Unknown iceberg storage type: {}", storage_type) def create_initial_data_file( @@ -170,32 +202,134 @@ def create_initial_data_file( return result_path +def default_upload_directory(started_cluster, storage_type, local_path, remote_path): + if storage_type == "local": + return LocalUploader().upload_directory(local_path, remote_path) + elif storage_type == "s3": + return started_cluster.default_s3_uploader.upload_directory( + local_path, remote_path + ) + elif storage_type == "azure": + return started_cluster.default_azure_uploader.upload_directory( + local_path, remote_path + ) + else: + raise Exception("Unknown iceberg storage type: {}", storage_type) + + @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_single_iceberg_file(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3"]) +def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket TABLE_NAME = "test_single_iceberg_file_" + format_version - inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" - parquet_data_path = create_initial_data_file( - started_cluster, instance, inserted_data, TABLE_NAME + write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) + + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - write_iceberg_from_file( - spark, parquet_data_path, TABLE_NAME, format_version=format_version - ) + # for bucket in minio_client.list_buckets(): + # for object in minio_client.list_objects(bucket.name, recursive=True): + # print("Object: ", object.object_name) + # extension = object.object_name.split(".")[-1] + # print("File extension: ", extension) + # try: + # response = minio_client.get_object( + # object.bucket_name, object.object_name + # ) - files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" - ) + # if extension == "avro": + # avro_bytes = response.read() + + # # Use BytesIO to create a file-like object from the byte string + # avro_file = io.BytesIO(avro_bytes) + + # # Read the Avro data + # reader = avro.datafile.DataFileReader( + # avro_file, avro.io.DatumReader() + # ) + # records = [record for record in reader] + + # # Close the reader + # reader.close() + + # # Now you can work with the records + # for record in records: + # # print(json.dumps(record, indent=4, sort_keys=True)) + # print(str(record)) + # # my_json = ( + # # str(record) + # # .replace("'", '"') + # # .replace("None", "null") + # # .replace('b"', '"') + # # ) + # # print(my_json) + # # data = json.loads(my_json) + # # s = json.dumps(data, indent=4, sort_keys=True) + # # print(s) + # elif extension == "json": + # my_bytes_value = response.read() + # my_json = my_bytes_value.decode("utf8").replace("'", '"') + # data = json.loads(my_json) + # s = json.dumps(data, indent=4, sort_keys=True) + # print(s) + # elif extension == "parquet": + # # print("To be continued...") + # # # Your byte string containing the Parquet data + # # parquet_bytes = response.read() + + # # # Use BytesIO to create a file-like object from the byte string + # # parquet_file = io.BytesIO(parquet_bytes) + + # # # Read the Parquet data into a PyArrow Table + # # table = pq.read_table(parquet_file) + + # # # Convert the PyArrow Table to a Pandas DataFrame + # # df = table.to_pandas() + + # # # Now you can work with s DataFrame + # # print(df) + # parquet_bytes = ( + # response.read() + # ) # Replace with your actual byte string + + # # Create a temporary file and write the byte string to it + # with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + # tmp_file.write(parquet_bytes) + # tmp_file_path = tmp_file.name + + # # Read the Parquet file using PySpark + # df = spark.read.parquet(tmp_file_path) + + # # Show the DataFrame + # print(df.toPandas()) + # else: + # print(response.read()) + + # finally: + # print("----------------") + # response.close() + # response.release_conn() create_iceberg_table(instance, TABLE_NAME) + + # print("Debug Print") + + # print(instance.query(f"SELECT * FROM {TABLE_NAME}")) + + # print(instance.query("SELECT number FROM numbers(100)")) + assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( - inserted_data + "SELECT number, toString(number + 1) FROM numbers(100)" ) + # assert 0 == 1 + @pytest.mark.parametrize("format_version", ["1", "2"]) def test_partition_by(started_cluster, format_version): @@ -215,7 +349,7 @@ def test_partition_by(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert len(files) == 14 # 10 partitiions + 4 metadata files @@ -240,7 +374,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" ) # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text', @@ -260,7 +394,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): format_version=format_version, ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" ) assert len(files) == 9 @@ -302,7 +436,9 @@ def test_types(started_cluster, format_version): spark, df, TABLE_NAME, mode="overwrite", format_version=format_version ) - upload_directory(minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "") + upload_directory( + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + ) create_iceberg_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1 @@ -345,7 +481,7 @@ def test_delete_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -354,7 +490,7 @@ def test_delete_files(started_cluster, format_version): spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 @@ -369,14 +505,14 @@ def test_delete_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 @@ -399,7 +535,7 @@ def test_evolved_schema(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -410,7 +546,7 @@ def test_evolved_schema(started_cluster, format_version): spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -437,7 +573,7 @@ def test_row_based_deletes(started_cluster): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -446,7 +582,7 @@ def test_row_based_deletes(started_cluster): spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -472,7 +608,7 @@ def test_schema_inference(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME, format) @@ -527,7 +663,7 @@ def test_metadata_file_selection(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -559,7 +695,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -586,7 +722,7 @@ def test_restart_broken(started_cluster): write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME, bucket=bucket) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 @@ -614,7 +750,7 @@ def test_restart_broken(started_cluster): minio_client.make_bucket(bucket) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 40cbf4b44a6..fea828dc3a5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from helpers.s3_tools import prepare_s3_bucket +from helpers.cloud_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From bcd53dcd20ae4ffce2eeceeea6637725b19a0803 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 31 Jul 2024 14:31:33 +0200 Subject: [PATCH 0447/1722] Fix docs --- docs/en/sql-reference/data-types/newjson.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 524dc7810e6..9e43216df6c 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -10,7 +10,7 @@ keywords: [json, data type] Stores JavaScript Object Notation (JSON) documents in a single column. :::note -This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json.md) instead. +This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-formats/json/overview) instead. If you want to use JSON type, set `allow_experimental_json_type = 1`. ::: From a45027f22c73b321f1f9cc110f0782fac10d5748 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 31 Jul 2024 15:22:48 +0000 Subject: [PATCH 0448/1722] Fix flaky `test_delayed_replica_failover` --- tests/integration/test_delayed_replica_failover/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index a480ee3f278..c545877797a 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -101,7 +101,7 @@ SELECT sum(x) FROM distributed WITH TOTALS SETTINGS # allow pings to zookeeper to timeout (must be greater than ZK session timeout). for _ in range(30): try: - node_2_2.query("SELECT * FROM system.zookeeper where path = '/'") + node_2_2.query("SELECT * FROM system.zookeeper where path = '/' SETTINGS insert_keeper_max_retries = 0") time.sleep(0.5) except: break @@ -120,7 +120,7 @@ SELECT sum(x) FROM distributed SETTINGS == "3" ) - # Regression for skip_unavailable_shards in conjunction with skip_unavailable_shards + # Prefer fallback_to_stale_replicas over skip_unavailable_shards assert ( instance_with_dist_table.query( """ From 33ed33d2af0821785165f0ffe80e3c8086c637e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0449/1722] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From af324f69e955310f54e451d1120e8526ac3150fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:56:20 +0200 Subject: [PATCH 0450/1722] Update test --- tests/queries/1_stateful/00158_cache_dictionary_has.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00158_cache_dictionary_has.sql b/tests/queries/1_stateful/00158_cache_dictionary_has.sql index 32c109417de..631a7751550 100644 --- a/tests/queries/1_stateful/00158_cache_dictionary_has.sql +++ b/tests/queries/1_stateful/00158_cache_dictionary_has.sql @@ -10,6 +10,8 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hits' PA LIFETIME(MIN 300 MAX 600) LAYOUT(CACHE(SIZE_IN_CELLS 100 QUERY_WAIT_TIMEOUT_MILLISECONDS 600000)); +SET timeout_before_checking_execution_speed = 300; + SELECT sum(flag) FROM (SELECT dictHas('db_dict.cache_hits', toUInt64(WatchID)) as flag FROM test.hits PREWHERE WatchID % 1400 == 0 LIMIT 100); SELECT count() from test.hits PREWHERE WatchID % 1400 == 0; @@ -20,4 +22,4 @@ SELECT sum(flag) FROM (SELECT dictHas('db_dict.cache_hits', toUInt64(WatchID)) a SELECT count() from test.hits PREWHERE WatchID % 5 == 0; DROP DICTIONARY IF EXISTS db_dict.cache_hits; -DROP DATABASE IF EXISTS db_dict; +DROP DATABASE IF EXISTS db_dict; From f9c9d85e4109511bed14f5e7edb0f31b0bf0beae Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 16:50:56 +0000 Subject: [PATCH 0451/1722] Automatic style fix --- tests/integration/test_delayed_replica_failover/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index c545877797a..1116d225b8c 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -101,7 +101,9 @@ SELECT sum(x) FROM distributed WITH TOTALS SETTINGS # allow pings to zookeeper to timeout (must be greater than ZK session timeout). for _ in range(30): try: - node_2_2.query("SELECT * FROM system.zookeeper where path = '/' SETTINGS insert_keeper_max_retries = 0") + node_2_2.query( + "SELECT * FROM system.zookeeper where path = '/' SETTINGS insert_keeper_max_retries = 0" + ) time.sleep(0.5) except: break From e7fc206069796f662bb31aab671cbf75b95984ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 31 Jul 2024 17:20:01 +0000 Subject: [PATCH 0452/1722] Add test --- .../format_schemas/string_key_value.capnp | 6 + .../format_schemas/string_key_value.format | 1 + .../format_schemas/string_key_value.proto | 6 + .../clickhouse_path/format_schemas/test.capnp | 2 +- tests/integration/test_storage_kafka/test.py | 129 ++++++++++++++++++ 5 files changed, 143 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp new file mode 100644 index 00000000000..4f3eabe22f0 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.capnp @@ -0,0 +1,6 @@ +@0x99f75f775fe63dae; + +struct StringKeyValuePair { + key@0 : Text; + value@1 : Text; +} diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format new file mode 100644 index 00000000000..83dff6ce401 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.format @@ -0,0 +1 @@ +(key = ${key:CSV}, value = ${value:CSV}) diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto new file mode 100644 index 00000000000..71905c63bdf --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/string_key_value.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message StringKeyValuePair { + string key = 1; + string value = 2; +} diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp index 44f1961205b..247e7b9ceca 100644 --- a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp @@ -7,4 +7,4 @@ struct TestRecordStruct val1 @2 : Text; val2 @3 : Float32; val3 @4 : UInt8; -} \ No newline at end of file +} diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8393e88db88..596933c1566 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5097,6 +5097,135 @@ def test_multiple_read_in_materialized_views(kafka_cluster, max_retries=15): ) +def test_kafka_produce_http_interface_row_based_format(kafka_cluster): + # reproduction of #https://github.com/ClickHouse/ClickHouse/issues/61060 with validating the written messages + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + + topic_prefix = "http_row_" + + # It is important to have: + # - long enough messages + # - enough messages + # I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages + # For the number of messages it seems like at least 3 messages is necessary + expected_key = "01234567890123456789" + expected_value = "aaaaabbbbbccccc" + + insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')" + insert_query_template = "INSERT INTO {table_name} " + insert_query_end + + extra_settings = { + "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", + "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", + "Template": ", format_template_row='string_key_value.format'" + } + + # Only the formats that can be used both and input and output format are tested + # Reasons to exclude following formats: + # - JSONStrings: not actually an input format + # - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken, + # because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed. + # - ProtobufList: I didn't want to deal with the envelope and stuff + # - Npy: supports only single column + # - LineAsString: supports only single column + # - RawBLOB: supports only single column + formats_to_test = [ + "TabSeparated", + "TabSeparatedRaw", + "TabSeparatedWithNames", + "TabSeparatedWithNamesAndTypes", + "TabSeparatedRawWithNames", + "TabSeparatedRawWithNamesAndTypes", + "Template", + "CSV", + "CSVWithNames", + "CSVWithNamesAndTypes", + "CustomSeparated", + "CustomSeparatedWithNames", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONColumns", + "JSONColumnsWithMetadata", + "JSONCompact", + "JSONCompactColumns", + "JSONEachRow", + "JSONStringsEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNames", + "JSONCompactEachRowWithNamesAndTypes", + "JSONCompactStringsEachRow", + "JSONCompactStringsEachRowWithNames", + "JSONCompactStringsEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "BSONEachRow", + "TSKV", + "Protobuf", + "Avro", + "Parquet", + "Arrow", + "ArrowStream", + "ORC", + "RowBinary", + "RowBinaryWithNames", + "RowBinaryWithNamesAndTypes", + "Native", + "CapnProto", + "MsgPack", + ] + for format in formats_to_test: + logging.debug(f"Creating tables and writing messages to {format}") + topic = topic_prefix + format + kafka_create_topic(admin_client, topic) + + extra_setting = extra_settings.get(format, "") + + # kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug + instance.query( + f""" + DROP TABLE IF EXISTS test.view_{topic}; + DROP TABLE IF EXISTS test.consumer_{topic}; + CREATE TABLE test.kafka_writer_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}', + kafka_max_rows_per_message = 2 {extra_setting}; + + CREATE TABLE test.kafka_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}' {extra_setting}; + + CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS + SELECT key, value FROM test.kafka_{topic}; + """ + ) + + instance.http_query(insert_query_template.format(table_name="test.kafka_writer_"+topic), method="POST") + + expected = f"""\ +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +""" + # give some times for the readers to read the messages + for format in formats_to_test: + logging.debug(f"Checking result for {format}") + topic = topic_prefix + format + + result = instance.query_with_retry(f"SELECT * FROM test.view_{topic}", check_callback=lambda res: res.count("\n") == 3) + + assert TSV(result) == TSV(expected) + + kafka_delete_topic(admin_client, topic) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 461251b519120dd6b0cb64471bfba70160137e50 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 19:21:47 +0200 Subject: [PATCH 0453/1722] Update a test --- .../02450_kill_distributed_query_deadlock.sh | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh index 445f907bcc5..96692ba325a 100755 --- a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh +++ b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh @@ -5,20 +5,24 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# Test that running distributed query and cancel it ASAP, -# this can trigger a hung/deadlock in ProcessorList. -for i in {1..50}; do +# Test that runs a distributed query and cancels it ASAP, +# this has a chance to trigger a hung/deadlock in ProcessorList. +for i in {1..50} +do query_id="$CLICKHOUSE_TEST_UNIQUE_NAME-$i" - $CLICKHOUSE_CLIENT --format Null --query_id "$query_id" --max_rows_to_read 0 -q "select * from remote('127.{1|2|3|4|5|6}', numbers(1e12))" 2>/dev/null & - while :; do + $CLICKHOUSE_CLIENT --format Null --query_id "$query_id" --max_rows_to_read 0 --max_bytes_to_read 0 --max_result_rows 0 --max_result_bytes 0 -q "select * from remote('127.{1|2|3|4|5|6}', numbers(1e12))" 2>/dev/null & + while true + do killed_queries="$($CLICKHOUSE_CLIENT -q "kill query where query_id = '$query_id' sync" | wc -l)" - if [[ "$killed_queries" -ge 1 ]]; then + if [[ "$killed_queries" -ge 1 ]] + then break fi done wait -n query_return_status=$? - if [[ $query_return_status -eq 0 ]]; then + if [[ $query_return_status -eq 0 ]] + then echo "Query $query_id should be cancelled, however it returns successfully" fi done From bada9ea9be4888ab4ef1b3f7fcdf015e11d994d1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 17:30:24 +0000 Subject: [PATCH 0454/1722] Automatic style fix --- tests/integration/test_storage_kafka/test.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 596933c1566..dd0bf1bf28f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5103,7 +5103,6 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) - topic_prefix = "http_row_" # It is important to have: @@ -5120,7 +5119,7 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): extra_settings = { "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", - "Template": ", format_template_row='string_key_value.format'" + "Template": ", format_template_row='string_key_value.format'", } # Only the formats that can be used both and input and output format are tested @@ -5208,7 +5207,10 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): """ ) - instance.http_query(insert_query_template.format(table_name="test.kafka_writer_"+topic), method="POST") + instance.http_query( + insert_query_template.format(table_name="test.kafka_writer_" + topic), + method="POST", + ) expected = f"""\ {expected_key}\t{expected_value} @@ -5220,12 +5222,16 @@ def test_kafka_produce_http_interface_row_based_format(kafka_cluster): logging.debug(f"Checking result for {format}") topic = topic_prefix + format - result = instance.query_with_retry(f"SELECT * FROM test.view_{topic}", check_callback=lambda res: res.count("\n") == 3) + result = instance.query_with_retry( + f"SELECT * FROM test.view_{topic}", + check_callback=lambda res: res.count("\n") == 3, + ) assert TSV(result) == TSV(expected) kafka_delete_topic(admin_client, topic) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From beb5d02cdc1f5fae58a8ee43fadb1c581868b894 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 31 Jul 2024 17:58:20 +0000 Subject: [PATCH 0455/1722] Move THROW back to InterpreterDelete. --- src/Interpreters/InterpreterDeleteQuery.cpp | 13 +++++++++++++ src/Interpreters/MutationsInterpreter.cpp | 10 ---------- src/Storages/MergeTree/MutateTask.cpp | 12 ++++++++---- .../03161_lightweight_delete_projection.sql | 4 ++-- 4 files changed, 23 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 0e988e7d031..3000292f047 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -85,6 +86,18 @@ BlockIO InterpreterDeleteQuery::execute() "Lightweight delete mutate is disabled. " "Set `enable_lightweight_delete` setting to enable it"); + if (metadata_snapshot->hasProjections()) + { + if (const auto * merge_tree_data = dynamic_cast(table.get())) + if (merge_tree_data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "DELETE query is not allowed for table {} because as it has projections and setting " + "lightweight_mutation_projection_mode is set to THROW. " + "User should change lightweight_mutation_projection_mode OR " + "drop all the projections manually before running the query", + table_id.getFullTableName()); + } + /// Build "ALTER ... UPDATE _row_exists = 0 WHERE predicate" query String alter_query = "ALTER TABLE " + table->getStorageID().getFullTableName() diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index db4ea9c0754..480c6736bc5 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -491,16 +491,6 @@ static void validateUpdateColumns( { if (!source.supportsLightweightDelete()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); - - if (const MergeTreeData * merge_tree_data = source.getMergeTreeData(); merge_tree_data != nullptr) - { - if (merge_tree_data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW - && merge_tree_data->hasProjection()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "DELETE query is not supported for table {} as it has projections. " - "User should drop all the projections manually before running the query", - source.getStorage()->getStorageID().getFullTableName()); - } } else if (virtual_columns.tryGet(column_name)) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6245d80508b..8b5829eb058 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2320,10 +2320,14 @@ bool MutateTask::prepare() ctx->context, ctx->materialized_indices); - bool lightweight_delete_projection_drop = lightweight_delete_mode - && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP; + auto lightweight_mutation_projection_mode = ctx->data->getSettings()->lightweight_mutation_projection_mode; + bool lightweight_delete_drops_projections = + lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP + || lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW; + + bool should_create_projections = !(lightweight_delete_mode && lightweight_delete_drops_projections); /// Under lightweight delete mode, if option is drop, projections_to_recalc should be empty. - if (!lightweight_delete_projection_drop) + if (should_create_projections) { ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( ctx->source_part, @@ -2342,7 +2346,7 @@ bool MutateTask::prepare() ctx->projections_to_recalc, ctx->stats_to_recalc, ctx->metadata_snapshot, - lightweight_delete_projection_drop); + !should_create_projections); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index f33653fc652..02b880d620a 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -21,7 +21,7 @@ SELECT 'testing throw default mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; @@ -85,7 +85,7 @@ INSERT INTO users VALUES (1231, 'John', 33); SELECT 'testing throw default mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231; -- { serverError NOT_IMPLEMENTED } +DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; From 557f9dbe3fb02e3bce62adbeb1fd5056f2d36b6c Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 31 Jul 2024 18:51:27 +0000 Subject: [PATCH 0456/1722] fix test --- .../0_stateless/02319_lightweight_delete_on_merge_tree.sql | 2 +- tests/queries/0_stateless/02792_drop_projection_lwd.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index f82f79dbe44..6491253cd5f 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -102,7 +102,7 @@ ALTER TABLE t_proj ADD PROJECTION p_1 (SELECT avg(a), avg(b), count()) SETTINGS INSERT INTO t_proj SELECT number + 1, number + 1 FROM numbers(1000); -DELETE FROM t_proj WHERE a < 100; -- { serverError NOT_IMPLEMENTED } +DELETE FROM t_proj WHERE a < 100; -- { serverError SUPPORT_IS_DISABLED } SELECT avg(a), avg(b), count() FROM t_proj; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql index dcde7dcc600..dad7f7cd028 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.sql +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -7,7 +7,7 @@ CREATE TABLE t_projections_lwd (a UInt32, b UInt32, PROJECTION p (SELECT * ORDER INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); -- LWD does not work, as expected -DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError NOT_IMPLEMENTED } +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError SUPPORT_IS_DISABLED } KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; -- drop projection From aaa7750bf98840e22f3521977b6d7cf168460b91 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 1 Aug 2024 11:24:50 +0000 Subject: [PATCH 0457/1722] Handle dynamic columns in typed paths --- src/Columns/ColumnObject.cpp | 10 ++++++++++ .../03214_json_typed_dynamic_path.reference | 4 ++++ .../03214_json_typed_dynamic_path.sql | 17 +++++++++++++++++ 3 files changed, 31 insertions(+) create mode 100644 tests/queries/0_stateless/03214_json_typed_dynamic_path.reference create mode 100644 tests/queries/0_stateless/03214_json_typed_dynamic_path.sql diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 8649e2314b9..e7bb7639dd2 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1211,6 +1211,16 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou } column->takeDynamicStructureFromSourceColumns(dynamic_path_source_columns); } + + /// Typed paths also can contain types with dynamic structure. + for (auto & [path, column] : typed_paths) + { + Columns typed_path_source_columns; + typed_path_source_columns.reserve(source_columns.size()); + for (const auto & source_column : source_columns) + typed_path_source_columns.push_back(assert_cast(*source_column).typed_paths.at(path)); + column->takeDynamicStructureFromSourceColumns(typed_path_source_columns); + } } size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const ColumnString & shared_data_paths, size_t start, size_t end) diff --git a/tests/queries/0_stateless/03214_json_typed_dynamic_path.reference b/tests/queries/0_stateless/03214_json_typed_dynamic_path.reference new file mode 100644 index 00000000000..1b3e6b7a8db --- /dev/null +++ b/tests/queries/0_stateless/03214_json_typed_dynamic_path.reference @@ -0,0 +1,4 @@ +{"a":"42"} +{"a":["1","2","3"]} +{"a":"42"} +{"a":["1","2","3"]} diff --git a/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql b/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql new file mode 100644 index 00000000000..1f6a025825a --- /dev/null +++ b/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql @@ -0,0 +1,17 @@ +-- Tags: no-fasttest + +set allow_experimental_json_type = 1; +drop table if exists test; +create table test (json JSON(a Dynamic)) engine=MergeTree order by tuple() settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; +insert into test select '{"a" : 42}'; +insert into test select '{"a" : [1, 2, 3]}'; +optimize table test; +select * from test order by toString(json); +drop table test; + +create table test (json JSON(a Dynamic)) engine=MergeTree order by tuple() settings min_rows_for_wide_part=10000000, min_bytes_for_wide_part=10000000; +insert into test select '{"a" : 42}'; +insert into test select '{"a" : [1, 2, 3]}'; +optimize table test; +select * from test order by toString(json); +drop table test; From d2e0668d5129a4e60f462de5e5b683099f49bf4b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 1 Aug 2024 13:51:35 +0200 Subject: [PATCH 0458/1722] fix settingsChangesHistory after merge with master --- src/Core/SettingsChangesHistory.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a01c5faaf10..28a732c6177 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,9 +81,6 @@ static std::initializer_list Date: Thu, 1 Aug 2024 13:10:13 +0000 Subject: [PATCH 0459/1722] Fix text --- ...56_optimize_skip_unused_shards_rewrite_in.reference | 10 +++++----- .../01756_optimize_skip_unused_shards_rewrite_in.sql | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 8d064020c1f..237bca6305a 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -27,7 +27,7 @@ order by query; (0, 2) (0, 2) -- --- w/ otimize_skip_unused_shards_rewrite_in=1 +-- w/ optimize_skip_unused_shards_rewrite_in=1 -- set optimize_skip_unused_shards_rewrite_in=1; @@ -49,7 +49,7 @@ order by query; tuple(2) select 'optimize_skip_unused_shards_rewrite_in(2,)'; optimize_skip_unused_shards_rewrite_in(2,) -with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2,); +with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2); system flush logs; select splitByString('IN', query)[-1] from system.query_log where event_date >= yesterday() and @@ -59,10 +59,10 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_2%') and type = 'QueryFinish' order by query; - tuple(2) + (2) select 'optimize_skip_unused_shards_rewrite_in(0,)'; optimize_skip_unused_shards_rewrite_in(0,) -with (select currentDatabase()) as id_00 select *, ignore(id_00) from dist_01756 where dummy in (0,); +with (select currentDatabase()) as id_00 select *, ignore(id_00) from dist_01756 where dummy in (0); 0 0 system flush logs; select splitByString('IN', query)[-1] from system.query_log where @@ -73,7 +73,7 @@ select splitByString('IN', query)[-1] from system.query_log where query like concat('%', currentDatabase(), '%AS%id_00%') and type = 'QueryFinish' order by query; - tuple(0) + (0) -- signed column select 'signed column'; signed column diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql index 9a1a00cc0a1..0b7a44665dc 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql @@ -63,7 +63,7 @@ select splitByString('IN', query)[-1] from system.query_log where order by query; select 'optimize_skip_unused_shards_rewrite_in(2,)'; -with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2,); +with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2); system flush logs; select splitByString('IN', query)[-1] from system.query_log where event_date >= yesterday() and @@ -75,7 +75,7 @@ select splitByString('IN', query)[-1] from system.query_log where order by query; select 'optimize_skip_unused_shards_rewrite_in(0,)'; -with (select currentDatabase()) as id_00 select *, ignore(id_00) from dist_01756 where dummy in (0,); +with (select currentDatabase()) as id_00 select *, ignore(id_00) from dist_01756 where dummy in (0); system flush logs; select splitByString('IN', query)[-1] from system.query_log where event_date >= yesterday() and From 3150833aa517bd0293a6f3b7f9114cfb89af06ad Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 1 Aug 2024 20:16:10 +0200 Subject: [PATCH 0460/1722] fix merge with master --- docs/en/operations/settings/settings.md | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b880c42a45b..c621f2db5ae 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5615,18 +5615,6 @@ Disable all insert and mutations (alter table update / alter table delete / alte Default value: `false`. -## restore_replace_external_engines_to_null - -For testing purposes. Replaces all external engines to Null to not initiate external connections. - -Default value: `False` - -## restore_replace_external_table_functions_to_null - -For testing purposes. Replaces all external table functions to Null to not initiate external connections. - -Default value: `False` - ## use_hive_partitioning When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. From af30e72e187713f4e435ff4f1f2382ea6671ba21 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Aug 2024 23:58:57 +0200 Subject: [PATCH 0461/1722] Better limits --- tests/config/users.d/limits.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/users.d/limits.yaml b/tests/config/users.d/limits.yaml index 46cff73142c..53cbbfa744a 100644 --- a/tests/config/users.d/limits.yaml +++ b/tests/config/users.d/limits.yaml @@ -26,6 +26,7 @@ profiles: max_execution_time_leaf: 600 max_execution_speed: 100G max_execution_speed_bytes: 10T + timeout_before_checking_execution_speed: 300 max_estimated_execution_time: 600 max_columns_to_read: 20K max_temporary_columns: 20K From a80c7c080cdbb3bec4662501686133298b7a4a2d Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 1 Aug 2024 18:23:11 -0600 Subject: [PATCH 0462/1722] Added support for reading multilinestring wkts --- docs/en/sql-reference/data-types/geo.md | 42 ++++++++++++ .../en/sql-reference/functions/geo/polygon.md | 42 ++++++++++-- src/DataTypes/DataTypeCustomGeo.cpp | 7 ++ src/DataTypes/DataTypeCustomGeo.h | 6 ++ src/Functions/geometryConverters.h | 64 +++++++++++++++++++ src/Functions/polygonsIntersection.cpp | 2 + src/Functions/polygonsSymDifference.cpp | 2 + src/Functions/polygonsUnion.cpp | 2 + src/Functions/polygonsWithin.cpp | 2 + src/Functions/readWkt.cpp | 30 +++++++++ .../03215_multilinestring_geometry.reference | 17 +++++ .../03215_multilinestring_geometry.sql | 12 ++++ .../aspell-ignore/en/aspell-dict.txt | 4 +- 13 files changed, 227 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03215_multilinestring_geometry.reference create mode 100644 tests/queries/0_stateless/03215_multilinestring_geometry.sql diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md index 7ffc7447d96..8ce53bb2ef2 100644 --- a/docs/en/sql-reference/data-types/geo.md +++ b/docs/en/sql-reference/data-types/geo.md @@ -52,6 +52,48 @@ Result: └───────────────────────────────┴───────────────┘ ``` +## LineString + +`LineString` is a line stored as an array of points: [Array](array.md)([Point](#point)). + +**Example** + +Query: + +```sql +CREATE TABLE geo_linestring (l LineString) ENGINE = Memory(); +INSERT INTO geo_linestring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]); +SELECT l, toTypeName(l) FROM geo_linestring; +``` +Result: + +``` text +┌─r─────────────────────────────┬─toTypeName(r)─┠+│ [(0,0),(10,0),(10,10),(0,10)] │ LineString │ +└───────────────────────────────┴───────────────┘ +``` + +## MultiLineString + +`MultiLineString` is multiple lines stored as an array of `LineString`: [Array](array.md)([LineString](#linestring)). + +**Example** + +Query: + +```sql +CREATE TABLE geo_multilinestring (l MultiLineString) ENGINE = Memory(); +INSERT INTO geo_multilinestring VALUES([[(0, 0), (10, 0), (10, 10), (0, 10)], [(1, 1), (2, 2), (3, 3)]]); +SELECT l, toTypeName(l) FROM geo_multilinestring; +``` +Result: + +``` text +┌─l───────────────────────────────────────────────────┬─toTypeName(l)───┠+│ [[(0,0),(10,0),(10,10),(0,10)],[(1,1),(2,2),(3,3)]] │ MultiLineString │ +└─────────────────────────────────────────────────────┴─────────────────┘ +``` + ## Polygon `Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring)). First element of outer array is the outer shape of polygon and all the following elements are holes. diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 25a7a1fac8e..c054e05d39c 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -6,11 +6,13 @@ title: "Functions for Working with Polygons" ## WKT -Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](../../data-types/geo.md). Supported WKT objects are: +Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](../../data-types/geo.md). Supported WKT objects are: - POINT - POLYGON - MULTIPOLYGON +- LINESTRING +- MULTILINESTRING **Syntax** @@ -26,12 +28,16 @@ WKT(geo_data) - [Ring](../../data-types/geo.md#ring) - [Polygon](../../data-types/geo.md#polygon) - [MultiPolygon](../../data-types/geo.md#multipolygon) +- [LineString](../../data-types/geo.md#linestring) +- [MultiLineString](../../data-types/geo.md#multilinestring) **Returned value** - WKT geometric object `POINT` is returned for a Point. - WKT geometric object `POLYGON` is returned for a Polygon -- WKT geometric object `MULTIPOLYGON` is returned for a MultiPolygon. +- WKT geometric object `MULTIPOLYGON` is returned for a MultiPolygon. +- WKT geometric object `LINESTRING` is returned for a LineString. +- WKT geometric object `MULTILINESTRING` is returned for a MultiLineString. **Examples** @@ -84,7 +90,7 @@ SELECT ### Input parameters -String starting with `MULTIPOLYGON` +String starting with `MULTIPOLYGON` ### Returned value @@ -170,6 +176,34 @@ SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); [(1,1),(2,2),(3,3),(1,1)] ``` +## readWKTMultiLineString + +Parses a Well-Known Text (WKT) representation of a MultiLineString geometry and returns it in the internal ClickHouse format. + +### Syntax + +```sql +readWKTMultiLineString(wkt_string) +``` + +### Arguments + +- `wkt_string`: The input WKT string representing a MultiLineString geometry. + +### Returned value + +The function returns a ClickHouse internal representation of the multilinestring geometry. + +### Example + +```sql +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')); +``` + +```response +[[(1,1),(2,2),(3,3)],[(4,4),(5,5),(6,6)]] +``` + ## readWKTRing Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format. @@ -219,7 +253,7 @@ UInt8, 0 for false, 1 for true ## polygonsDistanceSpherical -Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise. +Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise. ### Example diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index 0736d837d46..d72787647c3 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -24,6 +24,13 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique())); }); + // Custom type for mulitple lines stored as Array(LineString) + factory.registerSimpleDataTypeCustom("MultiLineString", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(LineString)"), + std::make_unique(std::make_unique())); + }); + // Custom type for simple polygon without holes stored as Array(Point) factory.registerSimpleDataTypeCustom("Ring", [] { diff --git a/src/DataTypes/DataTypeCustomGeo.h b/src/DataTypes/DataTypeCustomGeo.h index 0a1c83e4638..6a632f0d05c 100644 --- a/src/DataTypes/DataTypeCustomGeo.h +++ b/src/DataTypes/DataTypeCustomGeo.h @@ -17,6 +17,12 @@ public: DataTypeLineStringName() : DataTypeCustomFixedName("LineString") {} }; +class DataTypeMultiLineStringName : public DataTypeCustomFixedName +{ +public: + DataTypeMultiLineStringName() : DataTypeCustomFixedName("MultiLineString") {} +}; + class DataTypeRingName : public DataTypeCustomFixedName { public: diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 03831d37e0c..bf975017a6d 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -31,6 +31,9 @@ namespace ErrorCodes template using LineString = boost::geometry::model::linestring; +template +using MultiLineString = boost::geometry::model::multi_linestring>; + template using Ring = boost::geometry::model::ring; @@ -42,12 +45,14 @@ using MultiPolygon = boost::geometry::model::multi_polygon>; using CartesianPoint = boost::geometry::model::d2::point_xy; using CartesianLineString = LineString; +using CartesianMultiLineString = MultiLineString; using CartesianRing = Ring; using CartesianPolygon = Polygon; using CartesianMultiPolygon = MultiPolygon; using SphericalPoint = boost::geometry::model::point>; using SphericalLineString = LineString; +using SphericalMultiLineString = MultiLineString; using SphericalRing = Ring; using SphericalPolygon = Polygon; using SphericalMultiPolygon = MultiPolygon; @@ -113,6 +118,28 @@ struct ColumnToLineStringsConverter } }; +/** + * Class which converts Column with type Array(Array(Tuple(Float64, Float64))) to a vector of boost multi_linestring type. +*/ +template +struct ColumnToMultiLineStringsConverter +{ + static std::vector> convert(ColumnPtr col) + { + const IColumn::Offsets & offsets = typeid_cast(*col).getOffsets(); + size_t prev_offset = 0; + std::vector> answer(offsets.size()); + auto all_linestrings = ColumnToLineStringsConverter::convert(typeid_cast(*col).getDataPtr()); + for (size_t iter = 0; iter < offsets.size() && iter < all_linestrings.size(); ++iter) + { + for (size_t linestring_iter = prev_offset; linestring_iter < offsets[iter]; ++linestring_iter) + answer[iter].emplace_back(std::move(all_linestrings[linestring_iter])); + prev_offset = offsets[iter]; + } + return answer; + } +}; + /** * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type. */ @@ -268,6 +295,38 @@ private: ColumnUInt64::MutablePtr offsets; }; +/// Serialize Point, MultiLineString as MultiLineString +template +class MultiLineStringSerializer +{ +public: + MultiLineStringSerializer() + : offsets(ColumnUInt64::create()) + {} + + explicit MultiLineStringSerializer(size_t n) + : offsets(ColumnUInt64::create(n)) + {} + + void add(const MultiLineString & multilinestring) + { + size += multilinestring.size(); + offsets->insertValue(size); + for (const auto & linestring : multilinestring) + linestring_serializer.add(linestring); + } + + ColumnPtr finalize() + { + return ColumnArray::create(linestring_serializer.finalize(), std::move(offsets)); + } + +private: + size_t size = 0; + LineStringSerializer linestring_serializer; + ColumnUInt64::MutablePtr offsets; +}; + /// Almost the same as LineStringSerializer /// Serialize Point, Ring as Ring template @@ -411,6 +470,11 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "LineString") return f(ConverterType>()); + /// We should take the name into consideration to avoid ambiguity. + /// Because for example both MultiLineString and Polygon are resolved to Array(Tuple(Point)). + else if (factory.get("MultiLineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "MultiLineString") + return f(ConverterType>()); + /// For backward compatibility if we call this function not on a custom type, we will consider Array(Tuple(Point)) as type Ring. else if (factory.get("Ring")->equals(*type)) return f(ConverterType>()); diff --git a/src/Functions/polygonsIntersection.cpp b/src/Functions/polygonsIntersection.cpp index 329242e762e..43ab03f8c1f 100644 --- a/src/Functions/polygonsIntersection.cpp +++ b/src/Functions/polygonsIntersection.cpp @@ -75,6 +75,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index 3c219d0facb..6faec95bb7b 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -73,6 +73,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 969eb2f78fb..5378ff636f8 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -73,6 +73,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index c63ad5ef868..dacd1c0e18f 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -77,6 +77,8 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be MultiLineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index eb262777b0d..2010b5167e7 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -87,6 +87,11 @@ struct ReadWKTLineStringNameHolder static constexpr const char * name = "readWKTLineString"; }; +struct ReadWKTMultiLineStringNameHolder +{ + static constexpr const char * name = "readWKTMultiLineString"; +}; + struct ReadWKTRingNameHolder { static constexpr const char * name = "readWKTRing"; @@ -131,6 +136,31 @@ Parses a Well-Known Text (WKT) representation of a LineString geometry and retur }, .categories{"Unique identifiers"} }); + factory.registerFunction, ReadWKTMultiLineStringNameHolder>>(FunctionDocumentation + { + .description=R"( +Parses a Well-Known Text (WKT) representation of a MultiLineString geometry and returns it in the internal ClickHouse format. +)", + .syntax = "readWKTMultiLineString(wkt_string)", + .arguments{ + {"wkt_string", "The input WKT string representing a MultiLineString geometry."} + }, + .returned_value = "The function returns a ClickHouse internal representation of the multilinestring geometry.", + .examples{ + {"first call", "SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))');", R"( +┌─readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')─┠+│ [[(1,1),(2,2),(3,3)],[(4,4),(5,5),(6,6)]] │ +└──────────────────────────────────────────────────────────────────────────────┘ + + )"}, + {"second call", "SELECT toTypeName(readWKTLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'));", R"( +┌─toTypeName(readWKTLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'))─┠+│ MultiLineString │ +└─────────────────────────────────────────────────────────────────────────┘ + )"}, + }, + .categories{"Unique identifiers"} + }); factory.registerFunction, ReadWKTRingNameHolder>>(); factory.registerFunction, ReadWKTPolygonNameHolder>>(); factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.reference b/tests/queries/0_stateless/03215_multilinestring_geometry.reference new file mode 100644 index 00000000000..f4c5774018e --- /dev/null +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.reference @@ -0,0 +1,17 @@ +-- { echoOn } +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +[[(1,1),(2,2),(3,3),(1,1)]] +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); +MultiLineString +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); +MULTILINESTRING((1 1,2 2,3 3,1 1)) +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +[[(1,1),(2,2),(3,3),(1,1)],[(1,0),(2,0),(3,0)]] +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); +MultiLineString +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); +MULTILINESTRING((1 1,2 2,3 3,1 1),(1 0,2 0,3 0)) +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x +SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); +POLYGON((1 1,2 2,3 3,1 1)) String [[(1,1),(2,2),(3,3),(1,1)]] Polygon diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.sql b/tests/queries/0_stateless/03215_multilinestring_geometry.sql new file mode 100644 index 00000000000..71344920c52 --- /dev/null +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.sql @@ -0,0 +1,12 @@ +-- { echoOn } +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))')); + +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +SELECT toTypeName(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); +SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))')); + +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x +SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b21ae0764c6..3d7e77f213d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2942 +personal_ws-1.1 en 2942 AArch ACLs ALTERs @@ -561,6 +561,7 @@ MindsDB Mongodb Monotonicity MsgPack +MultiLineString MultiPolygon Multiline Multiqueries @@ -2361,6 +2362,7 @@ rankCorr rapidjson rawblob readWKTLineString +readWKTMultiLineString readWKTMultiPolygon readWKTPoint readWKTPolygon From c0d298781e72aaa1f34fc7fd610f5dbcaa9acf2d Mon Sep 17 00:00:00 2001 From: morning-color Date: Fri, 2 Aug 2024 10:36:15 +0800 Subject: [PATCH 0463/1722] Trigger test. --- .../queries/0_stateless/03174_exact_rows_before_aggregation.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql index 0afc0be4370..f9fd4ef5a7b 100644 --- a/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql +++ b/tests/queries/0_stateless/03174_exact_rows_before_aggregation.sql @@ -34,6 +34,7 @@ create table test (i int) engine MergeTree order by i; insert into test select arrayJoin(range(10000)); set optimize_aggregation_in_order=1; + select * from test where i < 10 group by i order by i FORMAT JSONCompact; select max(i) from test where i < 20 limit 1 FORMAT JSONCompact; From c50ef37a03438003c21076c5700d9c1f52c1c435 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 00:01:41 -0400 Subject: [PATCH 0464/1722] Fix inconsistent formatting for `GRANT CURRENT GRANTS` --- src/Parsers/Access/ASTGrantQuery.cpp | 21 +++------------------ 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/src/Parsers/Access/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp index f60fa7e4a23..eac88c75513 100644 --- a/src/Parsers/Access/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -97,24 +97,9 @@ namespace void formatCurrentGrantsElements(const AccessRightsElements & elements, const IAST::FormatSettings & settings) { - for (size_t i = 0; i != elements.size(); ++i) - { - const auto & element = elements[i]; - - bool next_element_on_same_db_and_table = false; - if (i != elements.size() - 1) - { - const auto & next_element = elements[i + 1]; - if (element.sameDatabaseAndTableAndParameter(next_element)) - next_element_on_same_db_and_table = true; - } - - if (!next_element_on_same_db_and_table) - { - settings.ostr << " "; - formatONClause(element, settings); - } - } + settings.ostr << "("; + formatElementsWithoutOptions(elements, settings); + settings.ostr << ")"; } } From 5ebb2c54b049ff740e30e8c2da6d5fe591e3e493 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Fri, 2 Aug 2024 14:07:44 +0800 Subject: [PATCH 0465/1722] Re-trigger ci --- tests/integration/test_replicated_database/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 72d45e45841..3a640897530 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1552,8 +1552,8 @@ def test_all_groups_cluster(started_cluster): def test_alter_modify_order_by(started_cluster): - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") main_node.query( "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');" @@ -1566,9 +1566,9 @@ def test_alter_modify_order_by(started_cluster): "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');" ) - query = "show create table alter_modify_order_by.t1;" + query = "show create table alter_modify_order_by.t1" expected = main_node.query(query) assert_eq_with_retry(snapshotting_node, query, expected) - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") From 80eda5c647f25a2cf24cf076f3bedc6a14712942 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Aug 2024 09:04:04 +0000 Subject: [PATCH 0466/1722] Fix binary deserialization of JSON --- .../Serializations/SerializationObject.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 305ebfa2e16..9091702326a 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -663,6 +663,19 @@ void SerializationObject::deserializeBinary(IColumn & col, ReadBuffer & istr, co restoreColumnObject(column_object, prev_size); throw; } + + /// Insert default to all remaining typed and dynamic paths. + for (auto & [_, column] : typed_paths) + { + if (column->size() == prev_size) + column->insertDefault(); + } + + for (auto & [_, column] : column_object.getDynamicPathsPtrs()) + { + if (column->size() == prev_size) + column->insertDefault(); + } } SerializationPtr SerializationObject::TypedPathSubcolumnCreator::create(const DB::SerializationPtr & prev) const From 3bff7ddcf8891d091bc5be2b827172029fb8b76f Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 2 Aug 2024 13:19:00 +0000 Subject: [PATCH 0467/1722] fix data race: delay reset of data hooks until the next sendRequest() --- src/Common/HTTPConnectionPool.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index acddcc8530d..68c13838c04 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -394,8 +394,12 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIORead(), log, request.getMethod(), request.getURI())); + else + Session::setReceiveDataHooks(); if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIOWrite(), log, request.getMethod(), request.getURI())); + else + Session::setSendDataHooks(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); @@ -453,8 +457,6 @@ private: } } response_stream = nullptr; - Session::setSendDataHooks(); - Session::setReceiveDataHooks(); group->atConnectionDestroy(); From 51b39a6c745d61cb2e6feb39659ddb3cac57ad03 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Aug 2024 18:24:45 +0100 Subject: [PATCH 0468/1722] some more --- tests/integration/test_executable_dictionary/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index a1de429a235..2a6af75e751 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -235,6 +235,7 @@ def test_executable_implicit_input_signalled_python(started_cluster): ) +@pytest.mark.repeat(50) def test_executable_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( From 77a2eb61ef965a6460bbdb74447aa3871cb1d0c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Aug 2024 17:43:33 +0000 Subject: [PATCH 0469/1722] Update test. --- ...61_lightweight_delete_projection.reference | 70 ++++++++++++++++++ .../03161_lightweight_delete_projection.sql | 74 ++++++++++--------- 2 files changed, 111 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index 960fa1dcc33..eef0c5a41b5 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -1,20 +1,90 @@ compact part testing throw default mode +-- { echoOn } + +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; +DELETE FROM users_compact WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +SELECT 'testing drop mode'; testing drop mode +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +DELETE FROM users_compact WHERE uid = 1231; +SELECT * FROM users_compact ORDER BY uid; +SYSTEM FLUSH LOGS; +-- all_1_1_0_2 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); all_1_1_0_2 +-- expecting no projection +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); +SELECT 'testing rebuild mode'; testing rebuild mode +INSERT INTO users_compact VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +DELETE FROM users_compact WHERE uid = 6666; +SELECT * FROM users_compact ORDER BY uid; 8888 Alice 50 +SYSTEM FLUSH LOGS; +-- all_1_1_0_4, all_3_3_0_4 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); all_1_1_0_4 all_3_3_0_4 +-- expecting projection p1, p2 +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); p1 all_3_3_0_4 p2 all_3_3_0_4 wide part testing throw default mode +-- { echoOn } + +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; +DELETE FROM users_wide WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +SELECT 'testing drop mode'; testing drop mode +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +DELETE FROM users_wide WHERE uid = 1231; +SELECT * FROM users_wide ORDER BY uid; +SYSTEM FLUSH LOGS; +-- all_1_1_0_2 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); all_1_1_0_2 +-- expecting no projection +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +SELECT 'testing rebuild mode'; testing rebuild mode +INSERT INTO users_wide VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +DELETE FROM users_wide WHERE uid = 6666; +SELECT * FROM users_wide ORDER BY uid; 8888 Alice 50 +SYSTEM FLUSH LOGS; +-- all_1_1_0_4, all_3_3_0_4 +SELECT + name +FROM system.parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); all_1_1_0_4 all_3_3_0_4 +-- expecting projection p1, p2 +SELECT + name, parent_name +FROM system.projection_parts +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); p1 all_3_3_0_4 p2 all_3_3_0_4 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 02b880d620a..28e5612a529 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,12 +1,12 @@ SET lightweight_deletes_sync = 2, alter_sync = 2; -DROP TABLE IF EXISTS users; +DROP TABLE IF EXISTS users_compact; SELECT 'compact part'; -CREATE TABLE users ( +CREATE TABLE users_compact ( uid Int16, name String, age Int16, @@ -15,20 +15,22 @@ CREATE TABLE users ( ) ENGINE = MergeTree order by uid SETTINGS min_bytes_for_wide_part = 10485760; -INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users_compact VALUES (1231, 'John', 33); SELECT 'testing throw default mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; +-- { echoOn } -DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; + +DELETE FROM users_compact WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; -DELETE FROM users WHERE uid = 1231; +DELETE FROM users_compact WHERE uid = 1231; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_compact ORDER BY uid; SYSTEM FLUSH LOGS; @@ -36,22 +38,22 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); -- expecting no projection SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); SELECT 'testing rebuild mode'; -INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +INSERT INTO users_compact VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +ALTER TABLE users_compact MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; -DELETE FROM users WHERE uid = 6666; +DELETE FROM users_compact WHERE uid = 6666; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_compact ORDER BY uid; SYSTEM FLUSH LOGS; @@ -59,19 +61,21 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); -- expecting projection p1, p2 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); -DROP TABLE users; +-- { echoOff } + +DROP TABLE users_compact; SELECT 'wide part'; -CREATE TABLE users ( +CREATE TABLE users_wide ( uid Int16, name String, age Int16, @@ -80,19 +84,22 @@ CREATE TABLE users ( ) ENGINE = MergeTree order by uid SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users_wide VALUES (1231, 'John', 33); SELECT 'testing throw default mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; -DELETE FROM users WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } +-- { echoOn } + +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'throw'; + +DELETE FROM users_wide WHERE uid = 1231; -- { serverError SUPPORT_IS_DISABLED } SELECT 'testing drop mode'; -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'drop'; -DELETE FROM users WHERE uid = 1231; +DELETE FROM users_wide WHERE uid = 1231; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_wide ORDER BY uid; SYSTEM FLUSH LOGS; @@ -100,22 +107,22 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); -- expecting no projection SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); SELECT 'testing rebuild mode'; -INSERT INTO users VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); +INSERT INTO users_wide VALUES (6666, 'Ksenia', 48), (8888, 'Alice', 50); -ALTER TABLE users MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; +ALTER TABLE users_wide MODIFY SETTING lightweight_mutation_projection_mode = 'rebuild'; -DELETE FROM users WHERE uid = 6666; +DELETE FROM users_wide WHERE uid = 6666; -SELECT * FROM users ORDER BY uid; +SELECT * FROM users_wide ORDER BY uid; SYSTEM FLUSH LOGS; @@ -123,13 +130,14 @@ SYSTEM FLUSH LOGS; SELECT name FROM system.parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); -- expecting projection p1, p2 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +-- { echoOff } -DROP TABLE users; \ No newline at end of file +DROP TABLE users_wide; From fb23cbdef6b38d1ba90aa1bb304487a3bb947e5c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0470/1722] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 15e9f8d9cbe4c2f0a1d9973650d93fa195a56276 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Aug 2024 22:52:01 +0200 Subject: [PATCH 0471/1722] Fix `02481_async_insert_race_long` --- tests/queries/0_stateless/02481_async_insert_race_long.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_race_long.sh b/tests/queries/0_stateless/02481_async_insert_race_long.sh index b0088017d32..def97409bc4 100755 --- a/tests/queries/0_stateless/02481_async_insert_race_long.sh +++ b/tests/queries/0_stateless/02481_async_insert_race_long.sh @@ -29,11 +29,8 @@ function insert3() { local TIMELIMIT=$((SECONDS+$1)) while [ $SECONDS -lt "$TIMELIMIT" ]; do - ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" & - sleep 0.05 + ${MY_CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --wait_for_async_insert 1 -q "INSERT INTO async_inserts_race VALUES (7, 'g') (8, 'h')" done - - wait } function select1() From 13b435d281ce39a4f8eee889da482dbf272dd1cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Jul 2024 17:38:20 +0200 Subject: [PATCH 0472/1722] Minor change --- src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f419f5811a1..734f354d9a5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -313,7 +313,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri std::lock_guard lock(mutex); if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table `{}`", table_name); } else { From 76847d9b4c27ec3011ef5866c3ea7b30fdcd2f08 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 2 Aug 2024 22:43:05 +0000 Subject: [PATCH 0473/1722] Fix(asan) : access destroyed shared context from handleCrash() --- programs/server/Server.cpp | 1 + src/Daemon/BaseDaemon.cpp | 2 -- src/Interpreters/Context.cpp | 27 +++++++++++++++++---------- src/Interpreters/Context.h | 8 ++++++-- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 20db4c2773c..5b9bb8b989d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -977,6 +977,7 @@ try /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. */ + global_context->resetSharedContext(); global_context.reset(); shared_context.reset(); LOG_DEBUG(log, "Destroyed global context."); diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index e7ae8ea5a1d..f74bd5e122c 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -23,9 +23,7 @@ #include #include -#include #include -#include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5413b568068..743e762c81a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -892,6 +892,12 @@ ContextData::ContextData(const ContextData &o) : { } +void ContextData::resetSharedContext() +{ + std::lock_guard lock(mutex_shared_context); + shared = nullptr; +} + Context::Context() = default; Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) {} @@ -913,14 +919,6 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) return res; } -void Context::initGlobal() -{ - assert(!global_context_instance); - global_context_instance = shared_from_this(); - DatabaseCatalog::init(shared_from_this()); - EventNotifier::init(); -} - SharedContextHolder Context::createShared() { return SharedContextHolder(std::make_unique()); @@ -2691,7 +2689,11 @@ void Context::makeSessionContext() void Context::makeGlobalContext() { - initGlobal(); + assert(!global_context_instance); + global_context_instance = shared_from_this(); + DatabaseCatalog::init(shared_from_this()); + EventNotifier::init(); + global_context = shared_from_this(); } @@ -4084,8 +4086,13 @@ void Context::initializeTraceCollector() } /// Call after unexpected crash happen. -void Context::handleCrash() const TSA_NO_THREAD_SAFETY_ANALYSIS +void Context::handleCrash() const { + std::lock_guard lock(mutex_shared_context); + if (!shared) + return; + + SharedLockGuard lock2(shared->mutex); if (shared->system_logs) shared->system_logs->handleCrash(); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5e35c3e4b3..9ab7e9169c4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,6 +490,8 @@ public: KitchenSink kitchen_sink; + void resetSharedContext(); + protected: using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -527,6 +529,10 @@ protected: mutable ThrottlerPtr local_write_query_throttler; /// A query-wide throttler for local IO writes mutable ThrottlerPtr backups_query_throttler; /// A query-wide throttler for BACKUPs + + mutable std::mutex mutex_shared_context; /// mutex to avoid accessing destroyed shared context pointer + /// some Context methods can be called after the shared context is destroyed + /// example, Context::handleCrash() method - called from signal handler }; /** A set of known objects that can be used in the query. @@ -1385,8 +1391,6 @@ private: ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard & lock); - void initGlobal(); - void setUserID(const UUID & user_id_); void setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user); From 51918dc080c9fa4b128a151cfbd0d28b294c56d3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Aug 2024 00:24:00 +0100 Subject: [PATCH 0474/1722] impl --- src/Common/ShellCommand.cpp | 9 ++++- src/Processors/Sources/ShellCommandSource.cpp | 39 +++++++++++++------ 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 98a21b43d76..79b0d667863 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -237,7 +237,14 @@ std::unique_ptr ShellCommand::executeImpl( res->write_fds.emplace(fd, fds.fds_rw[1]); } - LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid); + LOG_TRACE( + getLogger(), + "Started shell command '{}' with pid {} and file descriptors: read {}, write {}", + filename, + pid, + res->out.getFD(), + res->err.getFD()); + return res; } diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 1659287c227..923bdfad8f8 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -75,6 +75,15 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond while (true) { Stopwatch watch; + +#if defined(DEBUG_OR_SANITIZER_BUILD) + auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Polling descriptors: {}", + fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); +#endif + res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); if (res < 0) @@ -84,7 +93,16 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond const auto elapsed = watch.elapsedMilliseconds(); if (timeout_milliseconds <= elapsed) + { +#if defined(DEBUG_OR_SANITIZER_BUILD) + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Timeout exceeded: elapsed={}, timeout={}", + elapsed, + timeout_milliseconds); +#endif break; + } timeout_milliseconds -= elapsed; } else @@ -93,6 +111,15 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond } } +#if defined(DEBUG_OR_SANITIZER_BUILD) + auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; + LOG_TRACE( + getLogger("TimeoutReadBufferFromFileDescriptor"), + "Poll for descriptors: {} returned {}", + fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", "), + res); +#endif + return res; } @@ -139,15 +166,9 @@ public: while (!bytes_read) { - LOG_TRACE( - getLogger("TimeoutReadBufferFromFileDescriptor"), - "Starting polling on descriptors ({}) with timeout {} ms", - fmt::join(std::span(pfds, pfds + num_pfds) | std::views::transform([](const auto & pollfd) { return pollfd.fd; }), ", "), - timeout_milliseconds); pfds[0].revents = 0; pfds[1].revents = 0; size_t num_events = pollWithTimeout(pfds, num_pfds, timeout_milliseconds); - LOG_TRACE(getLogger("TimeoutReadBufferFromFileDescriptor"), "Poll returned with num_events={}", num_events); if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); @@ -208,12 +229,6 @@ public: return true; } - void reset() const - { - makeFdBlocking(stdout_fd); - makeFdBlocking(stderr_fd); - } - ~TimeoutReadBufferFromFileDescriptor() override { tryMakeFdBlocking(stdout_fd); From 2605bb36b66ccfb4621244a28475a242778b6cc4 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 3 Aug 2024 01:42:11 +0000 Subject: [PATCH 0475/1722] fix conflict --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8f73e10c44f..107a8e451c5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -92,7 +92,7 @@ static std::initializer_list Date: Sat, 3 Aug 2024 17:03:27 +0800 Subject: [PATCH 0476/1722] optimize array() with single arguments and map() with double arguments --- src/Functions/array/array.cpp | 28 +++++++++++++++------ src/Functions/map.cpp | 47 +++++++++++++++++++++++------------ 2 files changed, 51 insertions(+), 24 deletions(-) diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index 03b51808799..202aa83216b 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -46,8 +46,10 @@ public: size_t num_elements = arguments.size(); if (num_elements == 0) + { /// We should return constant empty array. return result_type->createColumnConstWithDefaultValue(input_rows_count); + } const DataTypePtr & elem_type = static_cast(*result_type).getNestedType(); @@ -59,7 +61,6 @@ public: Columns columns_holder(num_elements); ColumnRawPtrs column_ptrs(num_elements); - for (size_t i = 0; i < num_elements; ++i) { const auto & arg = arguments[i]; @@ -76,22 +77,33 @@ public: } /// Create and fill the result array. - auto out = ColumnArray::create(elem_type->createColumn()); IColumn & out_data = out->getData(); IColumn::Offsets & out_offsets = out->getOffsets(); out_data.reserve(input_rows_count * num_elements); - out_offsets.resize(input_rows_count); + out_offsets.resize_exact(input_rows_count); IColumn::Offset current_offset = 0; - for (size_t i = 0; i < input_rows_count; ++i) + if (num_elements == 1) { - for (size_t j = 0; j < num_elements; ++j) - out_data.insertFrom(*column_ptrs[j], i); + for (size_t i = 0; i < input_rows_count; ++i) + { + ++current_offset; + out_offsets[i] = current_offset; + } + out_data.insertManyFrom(*column_ptrs[0], 0, input_rows_count); + } + else + { + for (size_t i = 0; i < input_rows_count; ++i) + { + for (size_t j = 0; j < num_elements; ++j) + out_data.insertFrom(*column_ptrs[j], i); - current_offset += num_elements; - out_offsets[i] = current_offset; + current_offset += num_elements; + out_offsets[i] = current_offset; + } } return out; diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 66cd10a3f0b..b869a475af0 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -1,14 +1,15 @@ -#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include @@ -123,24 +124,38 @@ public: MutableColumnPtr keys_data = key_type->createColumn(); MutableColumnPtr values_data = value_type->createColumn(); - MutableColumnPtr offsets = DataTypeNumber().createColumn(); + MutableColumnPtr offsets = DataTypeUInt64().createColumn(); size_t total_elements = input_rows_count * num_elements / 2; keys_data->reserve(total_elements); values_data->reserve(total_elements); - offsets->reserve(input_rows_count); + auto & offsets_data = assert_cast(*offsets).getData(); + offsets_data.resize_exact(input_rows_count); IColumn::Offset current_offset = 0; - for (size_t i = 0; i < input_rows_count; ++i) + if (num_elements == 2) { - for (size_t j = 0; j < num_elements; j += 2) + for (size_t i = 0; i < input_rows_count; ++i) { - keys_data->insertFrom(*column_ptrs[j], i); - values_data->insertFrom(*column_ptrs[j + 1], i); + ++current_offset; + offsets_data[i] = current_offset; } + keys_data->insertManyFrom(*column_ptrs[0], 0, input_rows_count); + values_data->insertManyFrom(*column_ptrs[1], 0, input_rows_count); + } + else + { + for (size_t i = 0; i < input_rows_count; ++i) + { + for (size_t j = 0; j < num_elements; j += 2) + { + keys_data->insertFrom(*column_ptrs[j], i); + values_data->insertFrom(*column_ptrs[j + 1], i); + } - current_offset += num_elements / 2; - offsets->insert(current_offset); + current_offset += num_elements / 2; + offsets_data[i] = current_offset; + } } auto nested_column = ColumnArray::create( From e5d4eaa4580f5921a472174d805c852e8b057165 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 3 Aug 2024 21:08:21 +0800 Subject: [PATCH 0477/1722] fix bugs --- src/Functions/array/array.cpp | 2 +- src/Functions/map.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index 202aa83216b..a15fe86a232 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -92,7 +92,7 @@ public: ++current_offset; out_offsets[i] = current_offset; } - out_data.insertManyFrom(*column_ptrs[0], 0, input_rows_count); + out_data.insertRangeFrom(*column_ptrs[0], 0, input_rows_count); } else { diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index b869a475af0..ebfa9d4eab4 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -140,8 +140,8 @@ public: ++current_offset; offsets_data[i] = current_offset; } - keys_data->insertManyFrom(*column_ptrs[0], 0, input_rows_count); - values_data->insertManyFrom(*column_ptrs[1], 0, input_rows_count); + keys_data->insertRangeFrom(*column_ptrs[0], 0, input_rows_count); + values_data->insertRangeFrom(*column_ptrs[1], 0, input_rows_count); } else { From 113f7e0c8c9cc61ab72f5ccd8bb6e5fac58cbaea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Aug 2024 19:37:38 +0200 Subject: [PATCH 0478/1722] Maybe better --- tests/queries/1_stateful/00157_cache_dictionary.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00157_cache_dictionary.sql b/tests/queries/1_stateful/00157_cache_dictionary.sql index bb5a21d0779..f1bee538828 100644 --- a/tests/queries/1_stateful/00157_cache_dictionary.sql +++ b/tests/queries/1_stateful/00157_cache_dictionary.sql @@ -9,7 +9,8 @@ ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS storage_policy = 'default'; -INSERT INTO test.hits_1m SELECT * FROM test.hits LIMIT 1000000 SETTINGS min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_block_size = 8192; +INSERT INTO test.hits_1m SELECT * FROM test.hits LIMIT 1000000 + SETTINGS min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_block_size = 8192, max_insert_threads = 1, max_threads = 1; CREATE DATABASE IF NOT EXISTS db_dict; DROP DICTIONARY IF EXISTS db_dict.cache_hits; From a498259ec8c08d4f37f77c8a6870f519bc295801 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 4 Aug 2024 13:32:17 +0800 Subject: [PATCH 0479/1722] optimize array with mutiple arguments --- src/Functions/array/array.cpp | 212 +++++++++++++++++++++++++++++----- 1 file changed, 184 insertions(+), 28 deletions(-) diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index a15fe86a232..b25505e9ac1 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -1,10 +1,14 @@ -#include -#include +#include +#include +#include #include #include -#include -#include +#include +#include #include +#include +#include "Columns/ColumnFixedString.h" +#include "Columns/ColumnNullable.h" namespace DB @@ -43,7 +47,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - size_t num_elements = arguments.size(); + const size_t num_elements = arguments.size(); if (num_elements == 0) { @@ -81,42 +85,194 @@ public: IColumn & out_data = out->getData(); IColumn::Offsets & out_offsets = out->getOffsets(); - out_data.reserve(input_rows_count * num_elements); + /// Fill out_offsets out_offsets.resize_exact(input_rows_count); - IColumn::Offset current_offset = 0; + for (size_t i = 0; i < input_rows_count; ++i) + { + current_offset += num_elements; + out_offsets[i] = current_offset; + } + + /// Fill out_data + out_data.reserve(input_rows_count * num_elements); if (num_elements == 1) - { - for (size_t i = 0; i < input_rows_count; ++i) - { - ++current_offset; - out_offsets[i] = current_offset; - } out_data.insertRangeFrom(*column_ptrs[0], 0, input_rows_count); - } else - { - for (size_t i = 0; i < input_rows_count; ++i) - { - for (size_t j = 0; j < num_elements; ++j) - out_data.insertFrom(*column_ptrs[j], i); - - current_offset += num_elements; - out_offsets[i] = current_offset; - } - } - + executeAny(column_ptrs, out_data, input_rows_count); return out; } - private: + bool executeAny(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + return false // NOLINT + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) + || executeNumber(columns, out_data, input_rows_count) || executeString(columns, out_data, input_rows_count) + || executeNullable(columns, out_data, input_rows_count) || executeTuple(columns, out_data, input_rows_count) + || executeFixedString(columns, out_data, input_rows_count) || executeGeneric(columns, out_data, input_rows_count); + } + + bool executeGeneric(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + for (size_t i = 0; i < input_rows_count; ++i) + for (const auto * column : columns) + out_data.insertFrom(*column, i); + return true; + } + + template + bool executeNumber(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + using Container = ColumnVectorOrDecimal::Container; + std::vector containers(columns.size(), nullptr); + for (size_t i = 0; i < columns.size(); ++i) + { + const ColumnVectorOrDecimal * concrete_column = checkAndGetColumn>(columns[i]); + if (!concrete_column) + return false; + + containers[i] = &concrete_column->getData(); + } + + ColumnVectorOrDecimal & concrete_out_data = assert_cast &>(out_data); + Container & out_container = concrete_out_data.getData(); + out_container.resize_exact(columns.size() * input_rows_count); + + for (size_t row_i = 0; row_i < input_rows_count; ++row_i) + { + const size_t base = row_i * columns.size(); + for (size_t col_i = 0; col_i < columns.size(); ++col_i) + out_container[base + col_i] = (*containers[col_i])[row_i]; + } + return true; + } + + bool executeString(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + size_t total_bytes = 0; + std::vector concrete_columns(columns.size(), nullptr); + for (size_t i = 0; i < columns.size(); ++i) + { + const ColumnString * concrete_column = checkAndGetColumn(columns[i]); + if (!concrete_column) + return false; + + total_bytes += concrete_column->getChars().size(); + concrete_columns[i] = concrete_column; + } + + ColumnString & concreate_out_data = assert_cast(out_data); + auto & out_chars = concreate_out_data.getChars(); + auto & out_offsets = concreate_out_data.getOffsets(); + out_chars.resize_exact(total_bytes); + out_offsets.resize_exact(input_rows_count * columns.size()); + + size_t curr_out_offset = 0; + for (size_t row_i = 0; row_i < input_rows_count; ++row_i) + { + const size_t base = row_i * columns.size(); + for (size_t col_i = 0; col_i < columns.size(); ++col_i) + { + StringRef ref = concrete_columns[col_i]->getDataAt(row_i); + memcpySmallAllowReadWriteOverflow15(&out_chars[curr_out_offset], ref.data, ref.size); + out_chars[curr_out_offset + ref.size] = 0; + + curr_out_offset += ref.size + 1; + out_offsets[base + col_i] = curr_out_offset; + } + } + return true; + } + + bool executeFixedString(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + size_t total_bytes = 0; + std::vector concrete_columns(columns.size(), nullptr); + for (size_t i = 0; i < columns.size(); ++i) + { + const ColumnFixedString * concrete_column = checkAndGetColumn(columns[i]); + if (!concrete_column) + return false; + + total_bytes += concrete_column->getChars().size(); + concrete_columns[i] = concrete_column; + } + + ColumnFixedString & concreate_out_data = assert_cast(out_data); + auto & out_chars = concreate_out_data.getChars(); + out_chars.resize_exact(total_bytes); + + const size_t n = concreate_out_data.getN(); + size_t curr_out_offset = 0; + for (size_t row_i = 0; row_i < input_rows_count; ++row_i) + { + for (size_t col_i = 0; col_i < columns.size(); ++col_i) + { + StringRef ref = concrete_columns[col_i]->getDataAt(row_i); + memcpySmallAllowReadWriteOverflow15(&out_chars[curr_out_offset], ref.data, n); + curr_out_offset += n; + } + } + return true; + } + + bool executeNullable(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + ColumnRawPtrs null_maps(columns.size(), nullptr); + ColumnRawPtrs nested_columns(columns.size(), nullptr); + for (size_t i = 0; i < columns.size(); ++i) + { + const ColumnNullable * concrete_column = checkAndGetColumn(columns[i]); + if (!concrete_column) + return false; + + null_maps[i] = &concrete_column->getNullMapColumn(); + nested_columns[i] = &concrete_column->getNestedColumn(); + } + + ColumnNullable & concreate_out_data = assert_cast(out_data); + auto & out_null_map = concreate_out_data.getNullMapColumn(); + auto & out_nested_column = concreate_out_data.getNestedColumn(); + executeAny(null_maps, out_null_map, input_rows_count); + executeAny(nested_columns, out_nested_column, input_rows_count); + return true; + } + + bool executeTuple(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const + { + ColumnTuple * concreate_out_data = typeid_cast(&out_data); + if (!concreate_out_data) + return false; + + const size_t tuple_size = concreate_out_data->tupleSize(); + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnRawPtrs elem_columns(columns.size(), nullptr); + for (size_t j = 0; j < columns.size(); ++j) + { + const ColumnTuple * concrete_column = assert_cast(columns[j]); + elem_columns[j] = &concrete_column->getColumn(i); + } + executeAny(elem_columns, concreate_out_data->getColumn(i), input_rows_count); + } + return true; + } + String getName() const override { return name; } - bool addField(DataTypePtr type_res, const Field & f, Array & arr) const; - bool use_variant_as_common_type = false; }; From 00a9c30363e16aa1893320c35052ad551b803394 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 4 Aug 2024 16:16:51 +0800 Subject: [PATCH 0480/1722] optimize map function --- src/Functions/map.cpp | 44 +++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 42 insertions(+), 2 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index ebfa9d4eab4..994a03374c9 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -32,11 +32,18 @@ class FunctionMap : public IFunction public: static constexpr auto name = "map"; - explicit FunctionMap(bool use_variant_as_common_type_) : use_variant_as_common_type(use_variant_as_common_type_) {} + explicit FunctionMap(ContextPtr context_) + : context(std::move(context_)) + , use_variant_as_common_type( + context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type) + , function_array(FunctionFactory::instance().get("array", context)) + , function_map_from_arrays(FunctionFactory::instance().get("mapFromArrays", context)) + { + } static FunctionPtr create(ContextPtr context) { - return std::make_shared(context->getSettingsRef().allow_experimental_variant_type && context->getSettingsRef().use_variant_as_common_type); + return std::make_shared(std::move(context)); } String getName() const override @@ -97,7 +104,36 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { size_t num_elements = arguments.size(); + if (num_elements == 0) + return result_type->createColumnConstWithDefaultValue(input_rows_count); + ColumnsWithTypeAndName key_args; + ColumnsWithTypeAndName value_args; + for (size_t i = 0; i < num_elements; i += 2) + { + key_args.emplace_back(arguments[i]); + value_args.emplace_back(arguments[i+1]); + } + + const auto & result_type_map = static_cast(*result_type); + const DataTypePtr & key_type = result_type_map.getKeyType(); + const DataTypePtr & value_type = result_type_map.getValueType(); + const DataTypePtr & key_array_type = std::make_shared(key_type); + const DataTypePtr & value_array_type = std::make_shared(value_type); + + /// key_array = array(args[0], args[2]...) + ColumnPtr key_array = function_array->build(key_args)->execute(key_args, key_array_type, input_rows_count); + /// value_array = array(args[1], args[3]...) + ColumnPtr value_array = function_array->build(value_args)->execute(value_args, value_array_type, input_rows_count); + + /// result = mapFromArrays(key_array, value_array) + ColumnsWithTypeAndName map_args{{key_array, key_array_type, ""}, {value_array, value_array_type, ""}}; + return function_map_from_arrays->build(map_args)->execute(map_args, result_type, input_rows_count); + } + /* + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + size_t num_elements = arguments.size(); if (num_elements == 0) return result_type->createColumnConstWithDefaultValue(input_rows_count); @@ -164,9 +200,13 @@ public: return ColumnMap::create(nested_column); } + */ private: + ContextPtr context; bool use_variant_as_common_type = false; + FunctionOverloadResolverPtr function_array; + FunctionOverloadResolverPtr function_map_from_arrays; }; /// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays From 122c9794a65cb6d0579ec41f6fb3b4848e5c989b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 4 Aug 2024 17:04:04 +0800 Subject: [PATCH 0481/1722] remove useless codes --- src/Functions/map.cpp | 71 ------------------------------------------- 1 file changed, 71 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 994a03374c9..4b1e41af796 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -130,77 +130,6 @@ public: ColumnsWithTypeAndName map_args{{key_array, key_array_type, ""}, {value_array, value_array_type, ""}}; return function_map_from_arrays->build(map_args)->execute(map_args, result_type, input_rows_count); } - /* - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - size_t num_elements = arguments.size(); - if (num_elements == 0) - return result_type->createColumnConstWithDefaultValue(input_rows_count); - - const auto & result_type_map = static_cast(*result_type); - const DataTypePtr & key_type = result_type_map.getKeyType(); - const DataTypePtr & value_type = result_type_map.getValueType(); - - Columns columns_holder(num_elements); - ColumnRawPtrs column_ptrs(num_elements); - - for (size_t i = 0; i < num_elements; ++i) - { - const auto & arg = arguments[i]; - const auto to_type = i % 2 == 0 ? key_type : value_type; - - ColumnPtr preprocessed_column = castColumn(arg, to_type); - preprocessed_column = preprocessed_column->convertToFullColumnIfConst(); - - columns_holder[i] = std::move(preprocessed_column); - column_ptrs[i] = columns_holder[i].get(); - } - - /// Create and fill the result map. - - MutableColumnPtr keys_data = key_type->createColumn(); - MutableColumnPtr values_data = value_type->createColumn(); - MutableColumnPtr offsets = DataTypeUInt64().createColumn(); - - size_t total_elements = input_rows_count * num_elements / 2; - keys_data->reserve(total_elements); - values_data->reserve(total_elements); - auto & offsets_data = assert_cast(*offsets).getData(); - offsets_data.resize_exact(input_rows_count); - - IColumn::Offset current_offset = 0; - if (num_elements == 2) - { - for (size_t i = 0; i < input_rows_count; ++i) - { - ++current_offset; - offsets_data[i] = current_offset; - } - keys_data->insertRangeFrom(*column_ptrs[0], 0, input_rows_count); - values_data->insertRangeFrom(*column_ptrs[1], 0, input_rows_count); - } - else - { - for (size_t i = 0; i < input_rows_count; ++i) - { - for (size_t j = 0; j < num_elements; j += 2) - { - keys_data->insertFrom(*column_ptrs[j], i); - values_data->insertFrom(*column_ptrs[j + 1], i); - } - - current_offset += num_elements / 2; - offsets_data[i] = current_offset; - } - } - - auto nested_column = ColumnArray::create( - ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}), - std::move(offsets)); - - return ColumnMap::create(nested_column); - } - */ private: ContextPtr context; From d1305d9fadf3ea2ad2393ee5f13a676110bf6ef7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 5 Aug 2024 11:26:57 +0800 Subject: [PATCH 0482/1722] Some fixups and split tests --- .../mergetree-family/mergetree.md | 5 ++- src/Storages/Statistics/Statistics.cpp | 24 ++++++------ .../Statistics/StatisticsCountMinSketch.cpp | 4 +- .../Statistics/StatisticsCountMinSketch.h | 4 +- src/Storages/Statistics/StatisticsMinMax.cpp | 17 ++++----- src/Storages/Statistics/StatisticsMinMax.h | 11 +++--- src/Storages/Statistics/StatisticsTDigest.cpp | 4 +- src/Storages/Statistics/StatisticsTDigest.h | 4 +- src/Storages/Statistics/StatisticsUniq.cpp | 4 +- src/Storages/Statistics/StatisticsUniq.h | 4 +- src/Storages/StatisticsDescription.cpp | 8 ++-- ...atistics_create_materialize_drop.reference | 6 +++ ...864_statistics_create_materialize_drop.sql | 37 +++++++++++++++++++ .../02864_statistics_estimation.reference | 6 +-- .../02864_statistics_estimation.sql | 28 +++----------- 15 files changed, 94 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference create mode 100644 tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 8e8d5e55772..42968097799 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -991,9 +991,10 @@ They can be used for prewhere optimization only if we enable `set allow_statisti #### Available Types of Column Statistics {#available-types-of-column-statistics} -- `min_max` +- `MinMax` + + MinMax statistics allows to estimate selectivity of range condition for numeric columns. - min_max statistics allows to estimate selectivity of range condition for numeric columns. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 17ff8deb720..42b742419f0 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -4,9 +4,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -99,11 +99,11 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const #endif if (stats.contains(StatisticsType::Uniq)) { - auto cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); + UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); if (cardinality == 0) return 0; /// Assume that the value is uniformly distributed among the unique values. - return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality() * rows; + return 1.0 / cardinality * rows; } return rows * ConditionSelectivityEstimator::default_cond_equal_factor; @@ -183,18 +183,18 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { - registerValidator(StatisticsType::MinMax, minMaxValidator); - registerCreator(StatisticsType::MinMax, minMaxCreator); + registerValidator(StatisticsType::MinMax, minMaxStatisticsValidator); + registerCreator(StatisticsType::MinMax, minMaxStatisticsCreator); - registerValidator(StatisticsType::TDigest, tdigestValidator); - registerCreator(StatisticsType::TDigest, tdigestCreator); + registerValidator(StatisticsType::TDigest, tdigestStatisticsValidator); + registerCreator(StatisticsType::TDigest, tdigestStatisticsCreator); - registerValidator(StatisticsType::Uniq, uniqValidator); - registerCreator(StatisticsType::Uniq, uniqCreator); + registerValidator(StatisticsType::Uniq, uniqStatisticsValidator); + registerCreator(StatisticsType::Uniq, uniqStatisticsCreator); #if USE_DATASKETCHES - registerValidator(StatisticsType::CountMinSketch, countMinSketchValidator); - registerCreator(StatisticsType::CountMinSketch, countMinSketchCreator); + registerValidator(StatisticsType::CountMinSketch, countMinSketchStatisticsValidator); + registerCreator(StatisticsType::CountMinSketch, countMinSketchStatisticsCreator); #endif } @@ -222,7 +222,7 @@ ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescri { auto it = creators.find(type); if (it == creators.end()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'min_max', 'tdigest' 'uniq' and 'count_min'", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'count_min', 'minmax', 'tdigest' and 'uniq'", type); auto stat_ptr = (it->second)(desc, stats.data_type); column_stat->stats[type] = stat_ptr; } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index e69bbc1515b..50d3b6e515c 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -84,7 +84,7 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -92,7 +92,7 @@ void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index 6c8b74f8c35..d10bc78a88e 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -31,8 +31,8 @@ private: }; -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 083e8634841..f689f23d75d 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -19,9 +19,6 @@ extern const int ILLEGAL_STATISTICS; StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_) : IStatistics(stat_) - , min(std::numeric_limits::max()) - , max(std::numeric_limits::min()) - , row_count(0) , data_type(data_type_) { } @@ -41,7 +38,7 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const return row_count; if (max == min) - return row_count; + return (val_as_float < max) ? 0 : row_count; return ((val_as_float - min) / (max - min)) * row_count; } @@ -53,9 +50,9 @@ void StatisticsMinMax::update(const ColumnPtr & column) if (column->isNullAt(row)) continue; - auto data = column->getFloat64(row); - min = std::min(data, min); - max = std::max(data, max); + auto value = column->getFloat64(row); + min = std::min(value, min); + max = std::max(value, max); } row_count += column->size(); } @@ -75,15 +72,15 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) } -void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'min_max' do not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName()); } -StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 3e08e6b5768..4a67504350b 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -10,7 +10,6 @@ class StatisticsMinMax : public IStatistics { public: StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_); - ~StatisticsMinMax() override = default; Float64 estimateLess(const Field & val) const override; @@ -20,14 +19,14 @@ public: void deserialize(ReadBuffer & buf) override; private: - Float64 min; - Float64 max; - Float64 row_count; + Float64 min = std::numeric_limits::max(); + Float64 max = std::numeric_limits::min(); + UInt64 row_count = 0; DataTypePtr data_type; }; -void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 25cee4ac8e8..d418a761e7c 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -58,7 +58,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const return t_digest.getCountEqual(val_as_float); } -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -66,7 +66,7 @@ void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index f4de6cc41e0..b83132c5e0d 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -24,7 +24,7 @@ private: DataTypePtr data_type; }; -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index aace45a14b1..628ba98aba3 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,7 +60,7 @@ void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 81162b0a5b9..219e2ffbc55 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -27,7 +27,7 @@ private: }; -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index c54752dd42f..f3ce9deb836 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -51,9 +51,9 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::Uniq; if (type == "count_min") return StatisticsType::CountMinSketch; - if (type == "min_max") + if (type == "minmax") return StatisticsType::MinMax; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type); } String SingleStatisticsDescription::getTypeName() const @@ -67,9 +67,9 @@ String SingleStatisticsDescription::getTypeName() const case StatisticsType::CountMinSketch: return "count_min"; case StatisticsType::MinMax: - return "min_max"; + return "minmax"; default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type); } } diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference new file mode 100644 index 00000000000..5aff1ac63d4 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference @@ -0,0 +1,6 @@ +Test create statistics: +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +tab +Test materialize and drop statistics: +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql new file mode 100644 index 00000000000..442079ab947 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql @@ -0,0 +1,37 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS tab SYNC; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_suspicious_low_cardinality_types=1; +SET mutations_sync = 2; + + +SELECT 'Test create statistics:'; + +CREATE TABLE tab +( + a LowCardinality(Int64) STATISTICS(count_min, minmax, uniq), + b Nullable(Int64), + c LowCardinality(Nullable(Int64)) STATISTICS(minmax, count_min), + d DateTime STATISTICS(count_min, minmax, tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk; + +SHOW CREATE TABLE tab; + +SELECT name FROM system.tables WHERE name = 'tab' AND database = currentDatabase(); +INSERT INTO tab select number, number, number, toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; + + +SELECT 'Test materialize and drop statistics:'; + +ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, tdigest, uniq; +ALTER TABLE tab MATERIALIZE STATISTICS b; +SHOW CREATE TABLE tab; + +ALTER TABLE tab DROP STATISTICS a, b, c, d; +SHOW CREATE TABLE tab; + +DROP TABLE IF EXISTS tab SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index 64f264017d8..a5225c6338f 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -1,5 +1,5 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime64(3),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics min_max: +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics minmax: Prewhere info Prewhere filter Prewhere filter column: and(greater(d, _CAST(9998_DateTime(\'UTC\'), \'DateTime(\'UTC\')\')), less(c, -1), greater(b, 0)) (removed) @@ -28,5 +28,3 @@ Test estimating equals condition: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 0ea9eb38e60..117e24bd732 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -12,7 +12,7 @@ CREATE TABLE tab a String, b UInt64, c Int64, - d DateTime64, + d DateTime, pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; @@ -21,11 +21,11 @@ SHOW CREATE TABLE tab; INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; -SELECT 'Test statistics min_max:'; +SELECT 'Test statistics minmax:'; -ALTER TABLE tab ADD STATISTICS b TYPE min_max; -ALTER TABLE tab ADD STATISTICS c TYPE min_max; -ALTER TABLE tab ADD STATISTICS d TYPE min_max; +ALTER TABLE tab ADD STATISTICS b TYPE minmax; +ALTER TABLE tab ADD STATISTICS c TYPE minmax; +ALTER TABLE tab ADD STATISTICS d TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') @@ -70,7 +70,7 @@ ALTER TABLE tab DROP STATISTICS a, b, c, d; SELECT 'Test estimating range condition:'; -ALTER TABLE tab ADD STATISTICS b TYPE min_max; +ALTER TABLE tab ADD STATISTICS b TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) @@ -100,19 +100,3 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a; DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(min_max, count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(min_max, count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; From 045551f5098e36822c69cc2bb5852d7d16cfe41b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 5 Aug 2024 11:33:58 +0800 Subject: [PATCH 0483/1722] fix bugs --- src/Functions/map.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index f49d66c0329..d9643087b11 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -157,6 +157,7 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { From 94611dbddc08565c0ac154ad62dab1a7e8c6d8ed Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 5 Aug 2024 11:48:46 +0800 Subject: [PATCH 0484/1722] keep ColumnLowCardinality::getName() the same style with other columns --- src/Columns/ColumnLowCardinality.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 3766b247d60..5a23853e961 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -46,8 +46,8 @@ public: return Base::create(std::move(column_unique), std::move(indexes), is_shared); } - std::string getName() const override { return "ColumnLowCardinality"; } - const char * getFamilyName() const override { return "ColumnLowCardinality"; } + std::string getName() const override { return "LowCardinality(" + getDictionaryPtr()->getName() + ")"; } + const char * getFamilyName() const override { return "LowCardinality"; } TypeIndex getDataType() const override { return TypeIndex::LowCardinality; } ColumnPtr convertToFullColumn() const { return getDictionary().getNestedColumn()->index(getIndexes(), 0); } From 7765ff6d5227da58eecc426287c98c333c75402d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Aug 2024 06:46:04 +0000 Subject: [PATCH 0485/1722] Minor fixups --- .../mergetree-family/mergetree.md | 2 +- src/Storages/Statistics/Statistics.cpp | 22 ++++---- src/Storages/Statistics/Statistics.h | 12 ++-- .../Statistics/StatisticsCountMinSketch.cpp | 52 +++++++++--------- .../Statistics/StatisticsCountMinSketch.h | 11 ++-- src/Storages/Statistics/StatisticsMinMax.cpp | 55 +++++++++---------- src/Storages/Statistics/StatisticsMinMax.h | 11 ++-- src/Storages/Statistics/StatisticsTDigest.cpp | 11 ++-- src/Storages/Statistics/StatisticsTDigest.h | 13 +++-- src/Storages/Statistics/StatisticsUniq.cpp | 10 ++-- src/Storages/Statistics/StatisticsUniq.h | 12 ++-- 11 files changed, 107 insertions(+), 104 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 42968097799..c2438ac593e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -993,7 +993,7 @@ They can be used for prewhere optimization only if we enable `set allow_statisti - `MinMax` - MinMax statistics allows to estimate selectivity of range condition for numeric columns. + The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. - `TDigest` diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 42b742419f0..1f159211a93 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -62,14 +62,17 @@ Float64 IStatistics::estimateLess(const Field & /*val*/) const } /// ------------------------------------- -/// Implementation of the estimation: -/// Note: Each statistics object supports certain types predicates natively, e.g. -/// - TDigest: '< X' (less-than predicates) -/// - Count-min sketches: '= X' (equal predicates) -/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) -/// If multiple statistics objects are available per column, it is sometimes also possible to combine them in a clever way. -/// For that reason, all estimation are performed in a central place (here), and we don't simply pass the predicate to the first statistics -/// object that supports it natively. +/// Notes: +/// - Statistics object usually only support estimation for certain types of predicates, e.g. +/// - TDigest: '< X' (less-than predicates) +/// - Count-min sketches: '= X' (equal predicates) +/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) +/// +/// If multiple statistics objects in a column support estimating a predicate, we want to try statistics in order of descending accuracy +/// (e.g. MinMax statistics are simpler than TDigest statistics and thus worse for estimating 'less' predicates). +/// +/// Sometimes, it is possible to combine multiple statistics in a clever way. For that reason, all estimation are performed in a central +/// place (here), and we don't simply pass the predicate to the first statistics object that supports it natively. Float64 ColumnStatistics::estimateLess(const Field & val) const { @@ -102,8 +105,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); if (cardinality == 0) return 0; - /// Assume that the value is uniformly distributed among the unique values. - return 1.0 / cardinality * rows; + return 1.0 / cardinality * rows; /// assume uniform distribution } return rows * ConditionSelectivityEstimator::default_cond_equal_factor; diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 3eeb5cf92a8..1425be89d47 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -25,20 +25,20 @@ public: explicit IStatistics(const SingleStatisticsDescription & stat_); virtual ~IStatistics() = default; + virtual void update(const ColumnPtr & column) = 0; + + virtual void serialize(WriteBuffer & buf) = 0; + virtual void deserialize(ReadBuffer & buf) = 0; + /// Estimate the cardinality of the column. /// Throws if the statistics object is not able to do a meaningful estimation. virtual UInt64 estimateCardinality() const; /// Per-value estimations. - /// Throws if the statistics object is not able to do a meaningful estimation. + /// Throws a LOGICAL_ERROR if the statistics object is not able to do a meaningful estimation. virtual Float64 estimateEqual(const Field & val) const; /// cardinality of val in the column virtual Float64 estimateLess(const Field & val) const; /// summarized cardinality of values < val in the column - virtual void update(const ColumnPtr & column) = 0; - - virtual void serialize(WriteBuffer & buf) = 0; - virtual void deserialize(ReadBuffer & buf) = 0; - protected: SingleStatisticsDescription stat; }; diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index 50d3b6e515c..c0ea7a84ded 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -1,4 +1,3 @@ - #include #include #include @@ -25,34 +24,13 @@ extern const int ILLEGAL_STATISTICS; static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; -StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_) +StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_) + : IStatistics(statistics_description) , sketch(num_hashes, num_buckets) , data_type(data_type_) { } -Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const -{ - /// Try to convert field to data_type. Converting string to proper data types such as: number, date, datetime, IPv4, Decimal etc. - /// Return null if val larger than the range of data_type - /// - /// For example: if data_type is Int32: - /// 1. For 1.0, 1, '1', return Field(1) - /// 2. For 1.1, max_value_int64, return null - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) - return 0; - - if (data_type->isValueRepresentedByNumber()) - return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); - - if (isStringOrFixedString(data_type)) - return sketch.get_estimate(val.get()); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); -} - void StatisticsCountMinSketch::update(const ColumnPtr & column) { for (size_t row = 0; row < column->size(); ++row) @@ -83,8 +61,28 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) sketch = Sketch::deserialize(bytes.data(), size); } +Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const +{ + /// Try to convert field to data_type. Converting string to proper data types such as: number, date, datetime, IPv4, Decimal etc. + /// Return null if val larger than the range of data_type + /// + /// For example: if data_type is Int32: + /// 1. For 1.0, 1, '1', return Field(1) + /// 2. For 1.1, max_value_int64, return null + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; -void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) + if (data_type->isValueRepresentedByNumber()) + return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); + + if (isStringOrFixedString(data_type)) + return sketch.get_estimate(val.get()); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); +} + +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -92,9 +90,9 @@ void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, Data throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index d10bc78a88e..452483c8773 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "config.h" @@ -14,15 +15,15 @@ namespace DB class StatisticsCountMinSketch : public IStatistics { public: - StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); - - Float64 estimateEqual(const Field & val) const override; + StatisticsCountMinSketch(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_); void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + Float64 estimateEqual(const Field & val) const override; + private: using Sketch = datasketches::count_min_sketch; Sketch sketch; @@ -31,8 +32,8 @@ private: }; -void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index f689f23d75d..11a0704dae7 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -1,10 +1,10 @@ #include -#include -#include -#include #include +#include +#include #include #include +#include #include @@ -17,32 +17,12 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_) - : IStatistics(stat_) +StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_) + : IStatistics(statistics_description) , data_type(data_type_) { } -Float64 StatisticsMinMax::estimateLess(const Field & val) const -{ - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) - return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - - if (val_as_float < min) - return 0; - - if (val_as_float > max) - return row_count; - - if (max == min) - return (val_as_float < max) ? 0 : row_count; - - return ((val_as_float - min) / (max - min)) * row_count; -} - void StatisticsMinMax::update(const ColumnPtr & column) { for (size_t row = 0; row < column->size(); ++row) @@ -71,8 +51,27 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) readFloatBinary(max, buf); } +Float64 StatisticsMinMax::estimateLess(const Field & val) const +{ + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; -void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + + if (val_as_float < min) + return 0; + + if (val_as_float > max) + return row_count; + + if (min == max) + return (val_as_float != max) ? 0 : row_count; + + return ((val_as_float - min) / (max - min)) * row_count; +} + +void minMaxStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -80,9 +79,9 @@ void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName()); } -StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 4a67504350b..524494b520b 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -9,15 +10,15 @@ namespace DB class StatisticsMinMax : public IStatistics { public: - StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_); - - Float64 estimateLess(const Field & val) const override; + StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_); void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + Float64 estimateLess(const Field & val) const override; + private: Float64 min = std::numeric_limits::max(); Float64 max = std::numeric_limits::min(); @@ -26,7 +27,7 @@ private: DataTypePtr data_type; }; -void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void minMaxStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index d418a761e7c..cfd626c8ea8 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -11,8 +11,9 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_), data_type(data_type_) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_) + : IStatistics(statistics_description) + , data_type(data_type_) { } @@ -58,7 +59,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const return t_digest.getCountEqual(val_as_float); } -void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -66,9 +67,9 @@ void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index b83132c5e0d..ad2a03836aa 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -9,22 +10,22 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); - - Float64 estimateLess(const Field & val) const override; - Float64 estimateEqual(const Field & val) const override; + explicit StatisticsTDigest(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_); void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + Float64 estimateLess(const Field & val) const override; + Float64 estimateEqual(const Field & val) const override; + private: QuantileTDigest t_digest; DataTypePtr data_type; }; -void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 628ba98aba3..e895157635f 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type) - : IStatistics(stat_) +StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type) + : IStatistics(statistics_description) { arena = std::make_unique(); AggregateFunctionProperties properties; @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,9 +60,9 @@ void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr da throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 219e2ffbc55..b3000e4895a 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -10,24 +11,23 @@ namespace DB class StatisticsUniq : public IStatistics { public: - StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); + StatisticsUniq(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type); ~StatisticsUniq() override; - UInt64 estimateCardinality() const override; - void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + UInt64 estimateCardinality() const override; + private: std::unique_ptr arena; AggregateFunctionPtr collector; AggregateDataPtr data; - }; -void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); } From 2251ad963992d4656c5ae20a7221aff36a86cc1d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 5 Aug 2024 16:44:53 +0800 Subject: [PATCH 0486/1722] optimize orc string column reading --- .../Impl/NativeORCBlockInputFormat.cpp | 34 ++++++++++++++----- 1 file changed, 26 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 649721f28bf..a0a80ec4a58 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1143,24 +1143,42 @@ readColumnWithStringData(const orc::ColumnVectorBatch * orc_column, const orc::T reserver_size += 1; } - column_chars_t.reserve(reserver_size); - column_offsets.reserve(orc_str_column->numElements); + column_chars_t.resize_exact(reserver_size); + column_offsets.resize_exact(orc_str_column->numElements); size_t curr_offset = 0; - for (size_t i = 0; i < orc_str_column->numElements; ++i) + if (!orc_str_column->hasNulls) { - if (!orc_str_column->hasNulls || orc_str_column->notNull[i]) + for (size_t i = 0; i < orc_str_column->numElements; ++i) { const auto * buf = orc_str_column->data[i]; size_t buf_size = orc_str_column->length[i]; - column_chars_t.insert_assume_reserved(buf, buf + buf_size); + memcpy(&column_chars_t[curr_offset], buf, buf_size); curr_offset += buf_size; + + column_chars_t[curr_offset] = 0; + ++curr_offset; + + column_offsets[i] = curr_offset; } + } + else + { + for (size_t i = 0; i < orc_str_column->numElements; ++i) + { + if (orc_str_column->notNull[i]) + { + const auto * buf = orc_str_column->data[i]; + size_t buf_size = orc_str_column->length[i]; + memcpy(&column_chars_t[curr_offset], buf, buf_size); + curr_offset += buf_size; + } - column_chars_t.push_back(0); - ++curr_offset; + column_chars_t[curr_offset] = 0; + ++curr_offset; - column_offsets.push_back(curr_offset); + column_offsets[i] = curr_offset; + } } return {std::move(internal_column), std::move(internal_type), column_name}; } From f816158dbc000c4fb02ba413f5b47349795995c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Aug 2024 09:11:55 +0000 Subject: [PATCH 0487/1722] Address review comments --- src/Storages/MessageQueueSink.cpp | 4 ++-- tests/integration/test_storage_kafka/test.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index d4dabd60ef8..104e3506ade 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -46,6 +46,8 @@ void MessageQueueSink::consume(Chunk & chunk) if (columns.empty()) return; + /// The formatter might hold pointers to buffer (e.g. if PeekableWriteBuffer is used), which means the formatter + /// needs to be reset after buffer might reallocate its memory. In this exact case after restarting the buffer. if (row_format) { size_t row = 0; @@ -77,6 +79,4 @@ void MessageQueueSink::consume(Chunk & chunk) format->resetFormatter(); } } - - } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index dd0bf1bf28f..8793ab72a16 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5098,7 +5098,7 @@ def test_multiple_read_in_materialized_views(kafka_cluster, max_retries=15): def test_kafka_produce_http_interface_row_based_format(kafka_cluster): - # reproduction of #https://github.com/ClickHouse/ClickHouse/issues/61060 with validating the written messages + # reproduction of #61060 with validating the written messages admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) From 719ccaba5acb87734d1bb4cc2c4f5e76ad978c0a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 5 Aug 2024 18:36:46 +0800 Subject: [PATCH 0488/1722] optimize parquet string column reading --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 25 +++++++++++++++---- .../Impl/NativeORCBlockInputFormat.cpp | 1 + 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index ed91913de4d..fb56fdd4fe0 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -133,16 +133,31 @@ static ColumnWithTypeAndName readColumnWithStringData(const std::shared_ptr buffer = chunk.value_data(); const size_t chunk_length = chunk.length(); - for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + const size_t null_count = chunk.null_count(); + if (null_count == 0) { - if (!chunk.IsNull(offset_i) && buffer) + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) { const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); - } - column_chars_t.emplace_back('\0'); + column_chars_t.emplace_back('\0'); - column_offsets.emplace_back(column_chars_t.size()); + column_offsets.emplace_back(column_chars_t.size()); + } + } + else + { + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + { + if (!chunk.IsNull(offset_i) && buffer) + { + const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); + column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); + } + column_chars_t.emplace_back('\0'); + + column_offsets.emplace_back(column_chars_t.size()); + } } } return {std::move(internal_column), std::move(internal_type), column_name}; diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index a0a80ec4a58..81bea0af53b 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1,4 +1,5 @@ #include "NativeORCBlockInputFormat.h" +#include "Columns/ColumnsCommon.h" #if USE_ORC # include From c61225ede47e37bd773bd249804739f915feefd7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 5 Aug 2024 18:37:36 +0800 Subject: [PATCH 0489/1722] Rewrite tests --- ...atistics_create_materialize_drop.reference | 7 +- ...864_statistics_create_materialize_drop.sql | 16 ++-- .../02864_statistics_estimation.reference | 34 +++---- .../02864_statistics_estimation.sql | 92 ++++++++----------- 4 files changed, 62 insertions(+), 87 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference index 5aff1ac63d4..4ca2c5e5f9b 100644 --- a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference @@ -1,6 +1,5 @@ Test create statistics: -CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 -tab +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `b` LowCardinality(Nullable(String)) STATISTICS(uniq, count_min),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(tdigest, uniq, count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 Test materialize and drop statistics: -CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` LowCardinality(Nullable(String)) STATISTICS(uniq, count_min),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` LowCardinality(Nullable(String)),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql index 442079ab947..90a57c99624 100644 --- a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql @@ -12,26 +12,24 @@ SELECT 'Test create statistics:'; CREATE TABLE tab ( - a LowCardinality(Int64) STATISTICS(count_min, minmax, uniq), - b Nullable(Int64), - c LowCardinality(Nullable(Int64)) STATISTICS(minmax, count_min), + a LowCardinality(Int64) STATISTICS(count_min, minmax, tdigest, uniq), + b LowCardinality(Nullable(String)) STATISTICS(count_min, uniq), + c LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax, tdigest, uniq), d DateTime STATISTICS(count_min, minmax, tdigest, uniq), pk String, ) Engine = MergeTree() ORDER BY pk; +INSERT INTO tab select number, number, number, toDateTime(number), generateUUIDv4() FROM system.numbers LIMIT 10000; SHOW CREATE TABLE tab; -SELECT name FROM system.tables WHERE name = 'tab' AND database = currentDatabase(); -INSERT INTO tab select number, number, number, toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; - SELECT 'Test materialize and drop statistics:'; - -ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, tdigest, uniq; +ALTER TABLE tab DROP STATISTICS a, b, c, d; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq; ALTER TABLE tab MATERIALIZE STATISTICS b; SHOW CREATE TABLE tab; -ALTER TABLE tab DROP STATISTICS a, b, c, d; +ALTER TABLE tab DROP STATISTICS b; SHOW CREATE TABLE tab; DROP TABLE IF EXISTS tab SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index a5225c6338f..f59f94013da 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -1,30 +1,24 @@ CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics minmax: - Prewhere info - Prewhere filter - Prewhere filter column: and(greater(d, _CAST(9998_DateTime(\'UTC\'), \'DateTime(\'UTC\')\')), less(c, -1), greater(b, 0)) (removed) Test statistics count_min: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics minmax: + Prewhere info + Prewhere filter + Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) +Test statistics tdigest: + Prewhere info + Prewhere filter + Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) +Test statistics uniq: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), equals(b, 0), equals(c, 0)) (removed) Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test estimating range condition: - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) -Test estimating equals condition: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 117e24bd732..65b111c285b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -19,84 +19,68 @@ SETTINGS min_bytes_for_wide_part = 0; SHOW CREATE TABLE tab; -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), cast(number, 'DateTime'), generateUUIDv4() FROM system.numbers LIMIT 10000; + + +SELECT 'Test statistics count_min:'; + +ALTER TABLE tab ADD STATISTICS a, b, c TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS a, b, c; + SELECT 'Test statistics minmax:'; -ALTER TABLE tab ADD STATISTICS b TYPE minmax; -ALTER TABLE tab ADD STATISTICS c TYPE minmax; -ALTER TABLE tab ADD STATISTICS d TYPE minmax; +ALTER TABLE tab ADD STATISTICS b, c, d TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > toDateTime(9998, 'UTC')/*1*/) +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS b, c, d; -SELECT 'Test statistics count_min:'; +SELECT 'Test statistics tdigest:'; -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min; -ALTER TABLE tab ADD STATISTICS c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) +ALTER TABLE tab ADD STATISTICS b, c, d TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS b, c, d; -ALTER TABLE tab DROP STATISTICS a, b, c; + +SELECT 'Test statistics uniq:'; + +ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/ and b = 0/*100*/ and a = '0'/*100*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS a, b, c, d; SELECT 'Test statistics multi-types:'; -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS d TYPE count_min, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS c TYPE count_min, minmax, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; -SELECT 'Test estimating range condition:'; - -ALTER TABLE tab ADD STATISTICS b TYPE minmax; -ALTER TABLE tab MATERIALIZE STATISTICS b; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS b; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS b; - - -SELECT 'Test estimating equals condition:'; - -ALTER TABLE tab ADD STATISTICS a TYPE uniq; -ALTER TABLE tab MATERIALIZE STATISTICS a; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS a; - DROP TABLE IF EXISTS tab SYNC; From f147e5c39e19d1097361571ebedf4507c744c700 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 5 Aug 2024 18:37:55 +0800 Subject: [PATCH 0490/1722] optimize parquet string column reading --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 81bea0af53b..a0a80ec4a58 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1,5 +1,4 @@ #include "NativeORCBlockInputFormat.h" -#include "Columns/ColumnsCommon.h" #if USE_ORC # include From a36424fc8c99fd5c20b52d80a10d232c8a61451c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 5 Aug 2024 18:59:09 +0800 Subject: [PATCH 0491/1722] Add supported data types to documents --- .../table-engines/mergetree-family/mergetree.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index c2438ac593e..8fd60c36f94 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1007,6 +1007,19 @@ They can be used for prewhere optimization only if we enable `set allow_statisti [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. +#### Supported Data Types {#supported-data-types} + +| Data Type / Statistics | count_min | minmax | tdigest | uniq | +|-------------------------|-----------|--------|---------|------| +| Integer types | ✔ | ✔ | ✔ | ✔ | +| Floating-point numbers | ✔ | ✔ | ✔ | ✔ | +| Boolean | ✔ | ✔ | ✔ | ✔ | +| Strings | ✔ | ✗ | ✗ | ✔ | +| Dates | ✔ | ✔ | ✔ | ✔ | + +You can also add `LowCardinality` and `Nullable` modifiers to the data types. + + ## Column-level Settings {#column-level-settings} Certain MergeTree settings can be override at column level: From e2b74e4176dca494c07c69b4dc3f4435c29aad3e Mon Sep 17 00:00:00 2001 From: sunny19930321 Date: Mon, 5 Aug 2024 19:32:45 +0800 Subject: [PATCH 0492/1722] feat: add delete from in pastition cmd --- docs/en/sql-reference/statements/delete.md | 2 +- src/Interpreters/InterpreterDeleteQuery.cpp | 8 +++++++ src/Parsers/ASTDeleteQuery.cpp | 6 +++++ src/Parsers/ASTDeleteQuery.h | 5 ++++ src/Parsers/ParserDeleteQuery.cpp | 12 ++++++++++ ..._lightweight_delete_in_partition.reference | 4 ++++ .../02352_lightweight_delete_in_partition.sql | 23 +++++++++++++++++++ 7 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference create mode 100644 tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index a52b7204c30..fe3868bcab4 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -10,7 +10,7 @@ title: The Lightweight DELETE Statement The lightweight `DELETE` statement removes rows from the table `[db.]table` that match the expression `expr`. It is only available for the *MergeTree table engine family. ``` sql -DELETE FROM [db.]table [ON CLUSTER cluster] WHERE expr; +DELETE FROM [db.]table [ON CLUSTER cluster] [IN PARTITION partition_expr] WHERE expr; ``` It is called "lightweight `DELETE`" to contrast it to the [ALTER table DELETE](/en/sql-reference/statements/alter/delete) command, which is a heavyweight process. diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 291c8e19db0..69aae3ae909 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -79,6 +79,14 @@ BlockIO InterpreterDeleteQuery::execute() "ALTER TABLE " + table->getStorageID().getFullTableName() + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) + " UPDATE `_row_exists` = 0 WHERE " + serializeAST(*delete_query.predicate); + if (delete_query.partition) + { + alter_query = + "ALTER TABLE " + table->getStorageID().getFullTableName() + + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) + + " UPDATE `_row_exists` = 0 IN PARTITION " + serializeAST(*delete_query.partition) + " WHERE " + + serializeAST(*delete_query.predicate); + } ParserAlterQuery parser; ASTPtr alter_ast = parseQuery( diff --git a/src/Parsers/ASTDeleteQuery.cpp b/src/Parsers/ASTDeleteQuery.cpp index 67f3a85c9a5..434cc344a5a 100644 --- a/src/Parsers/ASTDeleteQuery.cpp +++ b/src/Parsers/ASTDeleteQuery.cpp @@ -45,6 +45,12 @@ void ASTDeleteQuery::formatQueryImpl(const FormatSettings & settings, FormatStat formatOnCluster(settings); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); predicate->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTDeleteQuery.h b/src/Parsers/ASTDeleteQuery.h index cc115a366db..6adc9b6a1c8 100644 --- a/src/Parsers/ASTDeleteQuery.h +++ b/src/Parsers/ASTDeleteQuery.h @@ -19,6 +19,11 @@ public: return removeOnCluster(clone(), params.default_database); } + /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE, DELETE FROM queries. + * The value or ID of the partition is stored here. + */ + ASTPtr partition; + ASTPtr predicate; protected: diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp index 00fac45c8ed..65d593b6c1b 100644 --- a/src/Parsers/ParserDeleteQuery.cpp +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -15,11 +16,14 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_delete(Keyword::DELETE); ParserKeyword s_from(Keyword::FROM); + ParserKeyword s_in_partition(Keyword::IN_PARTITION); ParserKeyword s_where(Keyword::WHERE); ParserExpression parser_exp_elem; ParserKeyword s_settings(Keyword::SETTINGS); ParserKeyword s_on{Keyword::ON}; + ParserPartition parser_partition; + if (s_delete.ignore(pos, expected)) { if (!s_from.ignore(pos, expected)) @@ -36,6 +40,12 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->cluster = cluster_str; } + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, query->partition, expected)) + return false; + } + if (!s_where.ignore(pos, expected)) return false; @@ -52,6 +62,8 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else return false; + if (query->partition) + query->children.push_back(query->partition); if (query->predicate) query->children.push_back(query->predicate); diff --git a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference new file mode 100644 index 00000000000..2bd7b51cea0 --- /dev/null +++ b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.reference @@ -0,0 +1,4 @@ +200 +200 +100 +100 diff --git a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql new file mode 100644 index 00000000000..08cf5816f74 --- /dev/null +++ b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS t_merge_tree SYNC; +DROP TABLE IF EXISTS t_replicated_merge_tree SYNC; + +CREATE TABLE t_merge_tree(time Date, id String , name String) ENGINE = MergeTree() PARTITION BY time ORDER BY id; +CREATE TABLE t_replicated_merge_tree(time Date, id String, name String) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_rep','1') PARTITION BY time ORDER BY id; + +INSERT INTO t_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); +INSERT INTO t_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); + +INSERT INTO t_replicated_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); +INSERT INTO t_replicated_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); + +SELECT COUNT() FROM t_merge_tree; +SELECT COUNT() FROM t_replicated_merge_tree; + +DELETE FROM t_merge_tree IN PARTITION '2024-08-01' WHERE id = '1'; +DELETE FROM t_replicated_merge_tree IN PARTITION '2024-08-01' WHERE id = '1'; + +SELECT COUNT() FROM t_merge_tree; +SELECT COUNT() FROM t_replicated_merge_tree; + +DROP TABLE t_merge_tree SYNC; +DROP TABLE t_replicated_merge_tree SYNC; From fd1e354e8503a968eff9dbe614a47c1135f67bdd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 5 Aug 2024 13:43:06 +0200 Subject: [PATCH 0493/1722] fix flaky check for integration tests --- .../test_storage_azure_blob_storage/test.py | 82 +++++++++++++------ .../test_cluster.py | 18 ++-- tests/integration/test_storage_hdfs/test.py | 47 +++++++---- 3 files changed, 94 insertions(+), 53 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6966abfee4f..15a1f6db2c1 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -135,6 +135,7 @@ def test_create_table_connection_string(cluster): Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV') """, ) + azure_query(node, "DROP TABLE IF EXISTS test_create_table_conn_string") def test_create_table_account_string(cluster): @@ -144,6 +145,7 @@ def test_create_table_account_string(cluster): f"CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," f"'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) + azure_query(node, "DROP TABLE IF EXISTS test_create_table_account_url") def test_simple_write_account_string(cluster): @@ -157,6 +159,7 @@ def test_simple_write_account_string(cluster): azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") print(get_azure_file_content("test_simple_write.csv", port)) assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n' + azure_query(node, "DROP TABLE test_simple_write") def test_simple_write_connection_string(cluster): @@ -170,6 +173,7 @@ def test_simple_write_connection_string(cluster): azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_c.csv", port)) assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n' + azure_query(node, "DROP TABLE test_simple_write_connection_string") def test_simple_write_named_collection_1(cluster): @@ -185,7 +189,7 @@ def test_simple_write_named_collection_1(cluster): ) print(get_azure_file_content("test_simple_write_named.csv", port)) assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n' - azure_query(node, "TRUNCATE TABLE test_simple_write_named_collection_1") + azure_query(node, "DROP TABLE test_simple_write_named_collection_1") def test_simple_write_named_collection_2(cluster): @@ -202,6 +206,7 @@ def test_simple_write_named_collection_2(cluster): ) print(get_azure_file_content("test_simple_write_named_2.csv", port)) assert get_azure_file_content("test_simple_write_named_2.csv", port) == '1,"a"\n' + azure_query(node, "DROP TABLE test_simple_write_named_collection_2") def test_partition_by(cluster): @@ -223,6 +228,7 @@ def test_partition_by(cluster): assert "1,2,3\n" == get_azure_file_content("test_3.csv", port) assert "3,2,1\n" == get_azure_file_content("test_1.csv", port) assert "78,43,45\n" == get_azure_file_content("test_45.csv", port) + azure_query(node, "DROP TABLE test_partitioned_write") def test_partition_by_string_column(cluster): @@ -243,6 +249,7 @@ def test_partition_by_string_column(cluster): assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv", port) assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv", port) assert '78,"你好"\n' == get_azure_file_content("test_你好.csv", port) + azure_query(node, "DROP TABLE test_partitioned_string_write") def test_partition_by_const_column(cluster): @@ -261,6 +268,7 @@ def test_partition_by_const_column(cluster): ) azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") assert values_csv == get_azure_file_content("test_88.csv", port) + azure_query(node, "DROP TABLE test_partitioned_const_write") def test_truncate(cluster): @@ -276,6 +284,7 @@ def test_truncate(cluster): azure_query(node, "TRUNCATE TABLE test_truncate") with pytest.raises(Exception): print(get_azure_file_content("test_truncate.csv", port)) + azure_query(node, "DROP TABLE test_truncate") def test_simple_read_write(cluster): @@ -292,6 +301,7 @@ def test_simple_read_write(cluster): assert get_azure_file_content("test_simple_read_write.csv", port) == '1,"a"\n' print(azure_query(node, "SELECT * FROM test_simple_read_write")) assert azure_query(node, "SELECT * FROM test_simple_read_write") == "1\ta\n" + azure_query(node, "DROP TABLE test_simple_read_write") def test_create_new_files_on_insert(cluster): @@ -344,6 +354,7 @@ def test_overwrite(cluster): result = azure_query(node, f"select count() from test_overwrite") assert int(result) == 200 + azure_query(node, f"DROP TABLE test_overwrite") def test_insert_with_path_with_globs(cluster): @@ -356,6 +367,7 @@ def test_insert_with_path_with_globs(cluster): node.query_and_get_error( f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" ) + azure_query(node, f"DROP TABLE test_insert_globs") def test_put_get_with_globs(cluster): @@ -364,6 +376,7 @@ def test_put_get_with_globs(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + used_names = [] for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( @@ -372,6 +385,8 @@ def test_put_get_with_globs(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" + used_names.append(f"test_put_{i}_{j}") + azure_query( node, f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " @@ -392,6 +407,9 @@ def test_put_get_with_globs(cluster): bucket="cont", max_path=max_path ) ] + azure_query(node, "DROP TABLE test_glob_select") + for name in used_names: + azure_query(node, f"DROP TABLE {name}") def test_azure_glob_scheherazade(cluster): @@ -400,12 +418,14 @@ def test_azure_glob_scheherazade(cluster): values = "(1, 1, 1)" nights_per_job = 1001 // 30 jobs = [] + used_names = [] for night in range(0, 1001, nights_per_job): def add_tales(start, end): for i in range(start, end): path = "night_{}/tale.csv".format(i) unique_num = random.randint(1, 10000) + used_names.append(f"test_scheherazade_{i}_{unique_num}") azure_query( node, f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " @@ -433,6 +453,9 @@ def test_azure_glob_scheherazade(cluster): ) query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] + azure_query(node, "DROP TABLE test_glob_select_scheherazade") + for name in used_names: + azure_query(node, f"DROP TABLE {name}") @pytest.mark.parametrize( @@ -506,6 +529,8 @@ def test_schema_inference_no_globs(cluster): assert azure_query(node, query).splitlines() == [ "499500\t2890\t332833500\ttest_schema_inference_no_globs.csv\tcont/test_schema_inference_no_globs.csv" ] + azure_query(node, f"DROP TABLE test_schema_inference_src") + azure_query(node, f"DROP TABLE test_select_inference") def test_schema_inference_from_globs(cluster): @@ -514,6 +539,7 @@ def test_schema_inference_from_globs(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + used_names = [] for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( @@ -521,6 +547,7 @@ def test_schema_inference_from_globs(cluster): ) max_path = max(path, max_path) values = f"({i},{j},{i + j})" + used_names.append(f"test_schema_{i}_{j}") azure_query( node, @@ -546,6 +573,9 @@ def test_schema_inference_from_globs(cluster): bucket="cont", max_path=max_path ) ] + azure_query(node, "DROP TABLE test_glob_select_inference") + for name in used_names: + azure_query(node, f"DROP TABLE {name}") def test_simple_write_account_string_table_function(cluster): @@ -595,7 +625,7 @@ def test_simple_write_named_collection_1_table_function(cluster): azure_query( node, - "TRUNCATE TABLE drop_table", + "DROP TABLE drop_table", ) @@ -605,7 +635,7 @@ def test_simple_write_named_collection_2_table_function(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a') SETTINGS azure_truncate_on_insert=1", ) print(get_azure_file_content("test_simple_write_named_2_tf.csv", port)) assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n' @@ -628,7 +658,7 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", ) query = ( f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " @@ -649,7 +679,7 @@ def test_schema_inference_no_globs_tf(cluster): query = ( f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " f"container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') " - f"SELECT number, toString(number), number * number FROM numbers(1000)" + f"SELECT number, toString(number), number * number FROM numbers(1000) SETTINGS azure_truncate_on_insert=1" ) azure_query(node, query) @@ -680,7 +710,7 @@ def test_schema_inference_from_globs_tf(cluster): query = ( f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " - f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1" ) azure_query(node, query) @@ -708,7 +738,7 @@ def test_partition_by_tf(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', " - f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", ) assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port) @@ -727,7 +757,7 @@ def test_filter_using_file(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " - f"'{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"'{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", ) query = ( @@ -745,7 +775,7 @@ def test_read_subcolumns(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," - f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS azure_truncate_on_insert=1", ) azure_query( @@ -795,7 +825,7 @@ def test_read_subcolumn_time(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumn_time.tsv', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," - f" 'a UInt32') select (42)", + f" 'a UInt32') select (42) SETTINGS azure_truncate_on_insert=1", ) res = node.query( @@ -825,7 +855,7 @@ def test_function_signatures(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3) SETTINGS azure_truncate_on_insert=1", ) # " - connection_string, container_name, blobpath\n" @@ -939,12 +969,12 @@ def test_union_schema_inference_mode(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1) SETTINGS azure_truncate_on_insert=1", ) azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2) SETTINGS azure_truncate_on_insert=1", ) node.query("system drop schema cache for azure") @@ -981,7 +1011,7 @@ def test_union_schema_inference_mode(cluster): assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error') SETTINGS azure_truncate_on_insert=1", ) error = azure_query( @@ -1003,7 +1033,7 @@ def test_schema_inference_cache(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') " - f"select * from numbers(100)", + f"select * from numbers(100) SETTINGS azure_truncate_on_insert=1", ) time.sleep(1) @@ -1210,19 +1240,19 @@ def test_filtering_by_file_or_path(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter1.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter2.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_filter3.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3 SETTINGS azure_truncate_on_insert=1", ) node.query( @@ -1246,19 +1276,19 @@ def test_size_virtual_column(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column1.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column2.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11 SETTINGS azure_truncate_on_insert=1", ) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_size_virtual_column3.tsv', 'devstoreaccount1', " - f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111", + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111 SETTINGS azure_truncate_on_insert=1", ) result = azure_query( @@ -1281,7 +1311,7 @@ def test_format_detection(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0) SETTINGS azure_truncate_on_insert=1", ) azure_query( @@ -1351,7 +1381,7 @@ def test_write_to_globbed_partitioned_path(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" error = azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_data_*_{{_partition_id}}', '{account_name}', '{account_key}', 'CSV', 'auto', 'x UInt64') partition by 42 select 42 SETTINGS azure_truncate_on_insert=1", expect_error="true", ) @@ -1475,7 +1505,7 @@ def test_hive_partitioning_with_one_parameter(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", ) query = ( @@ -1512,7 +1542,7 @@ def test_hive_partitioning_with_two_parameters(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", ) query = ( @@ -1558,7 +1588,7 @@ def test_hive_partitioning_without_setting(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", ) query = ( diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 6c5e2d20ca5..4d63016cf9a 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -71,7 +71,7 @@ def test_select_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1'," f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') " - f"VALUES (1, 'a'), (2, 'b')", + f"VALUES (1, 'a'), (2, 'b') SETTINGS azure_truncate_on_insert=1", ) print(get_azure_file_content("test_cluster_select_all.csv", port)) @@ -100,7 +100,7 @@ def test_count(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " - f"'auto', 'key UInt64') VALUES (1), (2)", + f"'auto', 'key UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) print(get_azure_file_content("test_cluster_count.csv", port)) @@ -128,7 +128,7 @@ def test_union_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_parquet_union_all', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', " - f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", + f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd') SETTINGS azure_truncate_on_insert=1", ) pure_azure = azure_query( @@ -179,7 +179,7 @@ def test_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2)", + f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) result = azure_query( node, @@ -199,7 +199,7 @@ def test_unset_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_unset_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2)", + f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) result = azure_query( node, @@ -217,7 +217,7 @@ def test_cluster_with_named_collection(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2)", + f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", ) pure_azure = azure_query( @@ -248,7 +248,7 @@ def test_partition_parallel_reading_with_cluster(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', '{filename}', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') " - f"PARTITION BY {partition_by} VALUES {values}", + f"PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", ) assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv", port) @@ -272,12 +272,12 @@ def test_format_detection(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10) SETTINGS azure_truncate_on_insert=1", ) azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10) SETTINGS azure_truncate_on_insert=1", ) expected_desc_result = azure_query( diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 79914855782..ca072f59e4b 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -41,6 +41,7 @@ def test_read_write_storage(started_cluster): node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" + node1.query("drop table if exists SimpleHDFSStorage") def test_read_write_storage_with_globs(started_cluster): @@ -94,6 +95,11 @@ def test_read_write_storage_with_globs(started_cluster): print(ex) assert "in readonly mode" in str(ex) + node1.query("DROP TABLE HDFSStorageWithRange") + node1.query("DROP TABLE HDFSStorageWithEnum") + node1.query("DROP TABLE HDFSStorageWithQuestionMark") + node1.query("DROP TABLE HDFSStorageWithAsterisk") + def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -335,6 +341,7 @@ def test_virtual_columns(started_cluster): ) == expected ) + node1.query("DROP TABLE virual_cols") def test_read_files_with_spaces(started_cluster): @@ -356,6 +363,7 @@ def test_read_files_with_spaces(started_cluster): ) assert node1.query("select * from test order by id") == "1\n2\n3\n" fs.delete(dir, recursive=True) + node1.query("DROP TABLE test") def test_truncate_table(started_cluster): @@ -427,7 +435,7 @@ def test_seekable_formats(started_cluster): f"hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String')" ) node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"SELECT count() FROM {table_function}") @@ -435,7 +443,7 @@ def test_seekable_formats(started_cluster): table_function = f"hdfs('hdfs://hdfs1:9000/orc', 'ORC', 'a Int32, b String')" node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"SELECT count() FROM {table_function}") assert int(result) == 5000000 @@ -459,7 +467,7 @@ def test_read_table_with_default(started_cluster): def test_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)" + f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/native', 'Native')") @@ -512,6 +520,7 @@ def test_hdfs_directory_not_exist(started_cluster): assert "" == node1.query( "select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1" ) + node1.query("DROP TABLE HDFSStorageWithNotExistDir") def test_overwrite(started_cluster): @@ -531,6 +540,7 @@ def test_overwrite(started_cluster): result = node1.query(f"select count() from test_overwrite") assert int(result) == 10 + node1.query(f"DROP TABLE test_overwrite") def test_multiple_inserts(started_cluster): @@ -567,6 +577,7 @@ def test_multiple_inserts(started_cluster): result = node1.query(f"select count() from test_multiple_inserts") assert int(result) == 60 + node1.query(f"DROP TABLE test_multiple_inserts") def test_format_detection(started_cluster): @@ -580,10 +591,10 @@ def test_format_detection(started_cluster): def test_schema_inference_with_globs(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" + f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" ) node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" + f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0 SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query( @@ -597,7 +608,7 @@ def test_schema_inference_with_globs(started_cluster): assert sorted(result.split()) == ["0", "\\N"] node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" + f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" ) filename = "data{1,3}.jsoncompacteachrow" @@ -609,7 +620,7 @@ def test_schema_inference_with_globs(started_cluster): assert "All attempts to extract table structure from files failed" in result node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" + f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]' SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query_and_get_error( @@ -621,7 +632,7 @@ def test_schema_inference_with_globs(started_cluster): def test_insert_select_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x" + f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x SETTINGS hdfs_truncate_on_insert=1" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/test.native.zst')") @@ -664,7 +675,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -672,7 +683,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" @@ -969,11 +980,11 @@ def test_read_subcolumns(started_cluster): node = started_cluster.instances["node1"] node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" ) node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" ) res = node.query( @@ -1019,11 +1030,11 @@ def test_union_schema_inference_mode(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a SETTINGS hdfs_truncate_on_insert=1" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b SETTINGS hdfs_truncate_on_insert=1" ) node.query("system drop schema cache for hdfs") @@ -1055,7 +1066,7 @@ def test_union_schema_inference_mode(started_cluster): ) assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error'" + f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error' SETTINGS hdfs_truncate_on_insert=1" ) error = node.query_and_get_error( @@ -1068,11 +1079,11 @@ def test_format_detection(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0)" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0) SETTINGS hdfs_truncate_on_insert=1" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10)" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10) SETTINGS hdfs_truncate_on_insert=1" ) expected_desc_result = node.query( @@ -1136,7 +1147,7 @@ def test_write_to_globbed_partitioned_path(started_cluster): node = started_cluster.instances["node1"] error = node.query_and_get_error( - "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42" + "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42 SETTINGS hdfs_truncate_on_insert=1" ) assert "DATABASE_ACCESS_DENIED" in error From 462fe8b7ffa98743ecc01c98eb0bfae840b05dca Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Aug 2024 18:23:56 +0200 Subject: [PATCH 0494/1722] repro for #67478 --- .../test_storage_policies/configs/disks.xml | 8 ++ .../integration/test_storage_policies/test.py | 79 +++++++++++++++++++ 2 files changed, 87 insertions(+) diff --git a/tests/integration/test_storage_policies/configs/disks.xml b/tests/integration/test_storage_policies/configs/disks.xml index 3331fee4e4f..dc60d93208c 100644 --- a/tests/integration/test_storage_policies/configs/disks.xml +++ b/tests/integration/test_storage_policies/configs/disks.xml @@ -1,4 +1,12 @@ + + system + blob_storage_log
+ toYYYYMM(event_date) + 7500 + event_date + INTERVAL 30 DAY +
+ diff --git a/tests/integration/test_storage_policies/test.py b/tests/integration/test_storage_policies/test.py index 389146b2171..08a5cc71b99 100644 --- a/tests/integration/test_storage_policies/test.py +++ b/tests/integration/test_storage_policies/test.py @@ -38,3 +38,82 @@ def test_storage_policy_configuration_change(started_cluster): "/etc/clickhouse-server/config.d/disks.xml", ) node.start_clickhouse() + + +def test_disk_is_immutable(started_cluster): + node.query("DROP TABLE IF EXISTS test_1") + + node.query( + """ + create table test_1 (a Int32) + engine = MergeTree() + order by tuple() + settings + disk=disk( + name='not_uniq_disk_name', + type = object_storage, + object_storage_type = local_blob_storage, + path='./03215_data_test_1/') + """ + ) + + node.query("INSERT INTO test_1 VALUES (1)") + node.query("SYSTEM FLUSH LOGS;") + + print( + node.query( + "SELECT 'test_1', * FROM system.blob_storage_log" + ) + ) + + print( + node.query( + "SELECT 'test_1', * FROM test_1" + ) + ) + + node.query("DROP TABLE test_1 SYNC") + node.query("DROP TABLE IF EXISTS test_2") + + node.query( + """ + create table test_2 (a Int32) + engine = MergeTree() + order by tuple() + settings + disk=disk( + name='not_uniq_disk_name', + type = object_storage, + object_storage_type = local_blob_storage, + path='./03215_data_test_2/') + """ + ) + + node.query("INSERT INTO test_2 VALUES (1)") + node.query("SYSTEM FLUSH LOGS;") + + print( + node.query( + "SELECT 'test_2', * FROM system.blob_storage_log" + ) + ) + + print( + node.query( + "SELECT 'test_2', * FROM test_2" + ) + ) + + node.restart_clickhouse() + + print( + node.query( + "SELECT 'test_2', * FROM system.blob_storage_log" + ) + ) + + print( + node.query( + "SELECT 'test_2', * FROM test_2" + ) + ) From 5d9d5bf919527822b4a3d42e4f1595ccd374ead9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Aug 2024 18:14:22 +0000 Subject: [PATCH 0495/1722] Automatic style fix --- .../integration/test_storage_policies/test.py | 36 ++++--------------- 1 file changed, 6 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_storage_policies/test.py b/tests/integration/test_storage_policies/test.py index 08a5cc71b99..f65096389af 100644 --- a/tests/integration/test_storage_policies/test.py +++ b/tests/integration/test_storage_policies/test.py @@ -60,17 +60,9 @@ def test_disk_is_immutable(started_cluster): node.query("INSERT INTO test_1 VALUES (1)") node.query("SYSTEM FLUSH LOGS;") - print( - node.query( - "SELECT 'test_1', * FROM system.blob_storage_log" - ) - ) + print(node.query("SELECT 'test_1', * FROM system.blob_storage_log")) - print( - node.query( - "SELECT 'test_1', * FROM test_1" - ) - ) + print(node.query("SELECT 'test_1', * FROM test_1")) node.query("DROP TABLE test_1 SYNC") node.query("DROP TABLE IF EXISTS test_2") @@ -92,28 +84,12 @@ def test_disk_is_immutable(started_cluster): node.query("INSERT INTO test_2 VALUES (1)") node.query("SYSTEM FLUSH LOGS;") - print( - node.query( - "SELECT 'test_2', * FROM system.blob_storage_log" - ) - ) + print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - print( - node.query( - "SELECT 'test_2', * FROM test_2" - ) - ) + print(node.query("SELECT 'test_2', * FROM test_2")) node.restart_clickhouse() - print( - node.query( - "SELECT 'test_2', * FROM system.blob_storage_log" - ) - ) + print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - print( - node.query( - "SELECT 'test_2', * FROM test_2" - ) - ) + print(node.query("SELECT 'test_2', * FROM test_2")) From d080f863ea41420ecbb1c5d65769d74e21a46aba Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:07:17 +0200 Subject: [PATCH 0496/1722] fix black --- tests/integration/test_storage_hdfs/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9ee8ac4cdfd..856715f28c8 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -675,7 +675,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -683,7 +685,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" From d8aa219783b6715f5424772bf25c092a26be5e2d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:43:29 +0200 Subject: [PATCH 0497/1722] fix build --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 03e0f4f0dc8..2080e8fbf0d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -77,7 +77,7 @@ static std::initializer_list Date: Mon, 5 Aug 2024 18:31:16 +0000 Subject: [PATCH 0498/1722] set max_threads --- .../queries/0_stateless/03161_lightweight_delete_projection.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 28e5612a529..618f3ac0cb8 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,5 +1,5 @@ -SET lightweight_deletes_sync = 2, alter_sync = 2; +SET max_threads = 1, lightweight_deletes_sync = 2, alter_sync = 2; DROP TABLE IF EXISTS users_compact; From cb6baefa948ef1270ce9454f72075ac10bf6e729 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 5 Aug 2024 21:49:36 +0100 Subject: [PATCH 0499/1722] better --- src/Common/ShellCommand.cpp | 2 +- src/Processors/Sources/ShellCommandSource.cpp | 12 +++--------- tests/integration/test_executable_dictionary/test.py | 2 -- .../test_executable_table_function/test.py | 1 - 4 files changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 79b0d667863..0d41669816c 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -239,7 +239,7 @@ std::unique_ptr ShellCommand::executeImpl( LOG_TRACE( getLogger(), - "Started shell command '{}' with pid {} and file descriptors: read {}, write {}", + "Started shell command '{}' with pid {} and file descriptors: out {}, err {}", filename, pid, res->out.getFD(), diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 923bdfad8f8..23359367a9b 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -76,13 +76,11 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond { Stopwatch watch; -#if defined(DEBUG_OR_SANITIZER_BUILD) auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; - LOG_TRACE( + LOG_TEST( getLogger("TimeoutReadBufferFromFileDescriptor"), "Polling descriptors: {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); -#endif res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); @@ -94,13 +92,11 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond const auto elapsed = watch.elapsedMilliseconds(); if (timeout_milliseconds <= elapsed) { -#if defined(DEBUG_OR_SANITIZER_BUILD) - LOG_TRACE( + LOG_TEST( getLogger("TimeoutReadBufferFromFileDescriptor"), "Timeout exceeded: elapsed={}, timeout={}", elapsed, timeout_milliseconds); -#endif break; } timeout_milliseconds -= elapsed; @@ -111,14 +107,12 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond } } -#if defined(DEBUG_OR_SANITIZER_BUILD) auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; - LOG_TRACE( + LOG_TEST( getLogger("TimeoutReadBufferFromFileDescriptor"), "Poll for descriptors: {} returned {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", "), res); -#endif return res; } diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index 2a6af75e751..22f3442bb95 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -235,7 +235,6 @@ def test_executable_implicit_input_signalled_python(started_cluster): ) -@pytest.mark.repeat(50) def test_executable_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( @@ -246,7 +245,6 @@ def test_executable_input_slow_python(started_cluster): ) -@pytest.mark.repeat(50) def test_executable_implicit_input_slow_python(started_cluster): skip_test_msan(node) assert node.query_and_get_error( diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index a79616fc008..801a3c7c14a 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -139,7 +139,6 @@ def test_executable_function_input_signalled_python(started_cluster): assert node.query(query.format(source="(SELECT id FROM test_data_table)")) == "" -@pytest.mark.repeat(50) def test_executable_function_input_slow_python(started_cluster): skip_test_msan(node) From 1b251fe08837aee56fe541bf3ef54ff647650869 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 21:10:31 +0000 Subject: [PATCH 0500/1722] Allow to specify min and max for random settings in the test --- docs/en/development/tests.md | 22 +++++ tests/clickhouse-test | 92 ++++++++++++++++--- ...mic_read_subcolumns_compact_merge_tree.sql | 1 + ...ynamic_read_subcolumns_wide_merge_tree.sql | 1 + ...merges_1_horizontal_compact_merge_tree.sql | 4 + ..._merges_1_horizontal_compact_wide_tree.sql | 2 + ...c_merges_1_vertical_compact_merge_tree.sql | 2 + ...amic_merges_1_vertical_wide_merge_tree.sql | 2 + ...merges_2_horizontal_compact_merge_tree.sql | 1 + ...ic_merges_2_horizontal_wide_merge_tree.sql | 1 + ...c_merges_2_vertical_compact_merge_tree.sql | 1 + ...amic_merges_2_vertical_wide_merge_tree.sql | 1 + ...sted_dynamic_merges_compact_horizontal.sql | 1 + ...nested_dynamic_merges_compact_vertical.sql | 1 + ..._nested_dynamic_merges_wide_horizontal.sql | 1 + ...38_nested_dynamic_merges_wide_vertical.sql | 1 + 16 files changed, 122 insertions(+), 12 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 269995a1a96..f0afa983fec 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -91,6 +91,28 @@ SELECT 1 In addition to the above settings, you can use `USE_*` flags from `system.build_options` to define usage of particular ClickHouse features. For example, if your test uses a MySQL table, you should add a tag `use-mysql`. +### Specifying limits for random settings + +A test can specify minimum and maximum allowed values for settings that can be randomized during test run. + +For `.sh` tests limits are written as a comment on the line next to tags or on the second line if no tags are specified: + +```bash +#!/usr/bin/env bash +# Tags: no-fasttest +# Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +``` + +For `.sql` tests tags are placed as a SQL comment in the line next to tags or in the first line: + +```sql +-- Tags: no-fasttest +-- Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +SELECT 1 +``` + +If you need to specify only one limit, you can use `None` for another one. + ### Choosing the Test Name The name of the test starts with a five-digit prefix followed by a descriptive name, such as `00422_hash_function_constexpr.sql`. To choose the prefix, find the largest prefix already present in the directory, and increment it by one. In the meantime, some other tests might be added with the same numeric prefix, but this is OK and does not lead to any problems, you don't have to change it later. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a29c786e998..ea488e7c3dd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -39,6 +39,7 @@ from errno import ESRCH from subprocess import PIPE, Popen from time import sleep, time from typing import Dict, List, Optional, Set, Tuple, Union +from ast import literal_eval as make_tuple try: import termcolor # type: ignore @@ -1068,9 +1069,25 @@ class TestCase: return description + "\n" + def apply_random_settings_limits(self, random_settings): + print("Random settings limits:", self.random_settings_limits) + for setting in random_settings: + if setting in self.random_settings_limits: + min = self.random_settings_limits[setting][0] + if min and random_settings[setting] < min: + random_settings[setting] = min + max = self.random_settings_limits[setting][1] + if max and random_settings[setting] > max: + random_settings[setting] = max + def __init__(self, suite, case: str, args, is_concurrent: bool): self.case: str = case # case file name self.tags: Set[str] = suite.all_tags[case] if case in suite.all_tags else set() + self.random_settings_limits = ( + suite.all_random_settings_limits[case] + if case in suite.all_random_settings_limits + else dict() + ) for tag in os.getenv("GLOBAL_TAGS", "").split(","): self.tags.add(tag.strip()) @@ -1112,11 +1129,13 @@ class TestCase: if self.randomize_settings: self.random_settings = SettingsRandomizer.get_random_settings(args) + self.apply_random_settings_limits(self.random_settings) if self.randomize_merge_tree_settings: self.merge_tree_random_settings = ( MergeTreeSettingsRandomizer.get_random_settings(args) ) + self.apply_random_settings_limits(self.merge_tree_random_settings) self.base_url_params = ( os.environ["CLICKHOUSE_URL_PARAMS"] @@ -1900,7 +1919,9 @@ class TestSuite: return test_name @staticmethod - def read_test_tags(suite_dir: str, all_tests: List[str]) -> Dict[str, Set[str]]: + def read_test_tags_and_random_settings_limits( + suite_dir: str, all_tests: List[str] + ) -> (Dict[str, Set[str]], Dict[str, Dict[str, Tuple[int, int]]]): def get_comment_sign(filename): if filename.endswith(".sql") or filename.endswith(".sql.j2"): return "--" @@ -1925,22 +1946,48 @@ class TestSuite: tags = {tag.strip() for tag in tags} return tags + def parse_random_settings_limits_from_line( + line, comment_sign + ) -> Dict[str, Tuple[int, int]]: + if not line.startswith(comment_sign): + return {} + random_settings_limits_str = line[ + len(comment_sign) : + ].lstrip() # noqa: ignore E203 + random_settings_limits_prefix = "Random settings limits:" + if not random_settings_limits_str.startswith(random_settings_limits_prefix): + return {} + random_settings_limits_str = random_settings_limits_str[ + len(random_settings_limits_prefix) : + ] # noqa: ignore E203 + # limits are specified in a form 'setting1=(min, max), setting2=(min,max), ...' + random_settings_limits = re.findall( + "([^=, ]+) *= *(\([^=]+\))", random_settings_limits_str + ) + random_settings_limits = { + pair[0]: make_tuple(pair[1]) for pair in random_settings_limits + } + return random_settings_limits + def is_shebang(line: str) -> bool: return line.startswith("#!") def find_tag_line(file): - for line in file: + line = file.readline() + while line != "": line = line.strip() if line and not is_shebang(line): return line + line = file.readline() return "" - def load_tags_from_file(filepath): + def load_tags_and_random_settings_limits_from_file(filepath): comment_sign = get_comment_sign(filepath) need_query_params = False with open(filepath, "r", encoding="utf-8") as file: try: tag_line = find_tag_line(file) + next_line = file.readline() except UnicodeDecodeError: return [] try: @@ -1950,21 +1997,35 @@ class TestSuite: need_query_params = True except UnicodeDecodeError: pass - parsed_tags = parse_tags_from_line(tag_line, comment_sign) - if need_query_params: - parsed_tags.add("need-query-parameters") - return parsed_tags + parsed_tags = parse_tags_from_line(tag_line, comment_sign) + if need_query_params: + parsed_tags.add("need-query-parameters") + random_settings_limits_line = next_line if parsed_tags else tag_line + random_settings_limits = parse_random_settings_limits_from_line( + random_settings_limits_line, comment_sign + ) + return parsed_tags, random_settings_limits all_tags = {} + all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - tags = load_tags_from_file(os.path.join(suite_dir, test_name)) + ( + tags, + random_settings_limits, + ) = load_tags_and_random_settings_limits_from_file( + os.path.join(suite_dir, test_name) + ) if tags: all_tags[test_name] = tags + if random_settings_limits: + all_random_settings_limits[test_name] = random_settings_limits elapsed = (datetime.now() - start_time).total_seconds() if elapsed > 1: - print(f"Tags for suite {suite_dir} read in {elapsed:.2f} seconds") - return all_tags + print( + f"Tags and random settings limits for suite {suite_dir} read in {elapsed:.2f} seconds" + ) + return all_tags, all_random_settings_limits def __init__(self, args, suite_path: str, suite_tmp_path: str, suite: str): self.args = args @@ -1994,9 +2055,16 @@ class TestSuite: self.all_tests: List[str] = self.get_tests_list( self.tests_in_suite_key_func, filter_func ) - self.all_tags: Dict[str, Set[str]] = self.read_test_tags( - self.suite_path, self.all_tests + + all_tags_and_random_settings_limits = ( + self.read_test_tags_and_random_settings_limits( + self.suite_path, self.all_tests + ) ) + self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql index ddfba4418bd..822393d3c78 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_compact_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql index 5aac5f7b72f..2394893dc8b 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns_wide_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index d2c787040e5..7c2e7c3d2be 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; @@ -31,3 +33,5 @@ optimize table test final; select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); drop table test; + +select 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql index f99bf771608..aa62435188a 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql index be81596d043..bfc7bb9d206 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql index f6396af42a8..233667db0a7 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -1,4 +1,6 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) + set allow_experimental_dynamic_type=1; drop table if exists test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index e133ac3001f..48a6a55378c 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index d527081b763..44b298b1c35 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index ebccfb77922..f42150720b3 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index 104d6018e41..ee4ff6af162 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -1,4 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 1d5c63dcdf1..e0636f053df 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index 2bffe35c577..edfad295e9a 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index fb686091ebb..79d488ec253 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index ed195452d56..e2a453b867a 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -1,4 +1,5 @@ -- Tags: long +-- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; From d3dc17453377368defd80cda9f4b95dda6adc9df Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 21:15:11 +0000 Subject: [PATCH 0501/1722] Remove log --- tests/clickhouse-test | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 389193836bf..5fcb9fb80f1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -682,7 +682,6 @@ class FailureReason(enum.Enum): BUILD = "not running for current build" NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" SHARED_MERGE_TREE = "no-shared-merge-tree" - DISTRIBUTED_CACHE = "distributed-cache" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -1071,7 +1070,6 @@ class TestCase: return description + "\n" def apply_random_settings_limits(self, random_settings): - print("Random settings limits:", self.random_settings_limits) for setting in random_settings: if setting in self.random_settings_limits: min = self.random_settings_limits[setting][0] @@ -1211,9 +1209,6 @@ class TestCase: elif tags and ("no-replicated-database" in tags) and args.replicated_database: return FailureReason.REPLICATED_DB - elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: - return FailureReason.DISTRIBUTED_CACHE - elif ( tags and ("atomic-database" in tags) @@ -1251,11 +1246,6 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and ( - 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) - ): - return FailureReason.SKIP - elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: @@ -2295,6 +2285,7 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool args, test_suite, client_options, server_logs_level ) test_result = test_case.process_result(test_result, MESSAGES) + break except TimeoutError: break finally: @@ -3280,12 +3271,6 @@ def parse_args(): default=False, help="Run tests over s3 storage", ) - parser.add_argument( - "--distributed-cache", - action="store_true", - default=False, - help="Run tests with enabled distributed cache", - ) parser.add_argument( "--azure-blob-storage", action="store_true", From 18a7a82458ce7ec3f12f7b6751699f119769ed55 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Aug 2024 21:16:18 +0000 Subject: [PATCH 0502/1722] Better formatting --- tests/clickhouse-test | 33 +++++++++++---------------------- 1 file changed, 11 insertions(+), 22 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5fcb9fb80f1..bcb8a12625b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1950,15 +1950,11 @@ class TestSuite: ) -> Dict[str, Tuple[int, int]]: if not line.startswith(comment_sign): return {} - random_settings_limits_str = line[ - len(comment_sign) : - ].lstrip() # noqa: ignore E203 + random_settings_limits_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 random_settings_limits_prefix = "Random settings limits:" if not random_settings_limits_str.startswith(random_settings_limits_prefix): return {} - random_settings_limits_str = random_settings_limits_str[ - len(random_settings_limits_prefix) : - ] # noqa: ignore E203 + random_settings_limits_str = random_settings_limits_str[len(random_settings_limits_prefix) :] # noqa: ignore E203 # limits are specified in a form 'setting1=(min, max), setting2=(min,max), ...' random_settings_limits = re.findall( "([^=, ]+) *= *(\([^=]+\))", random_settings_limits_str @@ -1996,25 +1992,20 @@ class TestSuite: need_query_params = True except UnicodeDecodeError: pass - parsed_tags = parse_tags_from_line(tag_line, comment_sign) - if need_query_params: - parsed_tags.add("need-query-parameters") - random_settings_limits_line = next_line if parsed_tags else tag_line - random_settings_limits = parse_random_settings_limits_from_line( - random_settings_limits_line, comment_sign - ) + parsed_tags = parse_tags_from_line(tag_line, comment_sign) + if need_query_params: + parsed_tags.add("need-query-parameters") + random_settings_limits_line = next_line if parsed_tags else tag_line + random_settings_limits = parse_random_settings_limits_from_line( + random_settings_limits_line, comment_sign + ) return parsed_tags, random_settings_limits all_tags = {} all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - ( - tags, - random_settings_limits, - ) = load_tags_and_random_settings_limits_from_file( - os.path.join(suite_dir, test_name) - ) + tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 if tags: all_tags[test_name] = tags if random_settings_limits: @@ -2061,9 +2052,7 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 self.sequential_tests = [] self.parallel_tests = [] From 67556a594bb12b944ce4c94cacf9164e92965dfe Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 5 Aug 2024 23:07:53 +0000 Subject: [PATCH 0503/1722] Conflict --- src/Core/SettingsChangesHistory.cpp | 263 +--------------------------- 1 file changed, 1 insertion(+), 262 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8a40ad94645..dce51b30382 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,268 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"allow_materialized_view_with_bad_select", true, false, "Stricter validation in CREATE MATERIALIZED VIEW"}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"restore_replace_external_table_functions_to_null", false, false, "New setting."}, - {"restore_replace_external_engines_to_null", false, false, "New setting."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -340,6 +78,7 @@ static std::initializer_list Date: Tue, 6 Aug 2024 09:56:38 +0800 Subject: [PATCH 0504/1722] Fix docs --- .../mergetree-family/mergetree.md | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 8fd60c36f94..80e5bf694d4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -994,30 +994,36 @@ They can be used for prewhere optimization only if we enable `set allow_statisti - `MinMax` The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `Uniq` [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) sketches which provide an estimation how many distinct values a column contains. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. - `count_min` [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. -#### Supported Data Types {#supported-data-types} +Note that all statistics types support `LowCardinality` and `Nullable` modifiers to data types. -| Data Type / Statistics | count_min | minmax | tdigest | uniq | -|-------------------------|-----------|--------|---------|------| -| Integer types | ✔ | ✔ | ✔ | ✔ | -| Floating-point numbers | ✔ | ✔ | ✔ | ✔ | -| Boolean | ✔ | ✔ | ✔ | ✔ | -| Strings | ✔ | ✗ | ✗ | ✔ | -| Dates | ✔ | ✔ | ✔ | ✔ | +#### Supported operations of Column Statistics {#supported-operations-of-column-statistics} + +| | Equals | Range | +|-----------|---------|-------| +| count_min | ✔ | ✔ | +| MinMax | ✔ | ✔ | +| TDigest | ✔ | ✔ | +| Uniq | ✔ | ✗ | + +Please note that operation range represents >, >=, <, <=. -You can also add `LowCardinality` and `Nullable` modifiers to the data types. ## Column-level Settings {#column-level-settings} From 46da03030cd6ae115edb7d4b9054704a8312283c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 6 Aug 2024 10:00:34 +0800 Subject: [PATCH 0505/1722] Add test for implicitly type conversion --- .../table-engines/mergetree-family/mergetree.md | 10 +++++++--- .../0_stateless/02864_statistics_estimation.sql | 6 ++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 80e5bf694d4..caf72e39467 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -994,22 +994,26 @@ They can be used for prewhere optimization only if we enable `set allow_statisti - `MinMax` The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `Uniq` [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) sketches which provide an estimation how many distinct values a column contains. - Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. + + Supported data types: (U)Int*, Float*, Decimal*, Boolean, Date* and (Fixed)String. - `count_min` [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. - Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. + + Supported data types: (U)Int*, Float*, Decimal*, Boolean, Date* and (Fixed)String. Note that all statistics types support `LowCardinality` and `Nullable` modifiers to data types. @@ -1022,7 +1026,7 @@ Note that all statistics types support `LowCardinality` and `Nullable` modifiers | TDigest | ✔ | ✔ | | Uniq | ✔ | ✗ | -Please note that operation range represents >, >=, <, <=. +Please note that operation `Range` represents >, >=, < or <=. diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 65b111c285b..4a221686069 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -80,6 +80,12 @@ SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTi FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'Test statistics implicitly type conversion:'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '10000'/*0*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + ALTER TABLE tab DROP STATISTICS a, b, c, d; From 8259a9827e5f1764fffa508f8e721f643c102d15 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 6 Aug 2024 10:29:42 +0800 Subject: [PATCH 0506/1722] Update reference file for tests --- .../engines/table-engines/mergetree-family/mergetree.md | 9 ++++----- .../0_stateless/02864_statistics_estimation.reference | 4 ++++ 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index caf72e39467..751b07afe91 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1021,18 +1021,17 @@ Note that all statistics types support `LowCardinality` and `Nullable` modifiers | | Equals | Range | |-----------|---------|-------| -| count_min | ✔ | ✔ | -| MinMax | ✔ | ✔ | -| TDigest | ✔ | ✔ | +| count_min | ✔ | ✗ | +| MinMax | ✗ | ✔ | +| TDigest | ✗ | ✔ | | Uniq | ✔ | ✗ | Please note that operation `Range` represents >, >=, < or <=. - ## Column-level Settings {#column-level-settings} -Certain MergeTree settings can be override at column level: +Certain MergeTree settings can be overridden at column level: - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index f59f94013da..cb274b2990e 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -22,3 +22,7 @@ Test statistics multi-types: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) +Test statistics implicitly type conversion: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) From 4a0300736db6ec2898ffdf82fa574b3466e5f44e Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 6 Aug 2024 12:46:22 +0800 Subject: [PATCH 0507/1722] empty commit From 222079e0323378ee56da7f42dca53e9fc5b3c2f1 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 6 Aug 2024 12:48:52 +0800 Subject: [PATCH 0508/1722] empty commit From e6f566e49d78080a954ca992d8d5e0f5fb1bb1e2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 6 Aug 2024 13:23:12 +0800 Subject: [PATCH 0509/1722] Small refactors in ORC output format --- .../Formats/Impl/ORCBlockOutputFormat.cpp | 99 +++++++------------ 1 file changed, 33 insertions(+), 66 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 6f543a05fba..bd89ae0fa86 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -203,25 +204,15 @@ template void ORCBlockOutputFormat::writeNumbers( orc::ColumnVectorBatch & orc_column, const IColumn & column, - const PaddedPODArray * null_bytemap, + const PaddedPODArray * /*null_bytemap*/, ConvertFunc convert) { NumberVectorBatch & number_orc_column = dynamic_cast(orc_column); const auto & number_column = assert_cast &>(column); - number_orc_column.resize(number_column.size()); + number_orc_column.data.resize(number_column.size()); for (size_t i = 0; i != number_column.size(); ++i) - { - if (null_bytemap && (*null_bytemap)[i]) - { - number_orc_column.notNull[i] = 0; - continue; - } - - number_orc_column.notNull[i] = 1; number_orc_column.data[i] = convert(number_column.getElement(i)); - } - number_orc_column.numElements = number_column.size(); } template @@ -229,7 +220,7 @@ void ORCBlockOutputFormat::writeDecimals( orc::ColumnVectorBatch & orc_column, const IColumn & column, DataTypePtr & type, - const PaddedPODArray * null_bytemap, + const PaddedPODArray * /*null_bytemap*/, ConvertFunc convert) { DecimalVectorBatch & decimal_orc_column = dynamic_cast(orc_column); @@ -238,71 +229,49 @@ void ORCBlockOutputFormat::writeDecimals( decimal_orc_column.precision = decimal_type->getPrecision(); decimal_orc_column.scale = decimal_type->getScale(); decimal_orc_column.resize(decimal_column.size()); - for (size_t i = 0; i != decimal_column.size(); ++i) - { - if (null_bytemap && (*null_bytemap)[i]) - { - decimal_orc_column.notNull[i] = 0; - continue; - } - decimal_orc_column.notNull[i] = 1; + decimal_orc_column.values.resize(decimal_column.size()); + for (size_t i = 0; i != decimal_column.size(); ++i) decimal_orc_column.values[i] = convert(decimal_column.getElement(i).value); - } - decimal_orc_column.numElements = decimal_column.size(); } template void ORCBlockOutputFormat::writeStrings( orc::ColumnVectorBatch & orc_column, const IColumn & column, - const PaddedPODArray * null_bytemap) + const PaddedPODArray * /*null_bytemap*/) { orc::StringVectorBatch & string_orc_column = dynamic_cast(orc_column); const auto & string_column = assert_cast(column); - string_orc_column.resize(string_column.size()); + string_orc_column.data.resize(string_column.size()); + string_orc_column.length.resize(string_column.size()); for (size_t i = 0; i != string_column.size(); ++i) { - if (null_bytemap && (*null_bytemap)[i]) - { - string_orc_column.notNull[i] = 0; - continue; - } - - string_orc_column.notNull[i] = 1; const std::string_view & string = string_column.getDataAt(i).toView(); string_orc_column.data[i] = const_cast(string.data()); string_orc_column.length[i] = string.size(); } - string_orc_column.numElements = string_column.size(); } template void ORCBlockOutputFormat::writeDateTimes( orc::ColumnVectorBatch & orc_column, const IColumn & column, - const PaddedPODArray * null_bytemap, + const PaddedPODArray * /*null_bytemap*/, GetSecondsFunc get_seconds, GetNanosecondsFunc get_nanoseconds) { orc::TimestampVectorBatch & timestamp_orc_column = dynamic_cast(orc_column); const auto & timestamp_column = assert_cast(column); - timestamp_orc_column.resize(timestamp_column.size()); + timestamp_orc_column.data.resize(timestamp_column.size()); + timestamp_orc_column.nanoseconds.resize(timestamp_column.size()); for (size_t i = 0; i != timestamp_column.size(); ++i) { - if (null_bytemap && (*null_bytemap)[i]) - { - timestamp_orc_column.notNull[i] = 0; - continue; - } - - timestamp_orc_column.notNull[i] = 1; timestamp_orc_column.data[i] = static_cast(get_seconds(timestamp_column.getElement(i))); timestamp_orc_column.nanoseconds[i] = static_cast(get_nanoseconds(timestamp_column.getElement(i))); } - timestamp_orc_column.numElements = timestamp_column.size(); } void ORCBlockOutputFormat::writeColumn( @@ -311,9 +280,19 @@ void ORCBlockOutputFormat::writeColumn( DataTypePtr & type, const PaddedPODArray * null_bytemap) { - orc_column.notNull.resize(column.size()); + orc_column.numElements = column.size(); if (null_bytemap) - orc_column.hasNulls = true; + { + orc_column.hasNulls = !memoryIsZero(null_bytemap->data(), 0, null_bytemap->size()); + if (orc_column.hasNulls) + { + orc_column.notNull.resize(null_bytemap->size()); + for (size_t i = 0; i < null_bytemap->size(); ++i) + orc_column.notNull[i] = !(*null_bytemap)[i]; + } + } + else + orc_column.hasNulls = false; /// ORC doesn't have unsigned types, so cast everything to signed and sign-extend to Int64 to /// make the ORC library calculate min and max correctly. @@ -471,6 +450,7 @@ void ORCBlockOutputFormat::writeColumn( } case TypeIndex::Nullable: { + chassert(!null_bytemap); const auto & nullable_column = assert_cast(column); const PaddedPODArray & new_null_bytemap = assert_cast &>(*nullable_column.getNullMapColumnPtr()).getData(); auto nested_type = removeNullable(type); @@ -485,19 +465,15 @@ void ORCBlockOutputFormat::writeColumn( const ColumnArray::Offsets & offsets = list_column.getOffsets(); size_t column_size = list_column.size(); - list_orc_column.resize(column_size); + list_orc_column.offsets.resize(column_size + 1); /// The length of list i in ListVectorBatch is offsets[i+1] - offsets[i]. list_orc_column.offsets[0] = 0; for (size_t i = 0; i != column_size; ++i) - { list_orc_column.offsets[i + 1] = offsets[i]; - list_orc_column.notNull[i] = 1; - } orc::ColumnVectorBatch & nested_orc_column = *list_orc_column.elements; - writeColumn(nested_orc_column, list_column.getData(), nested_type, null_bytemap); - list_orc_column.numElements = column_size; + writeColumn(nested_orc_column, list_column.getData(), nested_type, nullptr); break; } case TypeIndex::Tuple: @@ -505,10 +481,8 @@ void ORCBlockOutputFormat::writeColumn( orc::StructVectorBatch & struct_orc_column = dynamic_cast(orc_column); const auto & tuple_column = assert_cast(column); auto nested_types = assert_cast(type.get())->getElements(); - for (size_t i = 0; i != tuple_column.size(); ++i) - struct_orc_column.notNull[i] = 1; for (size_t i = 0; i != tuple_column.tupleSize(); ++i) - writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], null_bytemap); + writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], nullptr); break; } case TypeIndex::Map: @@ -520,25 +494,21 @@ void ORCBlockOutputFormat::writeColumn( size_t column_size = list_column.size(); - map_orc_column.resize(list_column.size()); + map_orc_column.offsets.resize(column_size + 1); /// The length of list i in ListVectorBatch is offsets[i+1] - offsets[i]. map_orc_column.offsets[0] = 0; for (size_t i = 0; i != column_size; ++i) - { map_orc_column.offsets[i + 1] = offsets[i]; - map_orc_column.notNull[i] = 1; - } + const auto nested_columns = assert_cast(list_column.getDataPtr().get())->getColumns(); orc::ColumnVectorBatch & keys_orc_column = *map_orc_column.keys; auto key_type = map_type.getKeyType(); - writeColumn(keys_orc_column, *nested_columns[0], key_type, null_bytemap); + writeColumn(keys_orc_column, *nested_columns[0], key_type, nullptr); orc::ColumnVectorBatch & values_orc_column = *map_orc_column.elements; auto value_type = map_type.getValueType(); - writeColumn(values_orc_column, *nested_columns[1], value_type, null_bytemap); - - map_orc_column.numElements = column_size; + writeColumn(values_orc_column, *nested_columns[1], value_type, nullptr); break; } default: @@ -575,10 +545,7 @@ void ORCBlockOutputFormat::consume(Chunk chunk) size_t columns_num = chunk.getNumColumns(); size_t rows_num = chunk.getNumRows(); - /// getMaxColumnSize is needed to write arrays. - /// The size of the batch must be no less than total amount of array elements - /// and no less than the number of rows (ORC writes a null bit for every row). - std::unique_ptr batch = writer->createRowBatch(getMaxColumnSize(chunk)); + std::unique_ptr batch = writer->createRowBatch(chunk.getNumRows()); orc::StructVectorBatch & root = dynamic_cast(*batch); auto columns = chunk.detachColumns(); From 74a2976810b86086819ee8e6ee1f110ab1e70a37 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 08:13:03 +0000 Subject: [PATCH 0510/1722] Fix pylint --- tests/clickhouse-test | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index bcb8a12625b..84f33860484 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1072,12 +1072,12 @@ class TestCase: def apply_random_settings_limits(self, random_settings): for setting in random_settings: if setting in self.random_settings_limits: - min = self.random_settings_limits[setting][0] - if min and random_settings[setting] < min: - random_settings[setting] = min - max = self.random_settings_limits[setting][1] - if max and random_settings[setting] > max: - random_settings[setting] = max + min_value = self.random_settings_limits[setting][0] + if min_value and random_settings[setting] < min_value: + random_settings[setting] = min_value + max_value = self.random_settings_limits[setting][1] + if max_value and random_settings[setting] > max_value: + random_settings[setting] = max_value def __init__(self, suite, case: str, args, is_concurrent: bool): self.case: str = case # case file name @@ -2005,7 +2005,7 @@ class TestSuite: all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 + tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 if tags: all_tags[test_name] = tags if random_settings_limits: @@ -2052,7 +2052,7 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 self.sequential_tests = [] self.parallel_tests = [] From 5226792b1d8b4e110c63a813fb68c9dd65ea07b7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 08:48:06 +0000 Subject: [PATCH 0511/1722] Fix bad merge with master --- tests/clickhouse-test | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 84f33860484..c4124982442 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1163,6 +1163,9 @@ class TestCase: elif args.cloud and ("no-replicated-database" in tags): return FailureReason.REPLICATED_DB + elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: + return FailureReason.DISTRIBUTED_CACHE + elif args.cloud and self.name in suite.cloud_skip_list: return FailureReason.NOT_SUPPORTED_IN_CLOUD @@ -1246,6 +1249,11 @@ class TestCase: ): return FailureReason.SKIP + elif "no-flaky-check" in tags and ( + 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) + ): + return FailureReason.SKIP + elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: @@ -2274,7 +2282,6 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool args, test_suite, client_options, server_logs_level ) test_result = test_case.process_result(test_result, MESSAGES) - break except TimeoutError: break finally: @@ -3260,6 +3267,12 @@ def parse_args(): default=False, help="Run tests over s3 storage", ) + parser.add_argument( + "--distributed-cache", + action="store_true", + default=False, + help="Run tests with enabled distributed cache", + ) parser.add_argument( "--azure-blob-storage", action="store_true", From bb33dca38470aba044da06938cc96ca55166262d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 08:49:08 +0000 Subject: [PATCH 0512/1722] Fix unrelated changes --- tests/clickhouse-test | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c4124982442..72136404796 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -682,6 +682,7 @@ class FailureReason(enum.Enum): BUILD = "not running for current build" NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" SHARED_MERGE_TREE = "no-shared-merge-tree" + DISTRIBUTED_CACHE = "distributed-cache" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -1163,9 +1164,6 @@ class TestCase: elif args.cloud and ("no-replicated-database" in tags): return FailureReason.REPLICATED_DB - elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: - return FailureReason.DISTRIBUTED_CACHE - elif args.cloud and self.name in suite.cloud_skip_list: return FailureReason.NOT_SUPPORTED_IN_CLOUD @@ -1212,6 +1210,9 @@ class TestCase: elif tags and ("no-replicated-database" in tags) and args.replicated_database: return FailureReason.REPLICATED_DB + elif tags and ("no-distributed-cache" in tags) and args.distributed_cache: + return FailureReason.DISTRIBUTED_CACHE + elif ( tags and ("atomic-database" in tags) @@ -1250,7 +1251,7 @@ class TestCase: return FailureReason.SKIP elif "no-flaky-check" in tags and ( - 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) + 1 == int(os.environ.get("IS_FLAKY_CHECK", 0)) ): return FailureReason.SKIP From 71c06b40cbf65abda49579bf5ac08e46575c7d29 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 09:07:21 +0000 Subject: [PATCH 0513/1722] Avoid regexp --- docs/en/development/tests.md | 4 ++-- tests/clickhouse-test | 14 +++++++------- ...amic_merges_1_horizontal_compact_merge_tree.sql | 2 +- ...namic_merges_1_horizontal_compact_wide_tree.sql | 2 +- ...ynamic_merges_1_vertical_compact_merge_tree.sql | 2 +- ...7_dynamic_merges_1_vertical_wide_merge_tree.sql | 2 +- ...amic_merges_2_horizontal_compact_merge_tree.sql | 2 +- ...dynamic_merges_2_horizontal_wide_merge_tree.sql | 2 +- ...ynamic_merges_2_vertical_compact_merge_tree.sql | 2 +- ...7_dynamic_merges_2_vertical_wide_merge_tree.sql | 2 +- ...38_nested_dynamic_merges_compact_horizontal.sql | 2 +- ...3038_nested_dynamic_merges_compact_vertical.sql | 2 +- ...03038_nested_dynamic_merges_wide_horizontal.sql | 2 +- .../03038_nested_dynamic_merges_wide_vertical.sql | 2 +- 14 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index f0afa983fec..bc9f85ef323 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -100,14 +100,14 @@ For `.sh` tests limits are written as a comment on the line next to tags or on t ```bash #!/usr/bin/env bash # Tags: no-fasttest -# Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +# Random settings limits: max_block_size=(1000, 10000); index_granularity=(100, None) ``` For `.sql` tests tags are placed as a SQL comment in the line next to tags or in the first line: ```sql -- Tags: no-fasttest --- Random settings limits: max_block_size=(1000, 10000), index_granularity=(100, None) +-- Random settings limits: max_block_size=(1000, 10000); index_granularity=(100, None) SELECT 1 ``` diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 72136404796..e5378e8c7f3 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1964,13 +1964,13 @@ class TestSuite: if not random_settings_limits_str.startswith(random_settings_limits_prefix): return {} random_settings_limits_str = random_settings_limits_str[len(random_settings_limits_prefix) :] # noqa: ignore E203 - # limits are specified in a form 'setting1=(min, max), setting2=(min,max), ...' - random_settings_limits = re.findall( - "([^=, ]+) *= *(\([^=]+\))", random_settings_limits_str - ) - random_settings_limits = { - pair[0]: make_tuple(pair[1]) for pair in random_settings_limits - } + # limits are specified in a form 'setting1=(min, max); setting2=(min,max); ...' + random_settings_limits = {} + for setting_and_limit in random_settings_limits_str.split(';'): + setting_and_limit = setting_and_limit.split('=') + random_settings_limits[setting_and_limit[0].strip()] = make_tuple( + setting_and_limit[1] + ) return random_settings_limits def is_shebang(line: str) -> bool: diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index 7c2e7c3d2be..46f1c78b255 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql index aa62435188a..bf0c6ef0374 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql index bfc7bb9d206..fb82369a7a3 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql index 233667db0a7..c026bc04a56 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type=1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql index 71c6841515a..7f1934091f2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql index 94ae1d867f5..f1f387fae9d 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_horizontal_wide_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql index 98ae230636a..cc11c454d38 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_compact_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql index f8f5bd5d9e1..ffb2aca8b35 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_2_vertical_wide_merge_tree.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_dynamic_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 13c1fd8b485..9ec4e4f949b 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index daa95071cdb..ed4de931841 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index dea7e7c0971..bd3c4b58a8f 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index bf1323f2ea9..81bcda5443d 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -1,5 +1,5 @@ -- Tags: long, no-tsan, no-msan, no-ubsan, no-asan --- Random settings limits: index_granularity=(100, None), merge_max_block_size=(100, None) +-- Random settings limits: index_granularity=(100, None); merge_max_block_size=(100, None) set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; From aa3d8086c32ce2b5a90fbe4788579cae970ec32f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 6 Aug 2024 12:30:39 +0200 Subject: [PATCH 0514/1722] fix integration tests --- .../test_storage_azure_blob_storage/test.py | 37 ++++++++++++------- .../test_cluster.py | 21 +++++++---- tests/integration/test_storage_hdfs/test.py | 10 ++--- 3 files changed, 41 insertions(+), 27 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 15a1f6db2c1..092c124855c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -635,7 +635,8 @@ def test_simple_write_named_collection_2_table_function(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a') SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + settings={"azure_truncate_on_insert": 1}, ) print(get_azure_file_content("test_simple_write_named_2_tf.csv", port)) assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n' @@ -658,7 +659,8 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) query = ( f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " @@ -710,9 +712,9 @@ def test_schema_inference_from_globs_tf(cluster): query = ( f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " - f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1" + f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" ) - azure_query(node, query) + azure_query(node, query, settings={"azure_truncate_on_insert": 1}) query = ( f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " @@ -738,7 +740,8 @@ def test_partition_by_tf(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', " - f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", + f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port) @@ -757,7 +760,8 @@ def test_filter_using_file(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', " f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " - f"'{table_format}') PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", + f"'{table_format}') PARTITION BY {partition_by} VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) query = ( @@ -855,7 +859,8 @@ def test_function_signatures(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3) SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)", + settings={"azure_truncate_on_insert": 1}, ) # " - connection_string, container_name, blobpath\n" @@ -969,12 +974,14 @@ def test_union_schema_inference_mode(cluster): account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1) SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)", + settings={"azure_truncate_on_insert": 1}, ) azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2) SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)", + settings={"azure_truncate_on_insert": 1}, ) node.query("system drop schema cache for azure") @@ -1011,7 +1018,8 @@ def test_union_schema_inference_mode(cluster): assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error') SETTINGS azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')", + settings={"azure_truncate_on_insert": 1}, ) error = azure_query( @@ -1505,7 +1513,8 @@ def test_hive_partitioning_with_one_parameter(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) query = ( @@ -1542,7 +1551,8 @@ def test_hive_partitioning_with_two_parameters(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + settings={"azure_truncate_on_insert": 1}, ) query = ( @@ -1588,7 +1598,8 @@ def test_hive_partitioning_without_setting(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2} SETTINGS azure_truncate_on_insert=1", + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + settings={"azure_truncate_on_insert": 1}, ) query = ( diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 4d63016cf9a..04baf007c69 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -71,7 +71,8 @@ def test_select_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1'," f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') " - f"VALUES (1, 'a'), (2, 'b') SETTINGS azure_truncate_on_insert=1", + f"VALUES (1, 'a'), (2, 'b')", + settings={"azure_truncate_on_insert": 1}, ) print(get_azure_file_content("test_cluster_select_all.csv", port)) @@ -100,7 +101,8 @@ def test_count(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " - f"'auto', 'key UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'key UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) print(get_azure_file_content("test_cluster_count.csv", port)) @@ -128,7 +130,8 @@ def test_union_all(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_parquet_union_all', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', " - f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd') SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", + settings={"azure_truncate_on_insert": 1}, ) pure_azure = azure_query( @@ -179,7 +182,8 @@ def test_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) result = azure_query( node, @@ -199,7 +203,8 @@ def test_unset_skip_unavailable_shards(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_unset_skip_unavailable.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) result = azure_query( node, @@ -217,7 +222,8 @@ def test_cluster_with_named_collection(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " - f"'auto', 'a UInt64') VALUES (1), (2) SETTINGS azure_truncate_on_insert=1", + f"'auto', 'a UInt64') VALUES (1), (2)", + settings={"azure_truncate_on_insert": 1}, ) pure_azure = azure_query( @@ -248,7 +254,8 @@ def test_partition_parallel_reading_with_cluster(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', '{filename}', 'devstoreaccount1', " f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') " - f"PARTITION BY {partition_by} VALUES {values} SETTINGS azure_truncate_on_insert=1", + f"PARTITION BY {partition_by} VALUES {values}", + settings={"azure_truncate_on_insert": 1}, ) assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv", port) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 856715f28c8..3fef6bc46cf 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -341,7 +341,7 @@ def test_virtual_columns(started_cluster): ) == expected ) - node1.query("DROP TABLE virual_cols") + node1.query("DROP TABLE virtual_cols") def test_read_files_with_spaces(started_cluster): @@ -675,9 +675,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -685,9 +683,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" From 995187006a8c2500ddb7fa234f3443c75d900be4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 5 Aug 2024 20:23:41 +0200 Subject: [PATCH 0515/1722] rework custom table's disk usage --- src/Disks/DiskFomAST.cpp | 191 ++++++++++++++++++ src/Disks/DiskFomAST.h | 15 ++ src/Disks/DiskSelector.h | 4 +- src/Disks/IDisk.h | 9 +- src/Disks/StoragePolicy.h | 2 + src/Disks/getOrCreateDiskFromAST.cpp | 121 ----------- src/Disks/getOrCreateDiskFromAST.h | 18 -- src/Interpreters/Context.cpp | 14 +- src/Interpreters/Context.h | 1 + src/Parsers/FieldFromAST.cpp | 1 - src/Storages/MergeTree/MergeTreeSettings.cpp | 10 +- .../test_storage_policies/configs/disks.xml | 8 - .../integration/test_storage_policies/test.py | 55 ----- ...02963_test_flexible_disk_configuration.sql | 34 +++- 14 files changed, 264 insertions(+), 219 deletions(-) create mode 100644 src/Disks/DiskFomAST.cpp create mode 100644 src/Disks/DiskFomAST.h delete mode 100644 src/Disks/getOrCreateDiskFromAST.cpp delete mode 100644 src/Disks/getOrCreateDiskFromAST.h diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp new file mode 100644 index 00000000000..c8a4f88547f --- /dev/null +++ b/src/Disks/DiskFomAST.cpp @@ -0,0 +1,191 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_DISK; +} + +std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string & serialization, ContextPtr context, bool attach) +{ + Poco::Util::AbstractConfiguration::Keys disk_settings_keys; + config->keys(disk_settings_keys); + + + // Check that no settings are defined when disk from the config is referred. + if (disk_settings_keys.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk function has no arguments. Invalid disk description."); + + if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk function `{}` has to have the other arguments which describe the disk. Invalid disk description.", + serialization); + } + + std::string disk_name; + if (config->has("name")) + { + disk_name = config->getString("name"); + } + + if (!disk_name.empty()) + { + if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` could not start with `{}`", + disk_name, DiskSelector::CUSTOM_DISK_PREFIX); + + if (auto disk = context->tryGetDisk(disk_name)) + { + /// the disk is defined by config + if (disk->isCustomDisk()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Disk with name `{}` already exist as a custom disk but the name does not start with `{}`", + disk_name, + DiskSelector::CUSTOM_DISK_PREFIX); + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk `{}` is already exist. It is impossible to redefine it.", disk_name); + } + } + + auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); + + std::string custom_disk_name; + if (disk_name.empty()) + { + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + "noname_" + toString(disk_settings_hash); + } + else + { + custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + disk_name; + } + + auto result_disk = context->getOrCreateDisk(custom_disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + auto disk = DiskFactory::instance().create( + disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(disk_settings_hash); + return disk; + }); + + if (!result_disk->isCustomDisk()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk with name `{}` expected to be custom disk", disk_name); + + if (result_disk->getCustomDiskSettings() != disk_settings_hash && !attach) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The disk `{}` is already configured as a custom disk in another table. It can't be redefined with different settings.", + disk_name); + + if (!attach && !result_disk->isRemote()) + { + static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, ""); + + if (disk_path_expected_prefix.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Base path for custom local disks must be defined in config file by `{}`", + custom_local_disks_base_dir_in_config); + + if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path of the custom local disk must be inside `{}` directory", + disk_path_expected_prefix); + } + + return custom_disk_name; +} + +class DiskConfigurationFlattener +{ +public: + struct Data + { + ContextPtr context; + bool attach; + }; + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data & data) + { + if (isDiskFunction(ast)) + { + const auto * function = ast->as(); + const auto * function_args_expr = assert_cast(function->arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(function_args, data.context); + auto disk_setting_string = serializeAST(*function); + auto disk_name = getOrCreateCustomDisk(config, disk_setting_string, data.context, data.attach); + ast = std::make_shared(disk_name); + } + } +}; + + +std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, ContextPtr context, bool attach) +{ + if (!isDiskFunction(disk_function_ast)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function"); + + auto ast = disk_function_ast->clone(); + + using FlattenDiskConfigurationVisitor = InDepthNodeVisitor; + FlattenDiskConfigurationVisitor::Data data{context, attach}; + FlattenDiskConfigurationVisitor{data}.visit(ast); + + auto disk_name = assert_cast(*ast).value.get(); + return disk_name; +} + +std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, ContextPtr context) +{ + if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` could not start with `{}`", + disk_name, DiskSelector::CUSTOM_DISK_PREFIX); + + if (auto result = context->tryGetDisk(disk_name)) + return disk_name; + + std::string custom_disk_name = DiskSelector::CUSTOM_DISK_PREFIX + disk_name; + if (auto result = context->tryGetDisk(custom_disk_name)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` is a custom disk that is used in other table." + "That disk could not be used by a reference. The custom disk should be fully specified with a disk function.", + disk_name); + + throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", disk_name); +} + +} diff --git a/src/Disks/DiskFomAST.h b/src/Disks/DiskFomAST.h new file mode 100644 index 00000000000..3a70484eda0 --- /dev/null +++ b/src/Disks/DiskFomAST.h @@ -0,0 +1,15 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +namespace DiskFomAST +{ + std::string getConfigDefinedDisk(const std::string & name, ContextPtr context); + std::string createCustomDisk(const ASTPtr & disk_function, ContextPtr context, bool attach); +} + +} diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 49a1be5cf50..0f7424460a2 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -6,6 +6,8 @@ #include #include +#include +#include namespace DB { @@ -18,7 +20,7 @@ using DiskSelectorPtr = std::shared_ptr; class DiskSelector { public: - static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; + static constexpr auto CUSTOM_DISK_PREFIX = "__"; explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } DiskSelector(const DiskSelector & from) = default; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 412ad27e94f..78d5f37e3a7 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -464,9 +464,9 @@ public: virtual void chmod(const String & /*path*/, mode_t /*mode*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk does not support chmod"); } /// Was disk created to be used without storage configuration? - bool isCustomDisk() const { return is_custom_disk; } - - void markDiskAsCustom() { is_custom_disk = true; } + bool isCustomDisk() const { return custom_disk_settings_hash != 0; } + UInt128 getCustomDiskSettings() const { return custom_disk_settings_hash; } + void markDiskAsCustom(UInt128 settings_hash) { custom_disk_settings_hash = settings_hash; } virtual DiskPtr getDelegateDiskIfExists() const { return nullptr; } @@ -504,7 +504,8 @@ protected: private: ThreadPool copying_thread_pool; - bool is_custom_disk = false; + // 0 means the disk is not custom, the disk is predefined in the config + UInt128 custom_disk_settings_hash = 0; /// Check access to the disk. void checkAccess(); diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 501e033abc3..e23598214b3 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -119,6 +120,7 @@ class StoragePolicySelector { public: static constexpr auto TMP_STORAGE_POLICY_PREFIX = "__"; + static_assert(std::string_view(DiskSelector::CUSTOM_DISK_PREFIX) == std::string_view(TMP_STORAGE_POLICY_PREFIX)); StoragePolicySelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp deleted file mode 100644 index fd43f31a009..00000000000 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ /dev/null @@ -1,121 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -namespace -{ - std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context, bool attach) - { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(function_args, context); - - std::string disk_name; - if (config->has("name")) - { - disk_name = config->getString("name"); - } - else - { - /// We need a unique name for a created custom disk, but it needs to be the same - /// after table is reattached or server is restarted, so take a hash of the disk - /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function); - disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - } - - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - auto disk = DiskFactory::instance().create( - disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); - /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(); - return disk; - }); - - if (!result_disk->isCustomDisk()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk with name `{}` already exist", disk_name); - - if (!attach && !result_disk->isRemote()) - { - static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory"; - auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, ""); - - if (disk_path_expected_prefix.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Base path for custom local disks must be defined in config file by `{}`", - custom_local_disks_base_dir_in_config); - - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path of the custom local disk must be inside `{}` directory", - disk_path_expected_prefix); - } - - return disk_name; - } - - class DiskConfigurationFlattener - { - public: - struct Data - { - ContextPtr context; - bool attach; - }; - - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } - - static void visit(ASTPtr & ast, Data & data) - { - if (isDiskFunction(ast)) - { - auto disk_name = getOrCreateDiskFromDiskAST(*ast->as(), data.context, data.attach); - ast = std::make_shared(disk_name); - } - } - }; - - /// Visits children first. - using FlattenDiskConfigurationVisitor = InDepthNodeVisitor; -} - - -std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context, bool attach) -{ - if (!isDiskFunction(disk_function)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function"); - - auto ast = disk_function->clone(); - - FlattenDiskConfigurationVisitor::Data data{context, attach}; - FlattenDiskConfigurationVisitor{data}.visit(ast); - - auto disk_name = assert_cast(*ast).value.get(); - LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); - return disk_name; -} - -} diff --git a/src/Disks/getOrCreateDiskFromAST.h b/src/Disks/getOrCreateDiskFromAST.h deleted file mode 100644 index 61e1decbee9..00000000000 --- a/src/Disks/getOrCreateDiskFromAST.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -class ASTFunction; - -/** - * Create a DiskPtr from disk AST function like disk(), - * add it to DiskSelector by a unique (but always the same for given configuration) disk name - * and return this name. - */ -std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context, bool attach); - -} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5413b568068..0acbef26805 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4,6 +4,7 @@ #include #include #include +#include "Common/Logger.h" #include #include #include @@ -4395,6 +4396,15 @@ DiskPtr Context::getDisk(const String & name) const return disk_selector->get(name); } +DiskPtr Context::tryGetDisk(const String & name) const +{ + std::lock_guard lock(shared->storage_policies_mutex); + + auto disk_selector = getDiskSelector(lock); + + return disk_selector->tryGet(name); +} + DiskPtr Context::getOrCreateDisk(const String & name, DiskCreator creator) const { std::lock_guard lock(shared->storage_policies_mutex); @@ -4422,9 +4432,11 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & disk_name) const { + LOG_DEBUG(getLogger("StoragePolicy"), "getStoragePolicyFromDisk disk_name {}", disk_name); + std::lock_guard lock(shared->storage_policies_mutex); - const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; + const std::string storage_policy_name = disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX) ? disk_name : StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; auto storage_policy_selector = getStoragePolicySelector(lock); StoragePolicyPtr storage_policy = storage_policy_selector->tryGet(storage_policy_name); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5e35c3e4b3..586eff768df 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1186,6 +1186,7 @@ public: /// Provides storage disks DiskPtr getDisk(const String & name) const; + DiskPtr tryGetDisk(const String & name) const; using DiskCreator = std::function; DiskPtr getOrCreateDisk(const String & name, DiskCreator creator) const; diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index ad1eab49eeb..64aeae1b570 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index c968ad84936..f72b24e3270 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -64,10 +64,14 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte auto ast = dynamic_cast(custom.getImpl()).ast; if (ast && isDiskFunction(ast)) { - auto disk_name = getOrCreateDiskFromDiskAST(ast, context, is_attach); - LOG_TRACE(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); + auto disk_name = DiskFomAST::createCustomDisk(ast, context, is_attach); + LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); value = disk_name; } + else + { + value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); + } } if (has("storage_policy")) diff --git a/tests/integration/test_storage_policies/configs/disks.xml b/tests/integration/test_storage_policies/configs/disks.xml index dc60d93208c..3331fee4e4f 100644 --- a/tests/integration/test_storage_policies/configs/disks.xml +++ b/tests/integration/test_storage_policies/configs/disks.xml @@ -1,12 +1,4 @@ - - system - blob_storage_log
- toYYYYMM(event_date) - 7500 - event_date + INTERVAL 30 DAY -
- diff --git a/tests/integration/test_storage_policies/test.py b/tests/integration/test_storage_policies/test.py index f65096389af..389146b2171 100644 --- a/tests/integration/test_storage_policies/test.py +++ b/tests/integration/test_storage_policies/test.py @@ -38,58 +38,3 @@ def test_storage_policy_configuration_change(started_cluster): "/etc/clickhouse-server/config.d/disks.xml", ) node.start_clickhouse() - - -def test_disk_is_immutable(started_cluster): - node.query("DROP TABLE IF EXISTS test_1") - - node.query( - """ - create table test_1 (a Int32) - engine = MergeTree() - order by tuple() - settings - disk=disk( - name='not_uniq_disk_name', - type = object_storage, - object_storage_type = local_blob_storage, - path='./03215_data_test_1/') - """ - ) - - node.query("INSERT INTO test_1 VALUES (1)") - node.query("SYSTEM FLUSH LOGS;") - - print(node.query("SELECT 'test_1', * FROM system.blob_storage_log")) - - print(node.query("SELECT 'test_1', * FROM test_1")) - - node.query("DROP TABLE test_1 SYNC") - node.query("DROP TABLE IF EXISTS test_2") - - node.query( - """ - create table test_2 (a Int32) - engine = MergeTree() - order by tuple() - settings - disk=disk( - name='not_uniq_disk_name', - type = object_storage, - object_storage_type = local_blob_storage, - path='./03215_data_test_2/') - """ - ) - - node.query("INSERT INTO test_2 VALUES (1)") - node.query("SYSTEM FLUSH LOGS;") - - print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - - print(node.query("SELECT 'test_2', * FROM test_2")) - - node.restart_clickhouse() - - print(node.query("SELECT 'test_2', * FROM system.blob_storage_log")) - - print(node.query("SELECT 'test_2', * FROM test_2")) diff --git a/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql b/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql index 7ebef866360..6b285d423e7 100644 --- a/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql +++ b/tests/queries/0_stateless/02963_test_flexible_disk_configuration.sql @@ -2,13 +2,33 @@ drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() -settings disk=disk(name='test1', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test1/'); +settings disk=disk(name='02963_custom_disk', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test1/'); -drop table test; +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='02963_custom_disk', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test2/'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='02963_custom_disk'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk='02963_custom_disk'; -- { serverError BAD_ARGUMENTS } + +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='s3_disk_02963'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk='s3_disk_02963'; -drop table test; +drop table if exists test; +create table test (a Int32) engine = MergeTree() order by tuple() +settings disk=disk(name='s3_disk_02963', type = object_storage, object_storage_type = local_blob_storage, path='./02963_test2/'); -- { serverError BAD_ARGUMENTS } + +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test1', type = object_storage, @@ -17,7 +37,7 @@ settings disk=disk(name='test1', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test2', type = object_storage, @@ -27,7 +47,7 @@ settings disk=disk(name='test2', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test3', type = object_storage, @@ -37,8 +57,8 @@ settings disk=disk(name='test3', endpoint = 'http://localhost:11111/test/common/', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test4', type = object_storage, @@ -48,8 +68,8 @@ settings disk=disk(name='test4', endpoint = 'http://localhost:11111/test/common/', access_key_id = clickhouse, secret_access_key = clickhouse); -drop table test; +drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple() settings disk=disk(name='test5', type = object_storage, From 27cdbb54d73f8f4b82d63850c1e6f6fd5669646e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 6 Aug 2024 13:10:03 +0200 Subject: [PATCH 0516/1722] fix black --- tests/integration/test_storage_hdfs/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 3fef6bc46cf..77921b885b0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -675,7 +675,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -683,7 +685,9 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query(f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1") + node1.query( + f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" + ) result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" From 621f4bbf9e04b62628a9c053b3f39c6b8a67a52d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 6 Aug 2024 14:13:20 +0200 Subject: [PATCH 0517/1722] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index f815a21b6a1..bb062deaab0 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,6 +79,7 @@ static std::initializer_list Date: Tue, 6 Aug 2024 20:44:42 +0800 Subject: [PATCH 0518/1722] Fix tests --- .../0_stateless/02864_statistics_estimation.reference | 6 +++--- tests/queries/0_stateless/02864_statistics_estimation.sql | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index cb274b2990e..fc370691830 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -14,15 +14,15 @@ Test statistics tdigest: Test statistics uniq: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), equals(b, 0), equals(c, 0)) (removed) + Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), equals(c, 0)) (removed) Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) + Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) + Prewhere filter column: and(equals(a, \'1\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 4a221686069..6436cf68738 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -59,7 +59,7 @@ SELECT 'Test statistics uniq:'; ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/ and b = 0/*100*/ and a = '0'/*100*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; @@ -73,7 +73,7 @@ ALTER TABLE tab ADD STATISTICS d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') @@ -83,7 +83,7 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT 'Test statistics implicitly type conversion:'; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '10000'/*0*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '1'/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; From 56415028d6be64b37bf9d3065f846ee1455f2711 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 6 Aug 2024 15:01:10 +0200 Subject: [PATCH 0519/1722] Fix pylint --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e5378e8c7f3..dea303ecdfb 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1086,7 +1086,7 @@ class TestCase: self.random_settings_limits = ( suite.all_random_settings_limits[case] if case in suite.all_random_settings_limits - else dict() + else {} ) for tag in os.getenv("GLOBAL_TAGS", "").split(","): From 20cd84960efa48c7978607f92e58bc4b66187840 Mon Sep 17 00:00:00 2001 From: sunny <923976094@qq.com> Date: Tue, 6 Aug 2024 21:42:14 +0800 Subject: [PATCH 0520/1722] modify code style modify code style --- src/Parsers/ParserDeleteQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp index 65d593b6c1b..3503fbe9ad2 100644 --- a/src/Parsers/ParserDeleteQuery.cpp +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -62,6 +62,7 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else return false; + if (query->partition) query->children.push_back(query->partition); From 99b18d31db32a077678661bd9ba84fb52ff49333 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 15:30:38 +0200 Subject: [PATCH 0521/1722] Update gdb to 15.1 (by compiling from sources) Right now there are couple of gdb bugs that makes CI unstable: - https://sourceware.org/bugzilla/show_bug.cgi?id=29185 - https://bugzilla.redhat.com/show_bug.cgi?id=1492496 But ubuntu 22.04 does not have 14+ anywhere, the ~ubuntu-toolchain-r/test contains only gdb 13, so there is no other options except for compiling it from sources. But there also other reasons to update it - optimizations, looks like older gdb versions does not use index fully - 5.6sec vs 56sec: # 15.1 $ time command gdb -batch -ex 'disas main' clickhouse ... real 0m5.692s user 0m29.948s sys 0m1.190s # 12.1 (from ubuntu 22.04) real 0m56.709s user 0m59.307s sys 0m0.585s Also note, that we cannot compile gdb in the fasttest (that contains compiler) since some images does not includes full toolchain, for instance gdb is added in the following images: - test-util -> test-base -> lots of other images (no toolchain) - performance-comparison (no toolchain) - integration-test (no toolchain) - integration-tests-runner (no toolchain) Signed-off-by: Azat Khuzhin --- docker/images.json | 4 +++ docker/packager/binary-builder/Dockerfile | 5 ++++ docker/packager/gdb/Dockerfile | 30 +++++++++++++++++++++++ 3 files changed, 39 insertions(+) create mode 100644 docker/packager/gdb/Dockerfile diff --git a/docker/images.json b/docker/images.json index 716b76ee217..eae4ad30a79 100644 --- a/docker/images.json +++ b/docker/images.json @@ -7,6 +7,10 @@ "name": "clickhouse/cctools", "dependent": [] }, + "docker/packager/gdb": { + "name": "clickhouse/gdb", + "dependent": [] + }, "docker/test/compatibility/centos": { "name": "clickhouse/test-old-centos", "dependent": [] diff --git a/docker/packager/binary-builder/Dockerfile b/docker/packager/binary-builder/Dockerfile index 7d6acdcd856..647ab8758a5 100644 --- a/docker/packager/binary-builder/Dockerfile +++ b/docker/packager/binary-builder/Dockerfile @@ -6,6 +6,11 @@ ENV CXX=clang++-${LLVM_VERSION} # If the cctools is updated, then first build it in the CI, then update here in a different commit COPY --from=clickhouse/cctools:d9e3596e706b /cctools /cctools +# TODO: same for gdb and in other places as well +# +# NOTE: here it will add circular dependency but it will be fixed after [1] +# +# [1]: https://github.com/ClickHouse/ClickHouse/issues/66493 # Rust toolchain and libraries ENV RUSTUP_HOME=/rust/rustup diff --git a/docker/packager/gdb/Dockerfile b/docker/packager/gdb/Dockerfile new file mode 100644 index 00000000000..1b5c3bbcb2e --- /dev/null +++ b/docker/packager/gdb/Dockerfile @@ -0,0 +1,30 @@ +# docker build -t clickhouse/gdb . + +ARG FROM_TAG=latest +FROM clickhouse/fasttest:$FROM_TAG + +ENV CC=clang-${LLVM_VERSION} +ENV CXX=clang++-${LLVM_VERSION} +# ld from binutils is 2.38, which has the following error: +# +# DWARF error: invalid or unhandled FORM value: 0x23 +# +ENV LD=ld.lld-${LLVM_VERSION} + +ARG GDB_VERSION=15.1 + +# gdb dependencies +RUN apt-get update \ + && apt-get install --yes \ + libgmp-dev \ + libmpfr-dev \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + +RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \ + && tar -xvf gdb-$GDB_VERSION.tar.gz \ + && cd gdb-$GDB_VERSION \ + && ./configure --prefix=/usr \ + && make -j $(nproc) \ + && make install \ + && rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz From 459dd1ff2db90f85d9c1f72329ff2f31117c13a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 16:35:49 +0200 Subject: [PATCH 0522/1722] Add a comment of the image name for cctools Signed-off-by: Azat Khuzhin --- docker/packager/cctools/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/cctools/Dockerfile b/docker/packager/cctools/Dockerfile index d986c6a3c86..56a4a65bcc9 100644 --- a/docker/packager/cctools/Dockerfile +++ b/docker/packager/cctools/Dockerfile @@ -1,3 +1,5 @@ +# docker build -t clickhouse/cctools . + # This is a hack to significantly reduce the build time of the clickhouse/binary-builder # It's based on the assumption that we don't care of the cctools version so much # It event does not depend on the clickhouse/fasttest in the `docker/images.json` From 16fcd5b825b50ba88de8ef42b37d40eac1af5596 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 14 Jul 2024 15:48:12 +0200 Subject: [PATCH 0523/1722] Remove .gdb-index (do not give any benefit for gdb 15.1) Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 8 -------- 1 file changed, 8 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f796e6c4616..f3c6b2abb30 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -187,14 +187,6 @@ else () set(NO_WHOLE_ARCHIVE --no-whole-archive) endif () -if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") - # Can be lld or ld-lld or lld-13 or /path/to/lld. - if (LINKER_NAME MATCHES "lld") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") - message (STATUS "Adding .gdb-index via --gdb-index linker option.") - endif () -endif() - if (NOT (SANITIZE_COVERAGE OR WITH_COVERAGE) AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" From 424f19d6c0d003d2a9e2c03017c7fa7dc9dd9233 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Aug 2024 22:49:27 +0200 Subject: [PATCH 0524/1722] Move gdb into cctools Signed-off-by: Azat Khuzhin --- docker/images.json | 4 ---- docker/packager/cctools/Dockerfile | 24 ++++++++++++++++++++++++ docker/packager/gdb/Dockerfile | 30 ------------------------------ 3 files changed, 24 insertions(+), 34 deletions(-) delete mode 100644 docker/packager/gdb/Dockerfile diff --git a/docker/images.json b/docker/images.json index eae4ad30a79..716b76ee217 100644 --- a/docker/images.json +++ b/docker/images.json @@ -7,10 +7,6 @@ "name": "clickhouse/cctools", "dependent": [] }, - "docker/packager/gdb": { - "name": "clickhouse/gdb", - "dependent": [] - }, "docker/test/compatibility/centos": { "name": "clickhouse/test-old-centos", "dependent": [] diff --git a/docker/packager/cctools/Dockerfile b/docker/packager/cctools/Dockerfile index 56a4a65bcc9..570a42d42d5 100644 --- a/docker/packager/cctools/Dockerfile +++ b/docker/packager/cctools/Dockerfile @@ -32,5 +32,29 @@ RUN git clone https://github.com/tpoechtrager/cctools-port.git \ && cd ../.. \ && rm -rf cctools-port +# +# GDB +# +# ld from binutils is 2.38, which has the following error: +# +# DWARF error: invalid or unhandled FORM value: 0x23 +# +ENV LD=ld.lld-${LLVM_VERSION} +ARG GDB_VERSION=15.1 +RUN apt-get update \ + && apt-get install --yes \ + libgmp-dev \ + libmpfr-dev \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* +RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \ + && tar -xvf gdb-$GDB_VERSION.tar.gz \ + && cd gdb-$GDB_VERSION \ + && ./configure --prefix=/opt/gdb \ + && make -j $(nproc) \ + && make install \ + && rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz + FROM scratch COPY --from=builder /cctools /cctools +COPY --from=builder /opt/gdb /opt/gdb diff --git a/docker/packager/gdb/Dockerfile b/docker/packager/gdb/Dockerfile deleted file mode 100644 index 1b5c3bbcb2e..00000000000 --- a/docker/packager/gdb/Dockerfile +++ /dev/null @@ -1,30 +0,0 @@ -# docker build -t clickhouse/gdb . - -ARG FROM_TAG=latest -FROM clickhouse/fasttest:$FROM_TAG - -ENV CC=clang-${LLVM_VERSION} -ENV CXX=clang++-${LLVM_VERSION} -# ld from binutils is 2.38, which has the following error: -# -# DWARF error: invalid or unhandled FORM value: 0x23 -# -ENV LD=ld.lld-${LLVM_VERSION} - -ARG GDB_VERSION=15.1 - -# gdb dependencies -RUN apt-get update \ - && apt-get install --yes \ - libgmp-dev \ - libmpfr-dev \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* - -RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \ - && tar -xvf gdb-$GDB_VERSION.tar.gz \ - && cd gdb-$GDB_VERSION \ - && ./configure --prefix=/usr \ - && make -j $(nproc) \ - && make install \ - && rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz From 815fdc43ac333a75adff646fef073ea591494d13 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 6 Aug 2024 14:36:02 +0000 Subject: [PATCH 0525/1722] Revert "Merge pull request #67800 from ClickHouse/revert-66510" This reverts commit 45c4a71ccb62bac6728d0e583fd04c0fc4f45a6f, reversing changes made to bb71c1eea8e6019a5a21b6add08c2244764ddea5. --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 21 ++++++++++++------- src/Storages/VirtualColumnUtils.h | 10 ++++++++- ..._with_non_deterministic_function.reference | 2 ++ ..._count_with_non_deterministic_function.sql | 4 ++++ 5 files changed, 30 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference create mode 100644 tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 49888596fbb..ce27ad24e10 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1146,7 +1146,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false); if (!filter_dag) return {}; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index ba1f4488005..90c2c7f93c1 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -275,7 +275,8 @@ bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( const ActionsDAG::Node * node, const Block * allowed_inputs, - ActionsDAG::Nodes & additional_nodes) + ActionsDAG::Nodes & additional_nodes, + bool allow_non_deterministic_functions) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -284,8 +285,14 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto & node_copy = additional_nodes.emplace_back(*node); node_copy.children.clear(); for (const auto * child : node->children) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) node_copy.children.push_back(child_copy); + /// Expression like (now_allowed AND allowed) is not allowed if allow_non_deterministic_functions = true. This is important for + /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is + /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply + /// trivial count. + else if (!allow_non_deterministic_functions) + return nullptr; if (node_copy.children.empty()) return nullptr; @@ -311,7 +318,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { auto & node_copy = additional_nodes.emplace_back(*node); for (auto & child : node_copy.children) - if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes); !child) + if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions); !child) return nullptr; return &node_copy; @@ -325,7 +332,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto index_hint_dag = index_hint->getActions().clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag.getOutputs()) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes)) + if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -359,13 +366,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs) +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions) { if (!predicate) return {}; ActionsDAG::Nodes additional_nodes; - const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes); + const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_non_deterministic_functions); if (!res) return {}; @@ -374,7 +381,7 @@ std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block); + auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); if (dag) filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index d75dc70ae44..abf46dc23a4 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -41,7 +41,15 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); +/// When allow_non_deterministic_functions is true then even if the predicate contains non-deterministic +/// functions, we still allow to extract a part of the predicate, otherwise we return nullptr. +/// allow_non_deterministic_functions must be false when we are going to use the result to filter parts in +/// MergeTreeData::totalRowsByPartitionPredicateImp. For example, if the query is +/// `SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1` +/// The predicate will be `_partition_id = '0' AND rowNumberInBlock() = 1`, and `rowNumberInBlock()` is +/// non-deterministic. If we still extract the part `_partition_id = '0'` for filtering parts, then trivial +/// count optimization will be mistakenly applied to the query. +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions = true); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql new file mode 100644 index 00000000000..bb3269da597 --- /dev/null +++ b/tests/queries/0_stateless/03203_count_with_non_deterministic_function.sql @@ -0,0 +1,4 @@ +CREATE TABLE t (p UInt8, x UInt64) Engine = MergeTree PARTITION BY p ORDER BY x; +INSERT INTO t SELECT 0, number FROM numbers(10) SETTINGS max_block_size = 100; +SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 0; +SELECT count() FROM t WHERE p = 0 AND rowNumberInAllBlocks() = 1 SETTINGS allow_experimental_analyzer = 1; From 2074485083e8860aafc36ac7886a54a75e144468 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 6 Aug 2024 14:25:02 +0000 Subject: [PATCH 0526/1722] Fix partial filtering in `filterBlockWithPredicate` --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +- src/Storages/VirtualColumnUtils.cpp | 86 +++++++++---------- src/Storages/VirtualColumnUtils.h | 16 ++-- ...03217_read_rows_in_system_tables.reference | 10 +++ .../03217_read_rows_in_system_tables.sql | 34 ++++++++ 5 files changed, 97 insertions(+), 52 deletions(-) create mode 100644 tests/queries/0_stateless/03217_read_rows_in_system_tables.reference create mode 100644 tests/queries/0_stateless/03217_read_rows_in_system_tables.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ce27ad24e10..b24d7968b61 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6914,7 +6914,8 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const auto * predicate = filter_dag->getOutputs().at(0); // Generate valid expressions for filtering - VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context); + VirtualColumnUtils::filterBlockWithPredicate( + predicate, virtual_columns_block, query_context, /*allow_filtering_with_partial_predicate =*/true); rows = virtual_columns_block.rows(); part_name_column = virtual_columns_block.getByName("_part").column; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 90c2c7f93c1..b40378250bb 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,51 +1,46 @@ -#include +#include + #include #include +#include +#include +#include +#include +#include #include #include - +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include -#include -#include #include +#include #include +#include #include - -#include #include -#include #include +#include +#include #include #include - -#include -#include -#include -#include - -#include -#include -#include -#include - -#include +#include +#include #include #include +#include #include -#include #include - -#include -#include #include -#include "Functions/FunctionsLogical.h" -#include "Functions/IFunction.h" -#include "Functions/IFunctionAdaptors.h" -#include "Functions/indexHint.h" -#include -#include -#include -#include namespace DB @@ -273,10 +268,7 @@ bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) } static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( - const ActionsDAG::Node * node, - const Block * allowed_inputs, - ActionsDAG::Nodes & additional_nodes, - bool allow_non_deterministic_functions) + const ActionsDAG::Node * node, const Block * allowed_inputs, ActionsDAG::Nodes & additional_nodes, bool allow_partial_result) { if (node->type == ActionsDAG::ActionType::FUNCTION) { @@ -285,13 +277,14 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto & node_copy = additional_nodes.emplace_back(*node); node_copy.children.clear(); for (const auto * child : node->children) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) + if (const auto * child_copy + = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_partial_result)) node_copy.children.push_back(child_copy); - /// Expression like (now_allowed AND allowed) is not allowed if allow_non_deterministic_functions = true. This is important for + /// Expression like (now_allowed AND allowed) is not allowed if allow_partial_result = true. This is important for /// trivial count optimization, otherwise we can get incorrect results. For example, if the query is /// SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1, we cannot apply /// trivial count. - else if (!allow_non_deterministic_functions) + else if (!allow_partial_result) return nullptr; if (node_copy.children.empty()) @@ -300,7 +293,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( if (node_copy.children.size() == 1) { const ActionsDAG::Node * res = node_copy.children.front(); - /// Expression like (not_allowed AND 256) can't be resuced to (and(256)) because AND requires + /// Expression like (not_allowed AND 256) can't be reduced to (and(256)) because AND requires /// at least two arguments; also it can't be reduced to (256) because result type is different. if (!res->result_type->equals(*node->result_type)) { @@ -318,7 +311,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( { auto & node_copy = additional_nodes.emplace_back(*node); for (auto & child : node_copy.children) - if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_non_deterministic_functions); !child) + if (child = splitFilterNodeForAllowedInputs(child, allowed_inputs, additional_nodes, allow_partial_result); !child) return nullptr; return &node_copy; @@ -332,7 +325,8 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( auto index_hint_dag = index_hint->getActions().clone(); ActionsDAG::NodeRawConstPtrs atoms; for (const auto & output : index_hint_dag.getOutputs()) - if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_non_deterministic_functions)) + if (const auto * child_copy + = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes, allow_partial_result)) atoms.push_back(child_copy); if (!atoms.empty()) @@ -366,22 +360,24 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( return node; } -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions) +std::optional +splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_partial_result) { if (!predicate) return {}; ActionsDAG::Nodes additional_nodes; - const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_non_deterministic_functions); + const auto * res = splitFilterNodeForAllowedInputs(predicate, allowed_inputs, additional_nodes, allow_partial_result); if (!res) return {}; return ActionsDAG::cloneSubDAG({res}, true); } -void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context) +void filterBlockWithPredicate( + const ActionsDAG::Node * predicate, Block & block, ContextPtr context, bool allow_filtering_with_partial_predicate) { - auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_non_deterministic_functions=*/ false); + auto dag = splitFilterDagForAllowedInputs(predicate, &block, /*allow_partial_result=*/allow_filtering_with_partial_predicate); if (dag) filterBlockWithExpression(buildFilterExpression(std::move(*dag), context), block); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index abf46dc23a4..f76cf2cad76 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -26,9 +26,13 @@ namespace VirtualColumnUtils /// /// Otherwise calling filter*() outside applyFilters() will throw "Not-ready Set is passed" /// if there are subqueries. +/// +/// Similar to filterBlockWithExpression(buildFilterExpression(splitFilterDagForAllowedInputs(...)))./// Similar to filterBlockWithQuery, but uses ActionsDAG as a predicate. +/// Basically it is filterBlockWithDAG(splitFilterDagForAllowedInputs). +/// If allow_filtering_with_partial_predicate is true, then the filtering will be done even if some part of the predicate +/// cannot be evaluated using the columns from the block. +void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context, bool allow_filtering_with_partial_predicate = true); -/// Similar to filterBlockWithExpression(buildFilterExpression(splitFilterDagForAllowedInputs(...))). -void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context); /// Just filters block. Block should contain all the required columns. ExpressionActionsPtr buildFilterExpression(ActionsDAG dag, ContextPtr context); @@ -41,15 +45,15 @@ void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); /// Extract a part of predicate that can be evaluated using only columns from input_names. -/// When allow_non_deterministic_functions is true then even if the predicate contains non-deterministic -/// functions, we still allow to extract a part of the predicate, otherwise we return nullptr. -/// allow_non_deterministic_functions must be false when we are going to use the result to filter parts in +/// When allow_partial_result is false, then the result will be empty if any part of if cannot be evaluated deterministically +/// on the given inputs. +/// allow_partial_result must be false when we are going to use the result to filter parts in /// MergeTreeData::totalRowsByPartitionPredicateImp. For example, if the query is /// `SELECT count() FROM table WHERE _partition_id = '0' AND rowNumberInBlock() = 1` /// The predicate will be `_partition_id = '0' AND rowNumberInBlock() = 1`, and `rowNumberInBlock()` is /// non-deterministic. If we still extract the part `_partition_id = '0'` for filtering parts, then trivial /// count optimization will be mistakenly applied to the query. -std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_non_deterministic_functions = true); +std::optional splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs, bool allow_partial_result = true); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference b/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference new file mode 100644 index 00000000000..b21ead49b1e --- /dev/null +++ b/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference @@ -0,0 +1,10 @@ +information_schema tables +default test_replica_1 r1 +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + ReadFromMerge + Filter (( + ( + ))) + ReadFromMergeTree (default.test_replica_1) +1 1 +1 1 diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql b/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql new file mode 100644 index 00000000000..3bea04ccccf --- /dev/null +++ b/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql @@ -0,0 +1,34 @@ +SELECT database, table FROM system.tables WHERE database = 'information_schema' AND table = 'tables'; + +-- To verify StorageSystemReplicas applies the filter properly +CREATE TABLE test_replica_1(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r1') + ORDER BY x; +CREATE TABLE test_replica_2(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r2') + ORDER BY x; + +SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_replica_1' AND replica_name = 'r1'; + + +-- To verify StorageMerge +CREATE TABLE all_replicas (x UInt32) + ENGINE = Merge(currentDatabase(), 'test_replica_*'); + +INSERT INTO test_replica_1 SELECT number AS x FROM numbers(10); +SYSTEM SYNC REPLICA test_replica_2; +-- If the filter not applied, then the plan will show both replicas +EXPLAIN SELECT _table, count() FROM all_replicas WHERE _table = 'test_replica_1' AND x >= 0 GROUP BY _table; + +SYSTEM FLUSH LOGS; +-- argMin-argMax make the test repeatable + +-- StorageSystemTables +SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND query LIKE '%SELECT database, table FROM system.tables WHERE database = \'information_schema\' AND table = \'tables\';' + AND type = 'QueryFinish'; + +-- StorageSystemReplicas +SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_replica_1\' AND replica_name = \'r1\';' + AND type = 'QueryFinish'; From 72ead6e8432daa1e643a5b0cc8559a4ff4d9efd0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 6 Aug 2024 14:56:42 +0000 Subject: [PATCH 0527/1722] Cleanup. --- src/Storages/IStorage.h | 6 ++-- src/Storages/MergeTree/MutateTask.cpp | 34 +++++++++---------- ...61_lightweight_delete_projection.reference | 4 +-- .../03161_lightweight_delete_projection.sql | 4 +-- 4 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index d2cdc5af34f..0477a08b0d2 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -259,12 +259,12 @@ public: /// Return true if there is at least one part containing lightweight deleted mask. virtual bool hasLightweightDeletedMask() const { return false; } - /// Return true if storage has any projection. - virtual bool hasProjection() const { return false; } - /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } + /// Return true if storage has any projection. + virtual bool hasProjection() const { return false; } + /// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8b5829eb058..3d9f49c9a7a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -659,10 +659,8 @@ static NameSet collectFilesToSkip( const Block & updated_header, const std::set & indices_to_recalc, const String & mrk_extension, - const std::set & projections_to_recalc, - const std::set & stats_to_recalc, - const StorageMetadataPtr & metadata_snapshot, - bool skip_all_projections) + const std::set & projections_to_skip, + const std::set & stats_to_recalc) { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); @@ -686,16 +684,8 @@ static NameSet collectFilesToSkip( } } - if (skip_all_projections) - { - for (const auto & projection : metadata_snapshot->getProjections()) - files_to_skip.insert(projection.getDirectoryName()); - } - else - { - for (const auto & projection : projections_to_recalc) - files_to_skip.insert(projection->getDirectoryName()); - } + for (const auto & projection : projections_to_skip) + files_to_skip.insert(projection->getDirectoryName()); for (const auto & stat : stats_to_recalc) files_to_skip.insert(stat->getFileName() + STATS_FILE_SUFFIX); @@ -2325,6 +2315,9 @@ bool MutateTask::prepare() lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP || lightweight_mutation_projection_mode == LightweightMutationProjectionMode::THROW; + std::set projections_to_skip_container; + auto * projections_to_skip = &projections_to_skip_container; + bool should_create_projections = !(lightweight_delete_mode && lightweight_delete_drops_projections); /// Under lightweight delete mode, if option is drop, projections_to_recalc should be empty. if (should_create_projections) @@ -2333,6 +2326,13 @@ bool MutateTask::prepare() ctx->source_part, ctx->metadata_snapshot, ctx->materialized_projections); + + projections_to_skip = &ctx->projections_to_recalc; + } + else + { + for (const auto & projection : ctx->metadata_snapshot->getProjections()) + projections_to_skip->insert(&projection); } ctx->stats_to_recalc = MutationHelpers::getStatisticsToRecalculate(ctx->metadata_snapshot, ctx->materialized_statistics); @@ -2343,10 +2343,8 @@ bool MutateTask::prepare() ctx->updated_header, ctx->indices_to_recalc, ctx->mrk_extension, - ctx->projections_to_recalc, - ctx->stats_to_recalc, - ctx->metadata_snapshot, - !should_create_projections); + *projections_to_skip, + ctx->stats_to_recalc); ctx->files_to_rename = MutationHelpers::collectFilesForRenames( ctx->source_part, diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference index eef0c5a41b5..8edf541c2a0 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference @@ -40,7 +40,7 @@ all_3_3_0_4 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1) AND parent_name like 'all_3_3%'; p1 all_3_3_0_4 p2 all_3_3_0_4 wide part @@ -85,6 +85,6 @@ all_3_3_0_4 SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1) AND parent_name like 'all_3_3%'; p1 all_3_3_0_4 p2 all_3_3_0_4 diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 618f3ac0cb8..0b05326e2c1 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -67,7 +67,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_compact') AND (active = 1) AND parent_name like 'all_3_3%'; -- { echoOff } @@ -136,7 +136,7 @@ WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = SELECT name, parent_name FROM system.projection_parts -WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1); +WHERE (database = currentDatabase()) AND (`table` = 'users_wide') AND (active = 1) AND parent_name like 'all_3_3%'; -- { echoOff } From 86e3b35f24a449b63172015a0f768434f9f203c6 Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Tue, 6 Aug 2024 09:07:17 -0600 Subject: [PATCH 0528/1722] spelling fixes --- src/DataTypes/DataTypeCustomGeo.cpp | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index d72787647c3..f90788ec403 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -24,7 +24,7 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique())); }); - // Custom type for mulitple lines stored as Array(LineString) + // Custom type for multiple lines stored as Array(LineString) factory.registerSimpleDataTypeCustom("MultiLineString", [] { return std::make_pair(DataTypeFactory::instance().get("Array(LineString)"), diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3d7e77f213d..71f5efca893 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2086,6 +2086,7 @@ multiSearchFirstPositionUTF multibyte multidirectory multiline +multilinestring multiplyDecimal multipolygon multisearchany From 5390d1b108956907bf4b038a56fdb2ed8e584308 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 6 Aug 2024 16:12:44 +0200 Subject: [PATCH 0529/1722] docker: use self-compiled gdb 15 from cctools Signed-off-by: Azat Khuzhin --- docker/packager/binary-builder/Dockerfile | 5 ----- docker/test/fasttest/Dockerfile | 2 +- docker/test/integration/base/Dockerfile | 4 +++- docker/test/integration/runner/Dockerfile | 3 ++- docker/test/performance-comparison/Dockerfile | 4 +++- docker/test/util/Dockerfile | 4 +++- 6 files changed, 12 insertions(+), 10 deletions(-) diff --git a/docker/packager/binary-builder/Dockerfile b/docker/packager/binary-builder/Dockerfile index 647ab8758a5..7d6acdcd856 100644 --- a/docker/packager/binary-builder/Dockerfile +++ b/docker/packager/binary-builder/Dockerfile @@ -6,11 +6,6 @@ ENV CXX=clang++-${LLVM_VERSION} # If the cctools is updated, then first build it in the CI, then update here in a different commit COPY --from=clickhouse/cctools:d9e3596e706b /cctools /cctools -# TODO: same for gdb and in other places as well -# -# NOTE: here it will add circular dependency but it will be fixed after [1] -# -# [1]: https://github.com/ClickHouse/ClickHouse/issues/66493 # Rust toolchain and libraries ENV RUSTUP_HOME=/rust/rustup diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 4cac2ee6135..5d311c673a4 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -83,7 +83,7 @@ RUN arch=${TARGETARCH:-amd64} \ # Give suid to gdb to grant it attach permissions # chmod 777 to make the container user independent -RUN chmod u+s /usr/bin/gdb \ +RUN chmod u+s /opt/gdb/bin/gdb \ && mkdir -p /var/lib/clickhouse \ && chmod 777 /var/lib/clickhouse diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 469251f648c..dc4d470a262 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -11,7 +11,6 @@ RUN apt-get update \ curl \ default-jre \ g++ \ - gdb \ iproute2 \ krb5-user \ libicu-dev \ @@ -73,3 +72,6 @@ maxClientCnxns=80' > /opt/zookeeper/conf/zoo.cfg && \ ENV TZ=Etc/UTC RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d250b746e7d..d62009f1be3 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -30,7 +30,6 @@ RUN apt-get update \ luajit \ libssl-dev \ libcurl4-openssl-dev \ - gdb \ default-jdk \ software-properties-common \ libkrb5-dev \ @@ -87,6 +86,8 @@ COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ COPY misc/ /misc/ +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" # Same options as in test/base/Dockerfile # (in case you need to override them in tests) diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index c68a39f6f70..f7139275282 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -9,7 +9,6 @@ RUN apt-get update \ curl \ dmidecode \ g++ \ - gdb \ git \ gnuplot \ imagemagick \ @@ -42,6 +41,9 @@ RUN pip3 --no-cache-dir install -r requirements.txt COPY run.sh / +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" + CMD ["bash", "/run.sh"] # docker run --network=host --volume :/workspace --volume=:/output -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/performance-comparison diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index dc928ba7195..8b949ed95db 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -44,7 +44,6 @@ RUN apt-get update \ bash \ bsdmainutils \ build-essential \ - gdb \ git \ gperf \ moreutils \ @@ -58,3 +57,6 @@ RUN apt-get update \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* COPY process_functional_tests_result.py / + +COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb +ENV PATH="/opt/gdb/bin:${PATH}" From 3ebc3852f404a1ce392e9b66b8356fa9da701097 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Mon, 22 Jul 2024 15:28:29 +0000 Subject: [PATCH 0530/1722] Allow filtering ip addresses by ip family in DNS resolver --- programs/server/Server.cpp | 3 + src/Common/DNSResolver.cpp | 93 ++++++-- src/Common/DNSResolver.h | 9 + src/Core/ServerSettings.h | 2 + src/Core/SettingsChangesHistory.cpp | 261 +++++++++++++++++++++++ tests/integration/test_dns_cache/test.py | 67 +++++- 6 files changed, 418 insertions(+), 17 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7800ee9ff00..3126f65ef09 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1768,6 +1768,9 @@ try new_server_settings.http_connections_store_limit, }); + DNSResolver::instance().setFilterSettings(new_server_settings.dns_allow_resolve_names_to_ipv4, new_server_settings.dns_allow_resolve_names_to_ipv6); + + if (global_context->isServerCompletelyStarted()) CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability); diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 4b577a251af..051e6e63091 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include #include "DNSPTRResolverProvider.h" @@ -139,12 +141,6 @@ DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) return addresses; } -DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase & cache, const std::string & host) -{ - auto [result, _ ] = cache.getOrSet(host, [&host]() {return std::make_shared(resolveIPAddressImpl(host), std::chrono::system_clock::now());}); - return result->addresses; -} - std::unordered_set reverseResolveImpl(const Poco::Net::IPAddress & address) { auto ptr_resolver = DB::DNSPTRResolverProvider::get(); @@ -198,21 +194,90 @@ struct DNSResolver::Impl std::atomic disable_cache{false}; }; +struct DNSResolver::AddressFilter +{ + struct DNSFilterSettings + { + std::atomic dns_allow_resolve_names_to_ipv4{true}; + std::atomic dns_allow_resolve_names_to_ipv6{true}; + }; -DNSResolver::DNSResolver() : impl(std::make_unique()), log(getLogger("DNSResolver")) {} + void performAddressFiltering(DNSResolver::IPAddresses & addresses) + { + bool dns_resolve_ipv4 = settings.dns_allow_resolve_names_to_ipv4; + bool dns_resolve_ipv6 = settings.dns_allow_resolve_names_to_ipv6; + + if (dns_resolve_ipv4 && dns_resolve_ipv6) + { + return; + } + if (!dns_resolve_ipv4 && !dns_resolve_ipv6) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "DNS can't resolve any address, because dns_resolve_ipv6_interfaces and dns_resolve_ipv4_interfaces both are disabled"); + } + addresses.erase( + std::remove_if(addresses.begin(), addresses.end(), + [dns_resolve_ipv6, dns_resolve_ipv4](const Poco::Net::IPAddress& address) + { + return (address.family() == Poco::Net::IPAddress::IPv6 && !dns_resolve_ipv6) + || (address.family() == Poco::Net::IPAddress::IPv4 && !dns_resolve_ipv4); + }), + addresses.end() + ); + } + + void setSettings(bool dns_allow_resolve_names_to_ipv4_, bool dns_allow_resolve_names_to_ipv6_) + { + settings.dns_allow_resolve_names_to_ipv4 = dns_allow_resolve_names_to_ipv4_; + settings.dns_allow_resolve_names_to_ipv6 = dns_allow_resolve_names_to_ipv6_; + } + + DNSFilterSettings settings; +}; + + +DNSResolver::DNSResolver() + : impl(std::make_unique()) + , addressFilter(std::make_unique()) + , log(getLogger("DNSResolver")) {} + + +DNSResolver::IPAddresses DNSResolver::getResolvedIPAdressessWithFiltering(const std::string & host) +{ + auto addresses = resolveIPAddressImpl(host); + addressFilter->performAddressFiltering(addresses); + + if (addresses.empty()) + { + ProfileEvents::increment(ProfileEvents::DNSError); + throw DB::NetException(ErrorCodes::DNS_ERROR, "After filtering there are no resolved address for host({}).", host); + } + return addresses; +} + +DNSResolver::IPAddresses DNSResolver::resolveIPAddressWithCache(const std::string & host) +{ + auto [result, _ ] = impl->cache_host.getOrSet(host, [&host, this]() {return std::make_shared(getResolvedIPAdressessWithFiltering(host), std::chrono::system_clock::now());}); + return result->addresses; +} Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host) { return pickAddress(resolveHostAll(host)); // random order -> random pick } +void DNSResolver::setFilterSettings(bool dns_allow_resolve_names_to_ipv4_, bool dns_allow_resolve_names_to_ipv6_) +{ + addressFilter->setSettings(dns_allow_resolve_names_to_ipv4_, dns_allow_resolve_names_to_ipv6_); +} + DNSResolver::IPAddresses DNSResolver::resolveHostAllInOriginOrder(const std::string & host) { if (impl->disable_cache) - return resolveIPAddressImpl(host); + return getResolvedIPAdressessWithFiltering(host); addToNewHosts(host); - return resolveIPAddressWithCache(impl->cache_host, host); + return resolveIPAddressWithCache(host); } DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host) @@ -232,7 +297,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_an splitHostAndPort(host_and_port, host, port); addToNewHosts(host); - return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(impl->cache_host, host)), port); + return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(host)), port); } Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, UInt16 port) @@ -241,7 +306,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U return Poco::Net::SocketAddress(host, port); addToNewHosts(host); - return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(impl->cache_host, host)), port); + return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(host)), port); } std::vector DNSResolver::resolveAddressList(const std::string & host, UInt16 port) @@ -254,7 +319,7 @@ std::vector DNSResolver::resolveAddressList(const std: if (!impl->disable_cache) addToNewHosts(host); - std::vector ips = impl->disable_cache ? hostByName(host) : resolveIPAddressWithCache(impl->cache_host, host); + std::vector ips = impl->disable_cache ? hostByName(host) : resolveIPAddressWithCache(host); auto ips_end = std::unique(ips.begin(), ips.end()); addresses.reserve(ips_end - ips.begin()); @@ -419,8 +484,8 @@ bool DNSResolver::updateCache(UInt32 max_consecutive_failures) bool DNSResolver::updateHost(const String & host) { - const auto old_value = resolveIPAddressWithCache(impl->cache_host, host); - auto new_value = resolveIPAddressImpl(host); + const auto old_value = resolveIPAddressWithCache(host); + auto new_value = getResolvedIPAdressessWithFiltering(host); const bool result = old_value != new_value; impl->cache_host.set(host, std::make_shared(std::move(new_value), std::chrono::system_clock::now())); return result; diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 1ddd9d3b991..b35f55dfcd2 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -68,6 +68,8 @@ public: /// Returns true if IP of any host has been changed or an element was dropped (too many failures) bool updateCache(UInt32 max_consecutive_failures); + void setFilterSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6); + /// Returns a copy of cache entries std::vector> cacheEntries() const; @@ -86,6 +88,10 @@ private: struct Impl; std::unique_ptr impl; + + struct AddressFilter; + std::unique_ptr addressFilter; + LoggerPtr log; /// Updates cached value and returns true it has been changed. @@ -94,6 +100,9 @@ private: void addToNewHosts(const String & host); void addToNewAddresses(const Poco::Net::IPAddress & address); + + IPAddresses resolveIPAddressWithCache(const std::string & host); + IPAddresses getResolvedIPAdressessWithFiltering(const std::string & host); }; } diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index d13e6251ca9..6c23e3b95f6 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -106,6 +106,8 @@ namespace DB M(UInt64, dns_cache_max_entries, 10000, "Internal DNS cache max entries.", 0) \ M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \ + M(Bool, dns_allow_resolve_names_to_ipv4, true, "Allows resolve names to ipv4 addresses.", 0) \ + M(Bool, dns_allow_resolve_names_to_ipv6, true, "Allows resolve names to ipv6 addresses.", 0) \ \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0ccbd874a3d..a76e214b1fc 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -500,6 +500,267 @@ static std::initializer_list col >= '2023-01-01' AND col <= '2023-12-31')"}, + {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, + {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, + {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, + {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, + {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + }}, + {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, + {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, + {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, + {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, + {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, + {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, + {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, + {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, + {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, + {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, + {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, + {"enable_vertical_final", false, true, "Use vertical final by default"}, + {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, + {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, + {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, + {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, + {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, + {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, + {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, + {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, + {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, + {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, + {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, + {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, + {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, + {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, + {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, + {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, + {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, + {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, + {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, + {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, + {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, + {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, + {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, + {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, + {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, + {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, + {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, + {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, + {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, + {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, + {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, + {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, + {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, + {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, + {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, + {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, + {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, + {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, + {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, + {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, + {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, + {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, + {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, + {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, + {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, + {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, + {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, + {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, + {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, + {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, + {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, + {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, + {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, + {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, + {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, + {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, + {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, + {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, + {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, + {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, + {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, + {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, + {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, + {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, + {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, + {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, + {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, + {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, + {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, + {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, + {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, + {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, + {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, + {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, + {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, + {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, + {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, + {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, + {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, + {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, + {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, + {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, + {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, + {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, + {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, + {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, + {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, + {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, + {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, + {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, + {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; diff --git a/tests/integration/test_dns_cache/test.py b/tests/integration/test_dns_cache/test.py index a6db26c8575..5e120dc42aa 100644 --- a/tests/integration/test_dns_cache/test.py +++ b/tests/integration/test_dns_cache/test.py @@ -32,6 +32,7 @@ node2 = cluster.add_instance( main_configs=["configs/listen_host.xml", "configs/dns_update_long.xml"], with_zookeeper=True, ipv6_address="2001:3984:3989::1:1112", + ipv4_address="10.5.95.11", ) @@ -39,9 +40,6 @@ node2 = cluster.add_instance( def cluster_without_dns_cache_update(): try: cluster.start() - - _fill_nodes([node1, node2], "test_table_drop") - yield cluster except Exception as ex: @@ -59,6 +57,8 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): # In this case we should manually set up the static DNS entries on the source host # to exclude resplving addresses automatically added by docker. # We use ipv6 for hosts, but resolved DNS entries may contain an unexpected ipv4 address. + _fill_nodes([node1, node2], "test_table_drop") + node2.set_hosts([("2001:3984:3989::1:1111", "node1")]) # drop DNS cache node2.query("SYSTEM DROP DNS CACHE") @@ -98,6 +98,67 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): assert_eq_with_retry(node2, "SELECT count(*) from test_table_drop", "7") +def _render_filter_config(allow_ipv4, allow_ipv6): + config = f""" + + {int(allow_ipv4)} + {int(allow_ipv6)} + + """ + return config + + +@pytest.mark.parametrize( + "allow_ipv4, allow_ipv6", + [ + (True, False), + (False, True), + (False, False), + ], +) +def test_dns_resolver_filter(cluster_without_dns_cache_update, allow_ipv4, allow_ipv6): + host_ipv6 = node2.ipv6_address + host_ipv4 = node2.ipv4_address + + node2.set_hosts( + [ + (host_ipv6, "test_host"), + (host_ipv4, "test_host"), + ] + ) + node2.replace_config( + "/etc/clickhouse-server/config.d/dns_filter.xml", + _render_filter_config(allow_ipv4, allow_ipv6), + ) + + node2.query("SYSTEM DROP DNS CACHE") + node2.query("SYSTEM DROP CONNECTIONS CACHE") + node2.query("SYSTEM RELOAD CONFIG") + + if not allow_ipv4 and not allow_ipv6: + with pytest.raises(QueryRuntimeException): + node4.query("SELECT * FROM remote('lost_host', 'system', 'one')") + else: + node2.query("SELECT * FROM remote('test_host', system, one)") + assert ( + node2.query( + "SELECT ip_address FROM system.dns_cache WHERE hostname='test_host'" + ) + == f"{host_ipv4 if allow_ipv4 else host_ipv6}\n" + ) + + node2.exec_in_container( + [ + "bash", + "-c", + "rm /etc/clickhouse-server/config.d/dns_filter.xml", + ], + privileged=True, + user="root", + ) + node2.query("SYSTEM RELOAD CONFIG") + + node3 = cluster.add_instance( "node3", main_configs=["configs/listen_host.xml"], From 0d5cb9f75a527e90aed18860efbd5ed1f9dcd775 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 23 Jul 2024 09:25:02 +0000 Subject: [PATCH 0531/1722] Review fixes --- src/Common/DNSResolver.cpp | 34 ++++++++++++++--------------- src/Core/SettingsChangesHistory.cpp | 2 +- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 051e6e63091..08111d7f2af 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -12,8 +12,7 @@ #include #include #include -#include -#include +#include "Common/MultiVersion.h" #include #include "DNSPTRResolverProvider.h" @@ -198,14 +197,17 @@ struct DNSResolver::AddressFilter { struct DNSFilterSettings { - std::atomic dns_allow_resolve_names_to_ipv4{true}; - std::atomic dns_allow_resolve_names_to_ipv6{true}; + bool dns_allow_resolve_names_to_ipv4{true}; + bool dns_allow_resolve_names_to_ipv6{true}; }; + AddressFilter() : settings(std::make_unique()) {} + void performAddressFiltering(DNSResolver::IPAddresses & addresses) { - bool dns_resolve_ipv4 = settings.dns_allow_resolve_names_to_ipv4; - bool dns_resolve_ipv6 = settings.dns_allow_resolve_names_to_ipv6; + const auto current_settings = settings.get(); + bool dns_resolve_ipv4 = current_settings->dns_allow_resolve_names_to_ipv4; + bool dns_resolve_ipv6 = current_settings->dns_allow_resolve_names_to_ipv6; if (dns_resolve_ipv4 && dns_resolve_ipv6) { @@ -215,24 +217,20 @@ struct DNSResolver::AddressFilter { throw Exception(ErrorCodes::BAD_ARGUMENTS, "DNS can't resolve any address, because dns_resolve_ipv6_interfaces and dns_resolve_ipv4_interfaces both are disabled"); } - addresses.erase( - std::remove_if(addresses.begin(), addresses.end(), - [dns_resolve_ipv6, dns_resolve_ipv4](const Poco::Net::IPAddress& address) - { - return (address.family() == Poco::Net::IPAddress::IPv6 && !dns_resolve_ipv6) - || (address.family() == Poco::Net::IPAddress::IPv4 && !dns_resolve_ipv4); - }), - addresses.end() - ); + + std::erase_if(addresses, [dns_resolve_ipv6, dns_resolve_ipv4](const Poco::Net::IPAddress& address) + { + return (address.family() == Poco::Net::IPAddress::IPv6 && !dns_resolve_ipv6) + || (address.family() == Poco::Net::IPAddress::IPv4 && !dns_resolve_ipv4); + }); } void setSettings(bool dns_allow_resolve_names_to_ipv4_, bool dns_allow_resolve_names_to_ipv6_) { - settings.dns_allow_resolve_names_to_ipv4 = dns_allow_resolve_names_to_ipv4_; - settings.dns_allow_resolve_names_to_ipv6 = dns_allow_resolve_names_to_ipv6_; + settings.set(std::make_unique(dns_allow_resolve_names_to_ipv4_, dns_allow_resolve_names_to_ipv6_)); } - DNSFilterSettings settings; + MultiVersion settings; }; diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a76e214b1fc..01b9bca795f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -523,7 +523,7 @@ static std::initializer_list Date: Tue, 23 Jul 2024 12:58:50 +0000 Subject: [PATCH 0532/1722] Fix test --- src/Core/SettingsChangesHistory.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 01b9bca795f..ac427e2e03e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -523,9 +523,7 @@ static std::initializer_list Date: Tue, 23 Jul 2024 16:56:29 +0000 Subject: [PATCH 0533/1722] Fix tidy build --- src/Common/DNSResolver.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 08111d7f2af..bbee7d259f0 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -203,7 +203,7 @@ struct DNSResolver::AddressFilter AddressFilter() : settings(std::make_unique()) {} - void performAddressFiltering(DNSResolver::IPAddresses & addresses) + void performAddressFiltering(DNSResolver::IPAddresses & addresses) const { const auto current_settings = settings.get(); bool dns_resolve_ipv4 = current_settings->dns_allow_resolve_names_to_ipv4; From 304d01e2c36ae11cd1c703135c493ee885bc3062 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Mon, 29 Jul 2024 19:18:14 +0000 Subject: [PATCH 0534/1722] Review fix --- programs/server/Server.cpp | 1 - src/Common/DNSResolver.cpp | 8 ++++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3126f65ef09..aa7c3d75163 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1770,7 +1770,6 @@ try DNSResolver::instance().setFilterSettings(new_server_settings.dns_allow_resolve_names_to_ipv4, new_server_settings.dns_allow_resolve_names_to_ipv6); - if (global_context->isServerCompletelyStarted()) CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability); diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index bbee7d259f0..68a8fa7d74c 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -225,9 +225,9 @@ struct DNSResolver::AddressFilter }); } - void setSettings(bool dns_allow_resolve_names_to_ipv4_, bool dns_allow_resolve_names_to_ipv6_) + void setSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6) { - settings.set(std::make_unique(dns_allow_resolve_names_to_ipv4_, dns_allow_resolve_names_to_ipv6_)); + settings.set(std::make_unique(dns_allow_resolve_names_to_ipv4, dns_allow_resolve_names_to_ipv6)); } MultiVersion settings; @@ -264,9 +264,9 @@ Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host) return pickAddress(resolveHostAll(host)); // random order -> random pick } -void DNSResolver::setFilterSettings(bool dns_allow_resolve_names_to_ipv4_, bool dns_allow_resolve_names_to_ipv6_) +void DNSResolver::setFilterSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6) { - addressFilter->setSettings(dns_allow_resolve_names_to_ipv4_, dns_allow_resolve_names_to_ipv6_); + addressFilter->setSettings(dns_allow_resolve_names_to_ipv4, dns_allow_resolve_names_to_ipv6); } DNSResolver::IPAddresses DNSResolver::resolveHostAllInOriginOrder(const std::string & host) From 4143eea587b808086cceb98025906646fa78c96a Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 31 Jul 2024 08:35:38 +0000 Subject: [PATCH 0535/1722] Add test to skip parallel --- tests/integration/parallel_skip.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 99fa626bd1e..6689572aeb7 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -1,6 +1,7 @@ [ "test_dns_cache/test.py::test_dns_cache_update", "test_dns_cache/test.py::test_ip_change_drop_dns_cache", + "test_dns_cache/test.py::test_dns_resolver_filter", "test_dns_cache/test.py::test_ip_change_update_dns_cache", "test_dns_cache/test.py::test_user_access_ip_change[node0]", "test_dns_cache/test.py::test_user_access_ip_change[node1]", From 012ea3cc6d09c50f29fe4d7964aa18ee038c35b2 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 31 Jul 2024 13:29:36 +0000 Subject: [PATCH 0536/1722] Rebase --- src/Core/SettingsChangesHistory.cpp | 259 ----------------------- tests/integration/test_dns_cache/test.py | 138 ++++++------ 2 files changed, 74 insertions(+), 323 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ac427e2e03e..0ccbd874a3d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -500,265 +500,6 @@ static std::initializer_list col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; diff --git a/tests/integration/test_dns_cache/test.py b/tests/integration/test_dns_cache/test.py index 5e120dc42aa..36401517429 100644 --- a/tests/integration/test_dns_cache/test.py +++ b/tests/integration/test_dns_cache/test.py @@ -32,7 +32,6 @@ node2 = cluster.add_instance( main_configs=["configs/listen_host.xml", "configs/dns_update_long.xml"], with_zookeeper=True, ipv6_address="2001:3984:3989::1:1112", - ipv4_address="10.5.95.11", ) @@ -40,6 +39,9 @@ node2 = cluster.add_instance( def cluster_without_dns_cache_update(): try: cluster.start() + + _fill_nodes([node1, node2], "test_table_drop") + yield cluster except Exception as ex: @@ -57,8 +59,6 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): # In this case we should manually set up the static DNS entries on the source host # to exclude resplving addresses automatically added by docker. # We use ipv6 for hosts, but resolved DNS entries may contain an unexpected ipv4 address. - _fill_nodes([node1, node2], "test_table_drop") - node2.set_hosts([("2001:3984:3989::1:1111", "node1")]) # drop DNS cache node2.query("SYSTEM DROP DNS CACHE") @@ -98,67 +98,6 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): assert_eq_with_retry(node2, "SELECT count(*) from test_table_drop", "7") -def _render_filter_config(allow_ipv4, allow_ipv6): - config = f""" - - {int(allow_ipv4)} - {int(allow_ipv6)} - - """ - return config - - -@pytest.mark.parametrize( - "allow_ipv4, allow_ipv6", - [ - (True, False), - (False, True), - (False, False), - ], -) -def test_dns_resolver_filter(cluster_without_dns_cache_update, allow_ipv4, allow_ipv6): - host_ipv6 = node2.ipv6_address - host_ipv4 = node2.ipv4_address - - node2.set_hosts( - [ - (host_ipv6, "test_host"), - (host_ipv4, "test_host"), - ] - ) - node2.replace_config( - "/etc/clickhouse-server/config.d/dns_filter.xml", - _render_filter_config(allow_ipv4, allow_ipv6), - ) - - node2.query("SYSTEM DROP DNS CACHE") - node2.query("SYSTEM DROP CONNECTIONS CACHE") - node2.query("SYSTEM RELOAD CONFIG") - - if not allow_ipv4 and not allow_ipv6: - with pytest.raises(QueryRuntimeException): - node4.query("SELECT * FROM remote('lost_host', 'system', 'one')") - else: - node2.query("SELECT * FROM remote('test_host', system, one)") - assert ( - node2.query( - "SELECT ip_address FROM system.dns_cache WHERE hostname='test_host'" - ) - == f"{host_ipv4 if allow_ipv4 else host_ipv6}\n" - ) - - node2.exec_in_container( - [ - "bash", - "-c", - "rm /etc/clickhouse-server/config.d/dns_filter.xml", - ], - privileged=True, - user="root", - ) - node2.query("SYSTEM RELOAD CONFIG") - - node3 = cluster.add_instance( "node3", main_configs=["configs/listen_host.xml"], @@ -378,3 +317,74 @@ def test_host_is_drop_from_cache_after_consecutive_failures( assert node4.wait_for_log_line( "Cached hosts dropped:.*InvalidHostThatDoesNotExist.*" ) + + +node7 = cluster.add_instance( + "node7", + main_configs=["configs/listen_host.xml", "configs/dns_update_long.xml"], + with_zookeeper=True, + ipv6_address="2001:3984:3989::1:1117", + ipv4_address="10.5.95.17", +) + + +def _render_filter_config(allow_ipv4, allow_ipv6): + config = f""" + + {int(allow_ipv4)} + {int(allow_ipv6)} + + """ + return config + + +@pytest.mark.parametrize( + "allow_ipv4, allow_ipv6", + [ + (True, False), + (False, True), + (False, False), + ], +) +def test_dns_resolver_filter(cluster_without_dns_cache_update, allow_ipv4, allow_ipv6): + node = node7 + host_ipv6 = node.ipv6_address + host_ipv4 = node.ipv4_address + + node.set_hosts( + [ + (host_ipv6, "test_host"), + (host_ipv4, "test_host"), + ] + ) + node.replace_config( + "/etc/clickhouse-server/config.d/dns_filter.xml", + _render_filter_config(allow_ipv4, allow_ipv6), + ) + + node.query("SYSTEM RELOAD CONFIG") + node.query("SYSTEM DROP DNS CACHE") + node.query("SYSTEM DROP CONNECTIONS CACHE") + + if not allow_ipv4 and not allow_ipv6: + with pytest.raises(QueryRuntimeException): + node.query("SELECT * FROM remote('lost_host', 'system', 'one')") + else: + node.query("SELECT * FROM remote('test_host', system, one)") + assert ( + node.query( + "SELECT ip_address FROM system.dns_cache WHERE hostname='test_host'" + ) + == f"{host_ipv4 if allow_ipv4 else host_ipv6}\n" + ) + + node.exec_in_container( + [ + "bash", + "-c", + "rm /etc/clickhouse-server/config.d/dns_filter.xml", + ], + privileged=True, + user="root", + ) + node.query("SYSTEM RELOAD CONFIG") From d124de847b44344d9346c4d1b76ada03b31c58c8 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Aug 2024 16:06:59 +0000 Subject: [PATCH 0537/1722] Fix style --- tests/clickhouse-test | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index dea303ecdfb..c3b1d4d907c 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1959,15 +1959,17 @@ class TestSuite: ) -> Dict[str, Tuple[int, int]]: if not line.startswith(comment_sign): return {} - random_settings_limits_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 + random_settings_limits_str = line[len(comment_sign) :].lstrip() random_settings_limits_prefix = "Random settings limits:" if not random_settings_limits_str.startswith(random_settings_limits_prefix): return {} - random_settings_limits_str = random_settings_limits_str[len(random_settings_limits_prefix) :] # noqa: ignore E203 + random_settings_limits_str = random_settings_limits_str[ + len(random_settings_limits_prefix) : + ] # limits are specified in a form 'setting1=(min, max); setting2=(min,max); ...' random_settings_limits = {} - for setting_and_limit in random_settings_limits_str.split(';'): - setting_and_limit = setting_and_limit.split('=') + for setting_and_limit in random_settings_limits_str.split(";"): + setting_and_limit = setting_and_limit.split("=") random_settings_limits[setting_and_limit[0].strip()] = make_tuple( setting_and_limit[1] ) @@ -2014,7 +2016,12 @@ class TestSuite: all_random_settings_limits = {} start_time = datetime.now() for test_name in all_tests: - tags, random_settings_limits = load_tags_and_random_settings_limits_from_file(os.path.join(suite_dir, test_name)) # noqa: ignore E203 + ( + tags, + random_settings_limits, + ) = load_tags_and_random_settings_limits_from_file( + os.path.join(suite_dir, test_name) + ) # noqa: ignore E203 if tags: all_tags[test_name] = tags if random_settings_limits: @@ -2061,7 +2068,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = all_tags_and_random_settings_limits[1] # noqa: ignore E203 + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] From 0ebe8e35511f764b61cb2428433132644f7deb96 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 6 Aug 2024 18:38:23 +0200 Subject: [PATCH 0538/1722] Fix style --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e6949dd4fba..1ebc9b07748 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,7 +79,7 @@ static std::initializer_list Date: Tue, 6 Aug 2024 15:28:45 +0000 Subject: [PATCH 0539/1722] better on-fly mutations --- .../Optimizations/projectionsCommon.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 107 +++++++++++------- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + src/Storages/MergeTree/AlterConversions.h | 13 ++- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 18 +-- src/Storages/MergeTree/MergeTreeData.h | 13 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 + .../MergeTree/MergeTreeDataSelectExecutor.h | 1 + .../MergeTree/MergeTreePrefetchedReadPool.cpp | 1 - .../MergeTree/MergeTreePrefetchedReadPool.h | 2 +- .../MergeTree/MergeTreeReadPoolBase.cpp | 8 +- .../MergeTree/MergeTreeReadPoolBase.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 25 +++- .../MergeTree/ReplicatedMergeTreeQueue.h | 1 + src/Storages/MutationCommands.cpp | 9 ++ src/Storages/MutationCommands.h | 1 + src/Storages/StorageMergeTree.cpp | 22 +++- src/Storages/StorageMergeTree.h | 1 + 20 files changed, 156 insertions(+), 76 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index e5f9c3b23ea..998b606ec57 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -243,6 +243,7 @@ bool analyzeProjectionCandidate( auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), + reading.getMutationsSnapshot()->cloneEmpty(), required_column_names, candidate.projection->metadata, projection_query_info, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7d6e650cba0..de29e7a9d5a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1373,6 +1373,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge { return selectRangesToRead( std::move(parts), + mutations_snapshot, metadata_for_reading, query_info, context, @@ -1390,9 +1391,11 @@ static void buildIndexes( const ActionsDAG * filter_actions_dag, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, + const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot, const ContextPtr & context, const SelectQueryInfo & query_info, - const StorageMetadataPtr & metadata_snapshot) + const StorageMetadataPtr & metadata_snapshot, + const LoggerPtr & log) { indexes.reset(); @@ -1418,19 +1421,21 @@ static void buildIndexes( indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } - indexes->part_values - = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; - bool final = query_info.isFinal(); - - if (final && !settings.use_skip_indexes_if_final) + if (query_info.isFinal() && !settings.use_skip_indexes_if_final) indexes->use_skip_indexes = false; if (!indexes->use_skip_indexes) return; + const auto & all_indexes = metadata_snapshot->getSecondaryIndices(); + + if (all_indexes.empty()) + return; + std::unordered_set ignored_index_names; if (settings.ignore_data_skipping_indices.changed) @@ -1455,49 +1460,68 @@ static void buildIndexes( throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); } + auto all_updated_columns = mutations_snapshot->getAllUpdatedColumns(); + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; - for (const auto & index : metadata_snapshot->getSecondaryIndices()) + for (const auto & index : all_indexes) { - if (!ignored_index_names.contains(index.name)) + if (ignored_index_names.contains(index.name)) + continue; + + auto index_helper = MergeTreeIndexFactory::instance().get(index); + + if (!all_updated_columns.empty()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) + auto required_columns = index_helper->getColumnsRequiredForIndexCalc(); + auto it = std::ranges::find_if(required_columns, [&](const auto & column_name) { - auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); - if (inserted) - { - skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - } + return all_updated_columns.contains(column_name); + }); - skip_indexes.merged_indices[it->second].addIndex(index_helper); - } - else + if (it != required_columns.end()) { - MergeTreeIndexConditionPtr condition; - if (index_helper->isVectorSearch()) - { -#ifdef ENABLE_ANNOY - if (const auto * annoy = typeid_cast(index_helper.get())) - condition = annoy->createIndexCondition(query_info, context); -#endif -#ifdef ENABLE_USEARCH - if (const auto * usearch = typeid_cast(index_helper.get())) - condition = usearch->createIndexCondition(query_info, context); -#endif - if (!condition) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); - } - else - condition = index_helper->createIndexCondition(filter_actions_dag, context); - - if (!condition->alwaysUnknownOrTrue()) - skip_indexes.useful_indices.emplace_back(index_helper, condition); + LOG_TRACE(log, "Index {} is not used because it depends on column {} which will be updated on fly", index.name, *it); + continue; } } + + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + } + + skip_indexes.merged_indices[it->second].addIndex(index_helper); + continue; + } + + MergeTreeIndexConditionPtr condition; + if (index_helper->isVectorSearch()) + { +#ifdef ENABLE_ANNOY + if (const auto * annoy = typeid_cast(index_helper.get())) + condition = annoy->createIndexCondition(query_info, context); +#endif +#ifdef ENABLE_USEARCH + if (const auto * usearch = typeid_cast(index_helper.get())) + condition = usearch->createIndexCondition(query_info, context); +#endif + if (!condition) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); + } + else + { + condition = index_helper->createIndexCondition(filter_actions_dag, context); + } + + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); } // move minmax indices to first positions, so they will be applied first as cheapest ones @@ -1535,14 +1559,17 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) query_info.filter_actions_dag.get(), data, prepared_parts, + mutations_snapshot, context, query_info, - metadata_for_reading); + metadata_for_reading, + log); } } ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, ContextPtr context_, @@ -1573,7 +1600,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & primary_key_column_names = primary_key.column_names; if (!indexes) - buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot); + buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, mutations_snapshot, context_, query_info_, metadata_snapshot, log); if (indexes->part_values && indexes->part_values->empty()) return std::make_shared(std::move(result)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 10b9e92d99b..e441eda7505 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -154,6 +154,7 @@ public: static AnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index cee23bc4efc..046cc1d2491 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include namespace DB @@ -11,11 +11,17 @@ namespace DB /// Alter conversions which should be applied on-fly for part. /// Built from of the most recent mutation commands for part. /// Now only ALTER RENAME COLUMN is applied. -class AlterConversions : private boost::noncopyable +class AlterConversions : private WithContext, boost::noncopyable { public: AlterConversions() = default; + AlterConversions(StorageMetadataPtr metadata_snapshot_, ContextPtr context_) + : WithContext(context_) + , metadata_snapshot(std::move(metadata_snapshot_)) + { + } + struct RenamePair { std::string rename_to; @@ -40,6 +46,7 @@ public: private: /// Rename map new_name -> old_name. std::vector rename_map; + StorageMetadataPtr metadata_snapshot; }; using AlterConversionsPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e9a48c655e8..ea4b9261af8 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -301,7 +301,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part_infos); } - global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot)); + global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot, global_ctx->metadata_snapshot, global_ctx->context)); } const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3852f282f65..32c0251ef53 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7137,11 +7137,11 @@ UInt64 MergeTreeData::estimateNumberOfRowsToRead( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; MergeTreeDataSelectExecutor reader(*this); auto result_ptr = reader.estimateNumMarksToRead( - parts, + snapshot_data.parts, + snapshot_data.mutations_snapshot, storage_snapshot->getMetadataForQuery()->getColumns().getAll().getNames(), storage_snapshot->metadata, query_info, @@ -8162,10 +8162,12 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S AlterConversionsPtr MergeTreeData::getAlterConversionsForPart( const MergeTreeDataPartPtr & part, - const MutationsSnapshotPtr & snapshot) + const MutationsSnapshotPtr & mutations, + const StorageMetadataPtr & metadata, + const ContextPtr & query_context) { - auto commands = snapshot->getAlterMutationCommandsForPart(part); - auto result = std::make_shared(); + auto commands = mutations->getAlterMutationCommandsForPart(part); + auto result = std::make_shared(metadata, query_context); for (const auto & command : commands | std::views::reverse) result->addMutationCommand(command); @@ -8758,8 +8760,7 @@ static void updateMutationsCounters( void incrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) + const MutationCommands & commands) { return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, 1); } @@ -8767,8 +8768,7 @@ void incrementMutationsCounters( void decrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) + const MutationCommands & commands) { return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, -1); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b14121e0c78..dc37d5e7dad 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -475,6 +475,8 @@ public: /// @return list of mutation commands, in *reverse* order (newest to oldest) virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; virtual std::shared_ptr cloneEmpty() const = 0; + virtual NameSet getAllUpdatedColumns() const = 0; + bool hasDataMutations() const { return params.need_data_mutations && info.num_data_mutations > 0; } virtual ~IMutationsSnapshot() = default; @@ -975,7 +977,9 @@ public: /// Return alter conversions for part which must be applied on fly. static AlterConversionsPtr getAlterConversionsForPart( const MergeTreeDataPartPtr & part, - const MutationsSnapshotPtr & snapshot); + const MutationsSnapshotPtr & mutations, + const StorageMetadataPtr & metadata, + const ContextPtr & query_context); /// Returns destination disk or volume for the TTL rule according to current storage policy. SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const; @@ -1769,17 +1773,14 @@ struct CurrentlySubmergingEmergingTagger }; /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. -/// Return true if the counter had been updated void incrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); + const MutationCommands & commands); void decrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); + const MutationCommands & commands); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e294def040b..2d9a5c6084c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -884,6 +884,7 @@ std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -898,6 +899,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar std::optional indexes; return ReadFromMergeTree::selectRangesToRead( std::move(parts), + mutations_snapshot, metadata_snapshot, query_info, context, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8e4c55f2c1d..3668eb0ad90 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -56,6 +56,7 @@ public: /// This method is used to select best projection for table. ReadFromMergeTree::AnalysisResultPtr estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index faed543420c..09bbf33ba9b 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -105,7 +105,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( column_names_, settings_, context_) - , WithContext(context_) , prefetch_threadpool(getContext()->getPrefetchThreadpool()) , log(getLogger("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) { diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 65a7d62ad2d..1a709250937 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -14,7 +14,7 @@ using MergeTreeReaderPtr = std::unique_ptr; /// A class which is responsible for creating read tasks /// which are later taken by readers via getTask method. /// Does prefetching for the read tasks it creates. -class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithContext +class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase { public: MergeTreePrefetchedReadPool( diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index fce8d649617..021b340d746 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -4,9 +4,6 @@ #include #include -#include - - namespace DB { @@ -26,7 +23,8 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const Names & column_names_, const PoolSettings & pool_settings_, const ContextPtr & context_) - : parts_ranges(std::move(parts_)) + : WithContext(context_) + , parts_ranges(std::move(parts_)) , mutations_snapshot(std::move(mutations_snapshot_)) , shared_virtual_fields(std::move(shared_virtual_fields_)) , storage_snapshot(storage_snapshot_) @@ -121,7 +119,7 @@ void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings) } read_task_info.part_index_in_query = part_with_ranges.part_index_in_query; - read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot); + read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot, storage_snapshot->metadata, getContext()); LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, read_task_info.alter_conversions); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 7b4e034d892..7f9106d476e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -6,7 +6,7 @@ namespace DB { -class MergeTreeReadPoolBase : public IMergeTreeReadPool +class MergeTreeReadPoolBase : public IMergeTreeReadPool, protected WithContext { public: using MutationsSnapshotPtr = MergeTreeData::MutationsSnapshotPtr; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a95cccdc0d2..86e63782c18 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2125,7 +2125,7 @@ bool MutateTask::prepare() }; auto mutations_snapshot = ctx->data->getMutationsSnapshot(params); - auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->context); auto context_for_reading = Context::createCopy(ctx->context); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c3d91ca0705..8661b2aa784 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -950,7 +950,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, state_lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands); } else it = mutations_by_znode.erase(it); @@ -1000,7 +1000,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; - incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) @@ -1961,6 +1961,23 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo return result; } +NameSet ReplicatedMergeTreeQueue::MutationsSnapshot::getAllUpdatedColumns() const +{ + if (!params.need_data_mutations) + return {}; + + NameSet res; + for (const auto & [partition_id, mutations] : mutations_by_partition) + { + for (const auto & [version, entry] : mutations) + { + auto names = entry->commands.getAllUpdatedColumns(); + std::move(names.begin(), names.end(), std::inserter(res, res.end())); + } + } + return res; +} + MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const { std::lock_guard lock(state_mutex); @@ -2122,7 +2139,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands); } else if (mutation.parts_to_do.size() == 0) { @@ -2179,7 +2196,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index af8e2521f81..91a23b6a3b6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -426,6 +426,7 @@ public: MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + NameSet getAllUpdatedColumns() const override; }; /// Return mutation commands for part which could be not applied to diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f736c863eee..1aa9f5e23f8 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -268,4 +268,13 @@ bool MutationCommands::containBarrierCommand() const return false; } +NameSet MutationCommands::getAllUpdatedColumns() const +{ + NameSet res; + for (const auto & command : *this) + for (const auto & [column_name, _] : command.column_to_update_expression) + res.insert(column_name); + return res; +} + } diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index f999aab1f4d..5ae537bb657 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -92,6 +92,7 @@ public: /// stick with other commands. Commands from one set have already been validated /// to be executed without issues on the creation state. bool containBarrierCommand() const; + NameSet getAllUpdatedColumns() const; }; using MutationCommandsConstPtr = std::shared_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index da782b3634a..19cb2bec2e7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -519,7 +519,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands); } LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); @@ -556,7 +556,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands); } } else @@ -838,7 +838,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands); } to_kill.emplace(std::move(it->second)); @@ -923,7 +923,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2464,6 +2464,20 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo return result; } +NameSet StorageMergeTree::MutationsSnapshot::getAllUpdatedColumns() const +{ + if (!params.need_data_mutations) + return {}; + + NameSet res; + for (const auto & [version, commands] : mutations_by_version) + { + auto names = commands->getAllUpdatedColumns(); + std::move(names.begin(), names.end(), std::inserter(res, res.end())); + } + return res; +} + MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { std::lock_guard lock(currently_processing_in_background_mutex); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8ea0db37528..ef333fe3f18 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -320,6 +320,7 @@ private: MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + NameSet getAllUpdatedColumns() const override; }; MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; From 5c4f4f85036838a7f145139f36949592720a1289 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 6 Aug 2024 18:52:29 +0200 Subject: [PATCH 0540/1722] do not add to custom disk names --- src/Disks/DiskFomAST.cpp | 88 +++++++++++------------------------- src/Disks/DiskSelector.h | 2 +- src/Disks/StoragePolicy.h | 1 - src/Interpreters/Context.cpp | 5 +- 4 files changed, 29 insertions(+), 67 deletions(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index c8a4f88547f..2a5e7368de9 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_DISK; } @@ -27,8 +26,6 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string { Poco::Util::AbstractConfiguration::Keys disk_settings_keys; config->keys(disk_settings_keys); - - // Check that no settings are defined when disk from the config is referred. if (disk_settings_keys.empty()) throw Exception( @@ -36,74 +33,48 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string "Disk function has no arguments. Invalid disk description."); if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) - { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Disk function `{}` has to have the other arguments which describe the disk. Invalid disk description.", serialization); - } + + auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); std::string disk_name; if (config->has("name")) { disk_name = config->getString("name"); } - - if (!disk_name.empty()) - { - if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` could not start with `{}`", - disk_name, DiskSelector::CUSTOM_DISK_PREFIX); - - if (auto disk = context->tryGetDisk(disk_name)) - { - /// the disk is defined by config - if (disk->isCustomDisk()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Disk with name `{}` already exist as a custom disk but the name does not start with `{}`", - disk_name, - DiskSelector::CUSTOM_DISK_PREFIX); - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk `{}` is already exist. It is impossible to redefine it.", disk_name); - } - } - - auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); - - std::string custom_disk_name; - if (disk_name.empty()) + else { /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk /// configuration serialized ast as a disk name suffix. - custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + "noname_" + toString(disk_settings_hash); - } - else - { - custom_disk_name = toString(DiskSelector::CUSTOM_DISK_PREFIX) + disk_name; + disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(disk_settings_hash); } - auto result_disk = context->getOrCreateDisk(custom_disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - auto disk = DiskFactory::instance().create( + + auto disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + auto result = DiskFactory::instance().create( disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true); /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(disk_settings_hash); - return disk; + result->markDiskAsCustom(disk_settings_hash); + return result; }); - if (!result_disk->isCustomDisk()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk with name `{}` expected to be custom disk", disk_name); + if (!disk->isCustomDisk()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The disk `{}` is already exist and described by the config." + " It is impossible to redefine it.", + disk_name); - if (result_disk->getCustomDiskSettings() != disk_settings_hash && !attach) + if (disk->getCustomDiskSettings() != disk_settings_hash && !attach) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The disk `{}` is already configured as a custom disk in another table. It can't be redefined with different settings.", disk_name); - if (!attach && !result_disk->isRemote()) + if (!attach && !disk->isRemote()) { static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory"; auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, ""); @@ -114,14 +85,14 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string "Base path for custom local disks must be defined in config file by `{}`", custom_local_disks_base_dir_in_config); - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + if (!pathStartsWith(disk->getPath(), disk_path_expected_prefix)) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Path of the custom local disk must be inside `{}` directory", disk_path_expected_prefix); } - return custom_disk_name; + return disk_name; } class DiskConfigurationFlattener @@ -168,22 +139,17 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, ContextPtr context) { - if (disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` could not start with `{}`", - disk_name, DiskSelector::CUSTOM_DISK_PREFIX); - if (auto result = context->tryGetDisk(disk_name)) - return disk_name; + { + if (result->isCustomDisk()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` is a custom disk that is used in other table." + "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", + disk_name); - std::string custom_disk_name = DiskSelector::CUSTOM_DISK_PREFIX + disk_name; - if (auto result = context->tryGetDisk(custom_disk_name)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` is a custom disk that is used in other table." - "That disk could not be used by a reference. The custom disk should be fully specified with a disk function.", - disk_name); + return disk_name; + } throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", disk_name); } diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 0f7424460a2..e6e2c257911 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -20,7 +20,7 @@ using DiskSelectorPtr = std::shared_ptr; class DiskSelector { public: - static constexpr auto CUSTOM_DISK_PREFIX = "__"; + static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } DiskSelector(const DiskSelector & from) = default; diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index e23598214b3..ccf2e2071b2 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -120,7 +120,6 @@ class StoragePolicySelector { public: static constexpr auto TMP_STORAGE_POLICY_PREFIX = "__"; - static_assert(std::string_view(DiskSelector::CUSTOM_DISK_PREFIX) == std::string_view(TMP_STORAGE_POLICY_PREFIX)); StoragePolicySelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0acbef26805..30f77f799e9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4,7 +4,6 @@ #include #include #include -#include "Common/Logger.h" #include #include #include @@ -4432,11 +4431,9 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & disk_name) const { - LOG_DEBUG(getLogger("StoragePolicy"), "getStoragePolicyFromDisk disk_name {}", disk_name); - std::lock_guard lock(shared->storage_policies_mutex); - const std::string storage_policy_name = disk_name.starts_with(DiskSelector::CUSTOM_DISK_PREFIX) ? disk_name : StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; + const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; auto storage_policy_selector = getStoragePolicySelector(lock); StoragePolicyPtr storage_policy = storage_policy_selector->tryGet(storage_policy_name); From 586b1df3ee3f683e61c92e55d85e31f163eaa5af Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 6 Aug 2024 19:38:54 +0200 Subject: [PATCH 0541/1722] Add keeper error description to the message --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2d826c6c2df..b85c7145a91 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -800,7 +800,8 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() { auto res = future.get(); if (res.error != Coordination::Error::ZOK && res.error != Coordination::Error::ZNODEEXISTS) - throw Coordination::Exception(res.error, "Failed to create new nodes {} at {}", res.path_created, zookeeper_path); + throw Coordination::Exception(res.error, "Failed to create new nodes {} at {} with error {}", + res.path_created, zookeeper_path, Coordination::errorMessage(res.error)); } } From 04438784e2178820537c65b66b5a8341f3d63b8d Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 6 Aug 2024 16:45:46 -0400 Subject: [PATCH 0542/1722] add a stateless test for `grant current grants` --- .../03215_grant_current_grants.reference | 2 ++ .../0_stateless/03215_grant_current_grants.sh | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/03215_grant_current_grants.reference create mode 100755 tests/queries/0_stateless/03215_grant_current_grants.sh diff --git a/tests/queries/0_stateless/03215_grant_current_grants.reference b/tests/queries/0_stateless/03215_grant_current_grants.reference new file mode 100644 index 00000000000..e4f6850b806 --- /dev/null +++ b/tests/queries/0_stateless/03215_grant_current_grants.reference @@ -0,0 +1,2 @@ +GRANT SELECT, CREATE TABLE, CREATE VIEW ON default.* +GRANT SELECT ON default.* diff --git a/tests/queries/0_stateless/03215_grant_current_grants.sh b/tests/queries/0_stateless/03215_grant_current_grants.sh new file mode 100755 index 00000000000..68af4a62bba --- /dev/null +++ b/tests/queries/0_stateless/03215_grant_current_grants.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +user1="user03215_1_${CLICKHOUSE_DATABASE}_$RANDOM" +user2="user03215_2_${CLICKHOUSE_DATABASE}_$RANDOM" +user3="user03215_3_${CLICKHOUSE_DATABASE}_$RANDOM" +db=${CLICKHOUSE_DATABASE} + + +${CLICKHOUSE_CLIENT} --query "CREATE USER $user1, $user2, $user3;"; +${CLICKHOUSE_CLIENT} --query "GRANT SELECT, CREATE TABLE, CREATE VIEW ON $db.* TO $user1 WITH GRANT OPTION;"; + +${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON $db.* TO $user2" --user $user1; +${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON $db.* TO $user3" --user $user2; + +${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR $user2" | sed 's/ TO.*//'; +${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR $user3" | sed 's/ TO.*//'; + +${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS(SELECT ON $db.*) TO $user3" --user $user1; +${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR $user3" | sed 's/ TO.*//'; + +${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS $user1, $user2, $user3"; From 5ae5cd35b5b263d14bdd62aa5cbaa1e22219208a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Aug 2024 21:50:31 +0100 Subject: [PATCH 0543/1722] update --- base/poco/Net/include/Poco/Net/HTTPServerSession.h | 4 ++-- src/Server/HTTP/sendExceptionToHTTPClient.cpp | 2 +- .../0_stateless/00408_http_keep_alive.reference | 6 +++--- tests/queries/0_stateless/00408_http_keep_alive.sh | 7 ++++--- tests/queries/0_stateless/00501_http_head.re | 12 ++++++++++++ tests/queries/0_stateless/00501_http_head.reference | 4 ++-- tests/queries/0_stateless/00501_http_head.sh | 5 +++-- 7 files changed, 27 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/00501_http_head.re diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index 93f31012336..54e7f2c8c50 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -57,10 +57,10 @@ namespace Net /// Returns the server's address. void setKeepAliveTimeout(Poco::Timespan keepAliveTimeout); - + size_t getKeepAliveTimeout() const { return _keepAliveTimeout.totalSeconds(); } - size_t getMaxKeepAliveRequests() const { return _maxKeepAliveRequests; } + size_t getMaxKeepAliveRequests() const { return _maxKeepAliveRequests; } private: bool _firstRequest; diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.cpp b/src/Server/HTTP/sendExceptionToHTTPClient.cpp index 022a763a9a2..658b7a4707a 100644 --- a/src/Server/HTTP/sendExceptionToHTTPClient.cpp +++ b/src/Server/HTTP/sendExceptionToHTTPClient.cpp @@ -29,7 +29,7 @@ void sendExceptionToHTTPClient( if (!out) { /// If nothing was sent yet. - WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT}; + WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD}; out_for_message.writeln(exception_message); out_for_message.finalize(); diff --git a/tests/queries/0_stateless/00408_http_keep_alive.reference b/tests/queries/0_stateless/00408_http_keep_alive.reference index d5d7dacce9e..5402036bfd7 100644 --- a/tests/queries/0_stateless/00408_http_keep_alive.reference +++ b/tests/queries/0_stateless/00408_http_keep_alive.reference @@ -1,6 +1,6 @@ < Connection: Keep-Alive -< Keep-Alive: timeout=10, max=10000 +< Keep-Alive: timeout=10, max=? < Connection: Keep-Alive -< Keep-Alive: timeout=10, max=10000 +< Keep-Alive: timeout=10, max=? < Connection: Keep-Alive -< Keep-Alive: timeout=10, max=10000 +< Keep-Alive: timeout=10, max=? diff --git a/tests/queries/0_stateless/00408_http_keep_alive.sh b/tests/queries/0_stateless/00408_http_keep_alive.sh index 4bd0e494eb8..4a1cb4ed712 100755 --- a/tests/queries/0_stateless/00408_http_keep_alive.sh +++ b/tests/queries/0_stateless/00408_http_keep_alive.sh @@ -6,9 +6,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/" -${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< "SELECT 1" 2>&1 | perl -lnE 'print if /Keep-Alive/'; -${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< " error here " 2>&1 | perl -lnE 'print if /Keep-Alive/'; -${CLICKHOUSE_CURL} -vsS "${URL}"ping 2>&1 | perl -lnE 'print if /Keep-Alive/'; +# the sed command here replaces the real number of left requests with a question mark, because it can vary and we don't really have control over it +${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< "SELECT 1" 2>&1 | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -i 'keep-alive'; +${CLICKHOUSE_CURL} -vsS "${URL}" --data-binary @- <<< " error here " 2>&1 | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -i 'keep-alive'; +${CLICKHOUSE_CURL} -vsS "${URL}"ping 2>&1 | perl -lnE 'print if /Keep-Alive/' | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -i 'keep-alive'; # no keep-alive: ${CLICKHOUSE_CURL} -vsS "${URL}"404/not/found/ 2>&1 | perl -lnE 'print if /Keep-Alive/'; diff --git a/tests/queries/0_stateless/00501_http_head.re b/tests/queries/0_stateless/00501_http_head.re new file mode 100644 index 00000000000..807bcd4922e --- /dev/null +++ b/tests/queries/0_stateless/00501_http_head.re @@ -0,0 +1,12 @@ +HTTP/1.1 200 OK +Connection: Keep-Alive +Content-Type: text/tab-separated-values; charset=UTF-8 +Transfer-Encoding: chunked +Keep-Alive: timeout=10, max=? + +HTTP/1.1 200 OK +Connection: Keep-Alive +Content-Type: text/tab-separated-values; charset=UTF-8 +Transfer-Encoding: chunked +Keep-Alive: timeout=10, max=? + diff --git a/tests/queries/0_stateless/00501_http_head.reference b/tests/queries/0_stateless/00501_http_head.reference index db82132b145..807bcd4922e 100644 --- a/tests/queries/0_stateless/00501_http_head.reference +++ b/tests/queries/0_stateless/00501_http_head.reference @@ -2,11 +2,11 @@ HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=10000 +Keep-Alive: timeout=10, max=? HTTP/1.1 200 OK Connection: Keep-Alive Content-Type: text/tab-separated-values; charset=UTF-8 Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=10000 +Keep-Alive: timeout=10, max=? diff --git a/tests/queries/0_stateless/00501_http_head.sh b/tests/queries/0_stateless/00501_http_head.sh index 60283f26833..30da64c31f0 100755 --- a/tests/queries/0_stateless/00501_http_head.sh +++ b/tests/queries/0_stateless/00501_http_head.sh @@ -4,8 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -( ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=SELECT%201"; - ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "X-ClickHouse-Query-Id:" | grep -v "X-ClickHouse-Format:" | grep -v "X-ClickHouse-Timezone:" +# the sed command here replaces the real number of left requests with a question mark, because it can vary and we don't really have control over it +( ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=SELECT%201" | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I'; + ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}&query=select+*+from+system.numbers+limit+1000000" ) | sed -r 's/(keep-alive: timeout=10, max=)[0-9]+/\1?/I' | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "X-ClickHouse-Query-Id:" | grep -v "X-ClickHouse-Format:" | grep -v "X-ClickHouse-Timezone:" if [[ $(${CLICKHOUSE_CURL} -sS -X POST -I "${CLICKHOUSE_URL}&query=SELECT+1" | grep -c '411 Length Required') -ne 1 ]]; then echo FAIL From 1f5c4101b2d74d7ccf798621083fb536bf35de18 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Aug 2024 21:54:15 +0100 Subject: [PATCH 0544/1722] rm redundant file --- tests/queries/0_stateless/00501_http_head.re | 12 ------------ 1 file changed, 12 deletions(-) delete mode 100644 tests/queries/0_stateless/00501_http_head.re diff --git a/tests/queries/0_stateless/00501_http_head.re b/tests/queries/0_stateless/00501_http_head.re deleted file mode 100644 index 807bcd4922e..00000000000 --- a/tests/queries/0_stateless/00501_http_head.re +++ /dev/null @@ -1,12 +0,0 @@ -HTTP/1.1 200 OK -Connection: Keep-Alive -Content-Type: text/tab-separated-values; charset=UTF-8 -Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=? - -HTTP/1.1 200 OK -Connection: Keep-Alive -Content-Type: text/tab-separated-values; charset=UTF-8 -Transfer-Encoding: chunked -Keep-Alive: timeout=10, max=? - From 6f6056477785842c047ab0faa4c089a514625f06 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 7 Aug 2024 02:28:13 +0000 Subject: [PATCH 0545/1722] fix build --- src/Storages/MergeTree/MergeTask.cpp | 12 +++++++----- src/Storages/MergeTree/MergeTask.h | 4 ++-- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 2e7cc03f5ed..e50a66c91ec 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -501,11 +501,13 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) { const auto & projection = *global_ctx->projections_to_rebuild[i]; - auto projection_block = ctx->projection_squashes[i].add(projection.calculate(block, global_ctx->context)); - if (projection_block) + Block block_to_squash = projection.calculate(block, global_ctx->context); + auto chunk = ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); + if (chunk) { + auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *global_ctx->data, ctx->log, projection_block, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); ctx->projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); @@ -1313,9 +1315,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe const auto & settings = global_ctx->context->getSettingsRef(); - for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + for (auto projection : global_ctx->projections_to_rebuild) { - ctx->projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + ctx->projection_squashes.emplace_back(projection->sample_block, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 0424b698f8a..f60a32252b2 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include @@ -231,7 +231,7 @@ private: using ProjectionNameToItsBlocks = std::map; ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; - std::vector projection_squashes; + std::vector projection_squashes; size_t projection_block_num = 0; ExecutableTaskPtr merge_projection_parts_task_ptr; From f0f1015162ddfb7ef2e8046fa9c952c5c7e6b89a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 7 Aug 2024 10:47:13 +0800 Subject: [PATCH 0546/1722] fix style --- src/Functions/array/array.cpp | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Functions/array/array.cpp b/src/Functions/array/array.cpp index 21b1841f3ed..f193acafe06 100644 --- a/src/Functions/array/array.cpp +++ b/src/Functions/array/array.cpp @@ -172,9 +172,9 @@ private: concrete_columns[i] = concrete_column; } - ColumnString & concreate_out_data = assert_cast(out_data); - auto & out_chars = concreate_out_data.getChars(); - auto & out_offsets = concreate_out_data.getOffsets(); + ColumnString & concrete_out_data = assert_cast(out_data); + auto & out_chars = concrete_out_data.getChars(); + auto & out_offsets = concrete_out_data.getOffsets(); out_chars.resize_exact(total_bytes); out_offsets.resize_exact(input_rows_count * columns.size()); @@ -197,7 +197,6 @@ private: bool executeFixedString(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const { - size_t total_bytes = 0; std::vector concrete_columns(columns.size(), nullptr); for (size_t i = 0; i < columns.size(); ++i) { @@ -205,15 +204,16 @@ private: if (!concrete_column) return false; - total_bytes += concrete_column->getChars().size(); concrete_columns[i] = concrete_column; } - ColumnFixedString & concreate_out_data = assert_cast(out_data); - auto & out_chars = concreate_out_data.getChars(); + ColumnFixedString & concrete_out_data = assert_cast(out_data); + auto & out_chars = concrete_out_data.getChars(); + + const size_t n = concrete_out_data.getN(); + size_t total_bytes = concrete_out_data.getN() * columns.size() * input_rows_count; out_chars.resize_exact(total_bytes); - const size_t n = concreate_out_data.getN(); size_t curr_out_offset = 0; for (size_t row_i = 0; row_i < input_rows_count; ++row_i) { @@ -241,9 +241,9 @@ private: nested_columns[i] = &concrete_column->getNestedColumn(); } - ColumnNullable & concreate_out_data = assert_cast(out_data); - auto & out_null_map = concreate_out_data.getNullMapColumn(); - auto & out_nested_column = concreate_out_data.getNestedColumn(); + ColumnNullable & concrete_out_data = assert_cast(out_data); + auto & out_null_map = concrete_out_data.getNullMapColumn(); + auto & out_nested_column = concrete_out_data.getNestedColumn(); executeAny(null_maps, out_null_map, input_rows_count); executeAny(nested_columns, out_nested_column, input_rows_count); return true; @@ -251,11 +251,11 @@ private: bool executeTuple(const ColumnRawPtrs & columns, IColumn & out_data, size_t input_rows_count) const { - ColumnTuple * concreate_out_data = typeid_cast(&out_data); - if (!concreate_out_data) + ColumnTuple * concrete_out_data = typeid_cast(&out_data); + if (!concrete_out_data) return false; - const size_t tuple_size = concreate_out_data->tupleSize(); + const size_t tuple_size = concrete_out_data->tupleSize(); for (size_t i = 0; i < tuple_size; ++i) { ColumnRawPtrs elem_columns(columns.size(), nullptr); @@ -264,7 +264,7 @@ private: const ColumnTuple * concrete_column = assert_cast(columns[j]); elem_columns[j] = &concrete_column->getColumn(i); } - executeAny(elem_columns, concreate_out_data->getColumn(i), input_rows_count); + executeAny(elem_columns, concrete_out_data->getColumn(i), input_rows_count); } return true; } From aafe498b7f6045e75414c17abecad168aa0efb88 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 7 Aug 2024 10:43:49 +0800 Subject: [PATCH 0547/1722] Handle edge case: col_int32 > 10.6 --- src/Storages/Statistics/Statistics.cpp | 22 +++++++++++++++++++ src/Storages/Statistics/Statistics.h | 6 +++++ src/Storages/Statistics/StatisticsMinMax.cpp | 10 +++------ src/Storages/Statistics/StatisticsTDigest.cpp | 16 +++++--------- .../02864_statistics_estimation.sql | 10 +++++---- 5 files changed, 43 insertions(+), 21 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 1f159211a93..e94d8b67aff 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include @@ -10,6 +12,7 @@ #include #include #include +#include #include "config.h" /// USE_DATASKETCHES @@ -28,6 +31,25 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; +std::optional StatisticsUtils::tryConvertToFloat64(const Field & value, const DataTypePtr & value_data_type) +{ + if (value_data_type->isValueRepresentedByNumber()) + { + Field val_converted; + + /// For case val_int32 < 10.5 or val_int32 < '10.5' we should convert 10.5 to Float64. + if (isInteger(value_data_type) && (value.getType() == Field::Types::Float64 || value.getType() == Field::Types::String)) + val_converted = convertFieldToType(value, *DataTypeFactory::instance().get("Float64")); + + /// We should convert value to the real column data type and then translate it to Float64. + /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. + val_converted = convertFieldToType(value, *value_data_type); + if (val_converted.isNull()) + return {}; + return applyVisitor(FieldVisitorConvertToNumber(), val_converted); + } + return {}; +} IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 1425be89d47..9f82f137f1c 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -15,6 +15,12 @@ constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; +struct StatisticsUtils +{ + /// Returns std::nullopt if input Field cannot be converted to a concrete value + static std::optional tryConvertToFloat64(const Field & value, const DataTypePtr & value_data_type); +}; + /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, /// what are the N most frequent values, diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 11a0704dae7..88522894427 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -1,10 +1,8 @@ #include -#include #include #include #include #include -#include #include @@ -53,12 +51,10 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) Float64 StatisticsMinMax::estimateLess(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - if (val_as_float < min) return 0; @@ -68,7 +64,7 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const if (min == max) return (val_as_float != max) ? 0 : row_count; - return ((val_as_float - min) / (max - min)) * row_count; + return ((*val_as_float - min) / (max - min)) * row_count; } void minMaxStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index cfd626c8ea8..a7ce43d2fe1 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -41,22 +41,18 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - return t_digest.getCountLessThan(val_as_float); + return t_digest.getCountLessThan(*val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - return t_digest.getCountEqual(val_as_float); + return t_digest.getCountEqual(*val_as_float); } void tdigestStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 6436cf68738..5eb51fe7111 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -67,9 +67,7 @@ ALTER TABLE tab DROP STATISTICS a, b, c, d; SELECT 'Test statistics multi-types:'; ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, minmax, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS d TYPE count_min, minmax, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS b, c, d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') @@ -82,10 +80,14 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT 'Test statistics implicitly type conversion:'; + SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '1'/*1*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and b > 50.1/*5000*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; From cfaa852af9e94bd25217bfbb690f2d6ad641f3d6 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 7 Aug 2024 11:40:01 +0800 Subject: [PATCH 0548/1722] Little fix --- src/Storages/Statistics/Statistics.cpp | 7 ++++--- .../0_stateless/02864_statistics_estimation.reference | 5 ++++- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index e94d8b67aff..691cf42aa42 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -40,10 +40,11 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & value, /// For case val_int32 < 10.5 or val_int32 < '10.5' we should convert 10.5 to Float64. if (isInteger(value_data_type) && (value.getType() == Field::Types::Float64 || value.getType() == Field::Types::String)) val_converted = convertFieldToType(value, *DataTypeFactory::instance().get("Float64")); + else + /// We should convert value to the real column data type and then translate it to Float64. + /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. + val_converted = convertFieldToType(value, *value_data_type); - /// We should convert value to the real column data type and then translate it to Float64. - /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. - val_converted = convertFieldToType(value, *value_data_type); if (val_converted.isNull()) return {}; return applyVisitor(FieldVisitorConvertToNumber(), val_converted); diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index fc370691830..d475926df8b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -25,4 +25,7 @@ Test statistics multi-types: Test statistics implicitly type conversion: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'1\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) + Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\')) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), greater(b, 50.1_Float64)) (removed) From 7d42101593e4f30b5d423ee38d169a76352c2b0d Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Wed, 7 Aug 2024 14:19:40 +0800 Subject: [PATCH 0549/1722] Re-trigger ci --- .../test_replicated_database/test.py | 23 ------ .../__init__.py | 0 .../configs/config.xml | 10 +++ .../configs/settings.xml | 15 ++++ .../test.py | 70 +++++++++++++++++++ 5 files changed, 95 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/__init__.py create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/test.py diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 3a640897530..60a6e099b22 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1549,26 +1549,3 @@ def test_all_groups_cluster(started_cluster): assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" ) - - -def test_alter_modify_order_by(started_cluster): - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") - - main_node.query( - "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');" - ) - main_node.query( - "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" - ) - main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") - snapshotting_node.query( - "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');" - ) - - query = "show create table alter_modify_order_by.t1" - expected = main_node.query(query) - assert_eq_with_retry(snapshotting_node, query, expected) - - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/__init__.py b/tests/integration/test_replicated_database_alter_modify_order_by/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml b/tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml new file mode 100644 index 00000000000..706628cf93b --- /dev/null +++ b/tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml @@ -0,0 +1,10 @@ + + 10 + 1 + + 10 + + 50 + 42 + false + diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml new file mode 100644 index 00000000000..c637fe8eead --- /dev/null +++ b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml @@ -0,0 +1,15 @@ + + + + 1 + 1 + 0 + 0 + + + + + default + + + diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/test.py b/tests/integration/test_replicated_database_alter_modify_order_by/test.py new file mode 100644 index 00000000000..1ea126ace10 --- /dev/null +++ b/tests/integration/test_replicated_database_alter_modify_order_by/test.py @@ -0,0 +1,70 @@ +import re +import pytest + + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain + + +cluster = ClickHouseCluster(__file__) + +main_node = cluster.add_instance( + "main_node", + main_configs=["configs/config.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) + +snapshotting_node = cluster.add_instance( + "snapshotting_node", + main_configs=["configs/config.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + macros={"shard": 2, "replica": 1}, +) + + +all_nodes = [ + main_node, + snapshotting_node, +] + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_alter_modify_order_by(started_cluster): + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + + main_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" + ) + main_node.query( + "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" + ) + main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") + snapshotting_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" + ) + + query = ( + "select count() from system.tables where database = 'alter_modify_order_by';" + ) + expected = main_node.query(query) + assert_eq_with_retry(snapshotting_node, query, expected) + + query = "show create table alter_modify_order_by.t1;" + assert main_node.query(query) == snapshotting_node.query(query) + + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") From deb58b4ede1cabff8e4490fb39b2627ee94a347a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 3 Jul 2024 18:23:38 +0800 Subject: [PATCH 0550/1722] any/anti/semi join support mixed join conditions --- src/Interpreters/HashJoin/HashJoin.cpp | 61 +++-- src/Interpreters/HashJoin/HashJoinMethods.h | 139 ++++++++---- src/Interpreters/HashJoin/JoinFeatures.h | 7 +- src/Interpreters/HashJoin/JoinUsedFlags.h | 33 ++- src/Interpreters/HashJoin/LeftHashJoin.cpp | 3 + src/Interpreters/joinDispatch.h | 108 +++++---- src/Storages/StorageJoin.cpp | 6 +- ..._join_on_inequal_expression_fast.reference | 214 ++++++++++++++++++ ...006_join_on_inequal_expression_fast.sql.j2 | 11 + 9 files changed, 469 insertions(+), 113 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 0c7cad4360d..769cb574ed7 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -291,12 +291,13 @@ void HashJoin::dataMapInit(MapsVariant & map) { if (kind == JoinKind::Cross) return; - joinDispatchInit(kind, strictness, map); - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); }); + auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; + joinDispatchInit(kind, strictness, map, prefer_use_maps_all); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.create(data->type); }); if (reserve_num) { - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); }); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); }); } if (!data) @@ -327,9 +328,10 @@ size_t HashJoin::getTotalRowCount() const } else { + auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; for (const auto & map : data->maps) { - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); }); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); }); } } @@ -367,9 +369,10 @@ size_t HashJoin::getTotalByteCount() const if (data->type != Type::CROSS) { + auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; for (const auto & map : data->maps) { - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); }); + joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); }); } } return res; @@ -520,6 +523,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) return true; } + bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; + size_t total_rows = 0; size_t total_bytes = 0; { @@ -592,7 +597,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) bool is_inserted = false; if (kind != JoinKind::Cross) { - joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) + joinDispatch(kind, strictness, data->maps[onexpr_idx], prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & map) { size_t size = HashJoinMethods>::insertFromBlockImpl( *this, @@ -608,10 +613,10 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) is_inserted); if (flag_per_row) - used_flags->reinit(stored_block); + used_flags->reinit, MapsAll>>(stored_block); else if (is_inserted) /// Number of buckets + 1 value from zero storage - used_flags->reinit(size + 1); + used_flags->reinit, MapsAll>>(size + 1); }); } @@ -869,7 +874,7 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block keys.insert(std::move(key)); } - static_assert(!MapGetter::flagged, + static_assert(!MapGetter::flagged, "joinGet are not protected from hash table changes between block processing"); std::vector maps_vector; @@ -910,16 +915,34 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) materializeBlockInplace(block); } + bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; { std::vectormaps[0])> * > maps_vector; for (size_t i = 0; i < table_join->getClauses().size(); ++i) maps_vector.push_back(&data->maps[i]); - if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) + if (joinDispatch(kind, strictness, maps_vector, prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & maps_vector_) { - using MapType = typename MapGetter::Map; - Block remaining_block = HashJoinMethods::joinBlockImpl( - *this, block, sample_block_with_columns_to_add, maps_vector_); + Block remaining_block; + if constexpr (std::is_same_v, std::vector>) + { + remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + } + else if constexpr (std::is_same_v, std::vector>) + { + remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + } + else if constexpr (std::is_same_v, std::vector>) + { + remaining_block = HashJoinMethods::joinBlockImpl( + *this, block, sample_block_with_columns_to_add, maps_vector_); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type"); + } if (remaining_block.rows()) not_processed = std::make_shared(ExtraBlock{std::move(remaining_block)}); else @@ -1019,7 +1042,8 @@ public: rows_added = fillColumnsFromMap(map, columns_right); }; - if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback)) + bool prefer_use_maps_all = parent.table_join->getMixedJoinExpression() != nullptr; + if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), prefer_use_maps_all, fill_callback)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness); } @@ -1216,11 +1240,12 @@ void HashJoin::reuseJoinedData(const HashJoin & join) if (flag_per_row) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported"); + bool prefer_use_maps_all = join.table_join->getMixedJoinExpression() != nullptr; for (auto & map : data->maps) { - joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_) + joinDispatch(kind, strictness, map, prefer_use_maps_all, [this](auto kind_, auto strictness_, auto & map_) { - used_flags->reinit(map_.getBufferSizeInCells(data->type) + 1); + used_flags->reinit, MapsAll>>(map_.getBufferSizeInCells(data->type) + 1); }); } } @@ -1300,7 +1325,9 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona additional_filter_expression->dumpActions()); } - bool is_supported = (strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind)); + bool is_supported = ((strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind))) + || ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti) + && (isLeft(kind) || isRight(kind))); if (!is_supported) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 0dfafa94efc..36785c08845 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -112,7 +112,7 @@ public: const MapsTemplateVector & maps_, bool is_join_get = false) { - constexpr JoinFeatures join_features; + constexpr JoinFeatures join_features; std::vector join_on_keys; const auto & onexprs = join.table_join->getClauses(); @@ -358,22 +358,20 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags) { - constexpr JoinFeatures join_features; - if constexpr (join_features.is_all_join) + constexpr JoinFeatures join_features; + if constexpr (join_features.is_maps_all) { - if (added_columns.additional_filter_expression) - { - bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; - return joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), - mapv, - added_columns, - used_flags, - need_filter, - join_features.need_flags, - join_features.add_missing, - mark_per_row_used); - } + if (added_columns.additional_filter_expression) + { + bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; + return joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), + mapv, + added_columns, + used_flags, + need_filter, + mark_per_row_used); + } } if (added_columns.additional_filter_expression) @@ -394,7 +392,7 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags) { - constexpr JoinFeatures join_features; + constexpr JoinFeatures join_features; size_t rows = added_columns.rows_to_add; if constexpr (need_filter) @@ -474,7 +472,7 @@ private: mapped, added_columns, current_offset, known_rows, used_flags_opt); } } - else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) + else if constexpr (join_features.is_any_join && join_features.inner) { bool used_once = used_flags.template setUsedOnce(find_result); @@ -655,24 +653,23 @@ private: } /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. - template + template static size_t joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], bool need_filter [[maybe_unused]], - bool need_flags [[maybe_unused]], - bool add_missing [[maybe_unused]], bool flag_per_row [[maybe_unused]]) { + constexpr JoinFeatures join_features; size_t left_block_rows = added_columns.rows_to_add; if (need_filter) added_columns.filter = IColumn::Filter(left_block_rows, 0); std::unique_ptr pool; - if constexpr (need_replication) + if constexpr (join_features.need_replication) added_columns.offsets_to_replicate = std::make_unique(left_block_rows); std::vector row_replicate_offset; @@ -699,7 +696,7 @@ private: selected_rows.clear(); for (; left_row_iter < left_block_rows; ++left_row_iter) { - if constexpr (need_replication) + if constexpr (join_features.need_replication) { if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) { @@ -743,7 +740,7 @@ private: for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) { bool any_matched = false; - /// For all right join, flag_per_row is true, we need mark used flags for each row. + /// For right join, flag_per_row is true, we need mark used flags for each row. if (flag_per_row) { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) @@ -751,10 +748,26 @@ private: if (filter_flags[replicated_row]) { any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); - total_added_rows += 1; - if (need_flags) - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + if constexpr (join_features.is_semi_join || join_features.is_any_join) + { + auto used_once = used_flags.template setUsedOnce(selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + total_added_rows += 1; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + else + { + total_added_rows += 1; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } } ++selected_right_row_it; } @@ -763,34 +776,66 @@ private: { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { - if (filter_flags[replicated_row]) + if constexpr (join_features.is_anti_join) { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); - total_added_rows += 1; + any_matched |= filter_flags[replicated_row]; + } + else if constexpr (join_features.need_replication) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; + } + else + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; + break; + } + else + ++selected_right_row_it; } - ++selected_right_row_it; } } - if (!any_matched) + + + if constexpr (join_features.is_anti_join) { - if (add_missing) - addNotFoundRow(added_columns, total_added_rows); - else - addNotFoundRow(added_columns, total_added_rows); + if (!any_matched) + { + if constexpr (join_features.left) + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + addNotFoundRow(added_columns, total_added_rows); + } } else { - if (!flag_per_row && need_flags) - used_flags.template setUsed(find_results[find_result_index]); - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - if (add_missing) - added_columns.applyLazyDefaults(); + if (!any_matched) + { + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if (!flag_per_row) + used_flags.template setUsed(find_results[find_result_index]); + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + if constexpr (join_features.add_missing) + added_columns.applyLazyDefaults(); + } } find_result_index += (prev_replicated_row != row_replicate_offset[i]); - if constexpr (need_replication) + if constexpr (join_features.need_replication) { (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; } @@ -817,7 +862,7 @@ private: auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); copy_final_matched_rows(left_start_row, filter_col); - if constexpr (need_replication) + if constexpr (join_features.need_replication) { // Add a check for current_added_rows to avoid run the filter expression on too small size batch. if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) @@ -825,7 +870,7 @@ private: } } - if constexpr (need_replication) + if constexpr (join_features.need_replication) { added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); added_columns.filter.resize_assume_reserved(left_row_iter); diff --git a/src/Interpreters/HashJoin/JoinFeatures.h b/src/Interpreters/HashJoin/JoinFeatures.h index 2f2bd1e29a2..a530179f0b4 100644 --- a/src/Interpreters/HashJoin/JoinFeatures.h +++ b/src/Interpreters/HashJoin/JoinFeatures.h @@ -3,15 +3,15 @@ #include namespace DB { -template +template struct JoinFeatures { static constexpr bool is_any_join = STRICTNESS == JoinStrictness::Any; - static constexpr bool is_any_or_semi_join = STRICTNESS == JoinStrictness::Any || STRICTNESS == JoinStrictness::RightAny || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Left); static constexpr bool is_all_join = STRICTNESS == JoinStrictness::All; static constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; static constexpr bool is_semi_join = STRICTNESS == JoinStrictness::Semi; static constexpr bool is_anti_join = STRICTNESS == JoinStrictness::Anti; + static constexpr bool is_any_or_semi_join = is_any_join || STRICTNESS == JoinStrictness::RightAny || (is_semi_join && KIND == JoinKind::Left); static constexpr bool left = KIND == JoinKind::Left; static constexpr bool right = KIND == JoinKind::Right; @@ -22,7 +22,8 @@ struct JoinFeatures static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); static constexpr bool add_missing = (left || full) && !is_semi_join; - static constexpr bool need_flags = MapGetter::flagged; + static constexpr bool need_flags = MapGetter, HashJoin::MapsOne>>::flagged; + static constexpr bool is_maps_all = std::is_same_v, HashJoin::MapsAll>; }; } diff --git a/src/Interpreters/HashJoin/JoinUsedFlags.h b/src/Interpreters/HashJoin/JoinUsedFlags.h index bd41ba2073f..c84c6ec3fea 100644 --- a/src/Interpreters/HashJoin/JoinUsedFlags.h +++ b/src/Interpreters/HashJoin/JoinUsedFlags.h @@ -26,10 +26,10 @@ public: /// Update size for vector with flags. /// Calling this method invalidates existing flags. /// It can be called several times, but all of them should happen before using this structure. - template + template void reinit(size_t size) { - if constexpr (MapGetter::flagged) + if constexpr (MapGetter::flagged) { assert(flags[nullptr].size() <= size); need_flags = true; @@ -43,10 +43,10 @@ public: } } - template + template void reinit(const Block * block_ptr) { - if constexpr (MapGetter::flagged) + if constexpr (MapGetter::flagged) { assert(flags[block_ptr].size() <= block_ptr->rows()); need_flags = true; @@ -148,6 +148,31 @@ public: } } + template + bool setUsedOnce(const Block * block, size_t row_num, size_t offset) + { + if constexpr (!use_flags) + return true; + + if constexpr (flag_per_row) + { + /// fast check to prevent heavy CAS with seq_cst order + if (flags[block][row_num].load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return flags[block][row_num].compare_exchange_strong(expected, true); + } + else + { + /// fast check to prevent heavy CAS with seq_cst order + if (flags[nullptr][offset].load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return flags[nullptr][offset].compare_exchange_strong(expected, true); + } + } }; } diff --git a/src/Interpreters/HashJoin/LeftHashJoin.cpp b/src/Interpreters/HashJoin/LeftHashJoin.cpp index 69e17ff70bd..a53ffaac0b5 100644 --- a/src/Interpreters/HashJoin/LeftHashJoin.cpp +++ b/src/Interpreters/HashJoin/LeftHashJoin.cpp @@ -4,8 +4,11 @@ namespace DB { template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; } diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 54c5c7dc83a..982c56e8210 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -12,38 +12,41 @@ namespace DB { -template +template struct MapGetter; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; /// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; /// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; -template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; +template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template -struct MapGetter { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; }; +template +struct MapGetter { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; }; static constexpr std::array STRICTNESSES = { JoinStrictness::RightAny, @@ -62,7 +65,7 @@ static constexpr std::array KINDS = { }; /// Init specified join map -inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps) +inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps, bool prefer_use_maps_all = false) { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { @@ -70,7 +73,10 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin: constexpr auto j = ij % STRICTNESSES.size(); if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { - maps = typename MapGetter::Map(); + if (prefer_use_maps_all) + maps = typename MapGetter::Map(); + else + maps = typename MapGetter::Map(); return true; } return false; @@ -79,7 +85,7 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin: /// Call function on specified join map template -inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, Func && func) +inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, bool prefer_use_maps_all, Func && func) { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { @@ -89,10 +95,16 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & constexpr auto j = ij % STRICTNESSES.size(); if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { - func( - std::integral_constant(), - std::integral_constant(), - std::get::Map>(maps)); + if (prefer_use_maps_all) + func( + std::integral_constant(), + std::integral_constant(), + std::get::Map>(maps)); + else + func( + std::integral_constant(), + std::integral_constant(), + std::get::Map>(maps)); return true; } return false; @@ -101,7 +113,7 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & /// Call function on specified join map template -inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector & mapsv, Func && func) +inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector & mapsv, bool prefer_use_maps_all, Func && func) { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { @@ -111,17 +123,31 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector::Map; - std::vector v; - v.reserve(mapsv.size()); - for (const auto & el : mapsv) - v.push_back(&std::get(*el)); + if (prefer_use_maps_all) + { + using MapType = typename MapGetter::Map; + std::vector v; + v.reserve(mapsv.size()); + for (const auto & el : mapsv) + v.push_back(&std::get(*el)); - func( - std::integral_constant(), - std::integral_constant(), - v - /*std::get::Map>(maps)*/); + func( + std::integral_constant(), std::integral_constant(), v + /*std::get::Map>(maps)*/); + } + else + { + using MapType = typename MapGetter::Map; + std::vector v; + v.reserve(mapsv.size()); + for (const auto & el : mapsv) + v.push_back(&std::get(*el)); + + func( + std::integral_constant(), std::integral_constant(), v + /*std::get::Map>(maps)*/); + + } return true; } return false; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a0d6cf11b64..974df940e91 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -504,7 +504,11 @@ protected: return {}; Chunk chunk; - if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(), + if (!joinDispatch( + join->kind, + join->strictness, + join->data->maps.front(), + join->table_join->getMixedJoinExpression() != nullptr, [&](auto kind, auto strictness, auto & map) { chunk = createChunk(map); })) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness"); return chunk; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 46f24f73356..3e413afd98e 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -382,6 +382,220 @@ key1 e 5 5 5 key1 C 3 4 5 key2 a2 1 1 1 0 0 \N key4 f 2 3 4 key4 F 1 1 1 SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 d 4 7 2 key1 0 0 \N +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SELECT t1.*, t2.* FROM t1 RIGHT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 LEFT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 d 4 7 2 key1 0 0 \N +key1 e 5 5 5 key1 0 0 \N +key2 a2 1 1 1 key2 0 0 \N +key4 f 2 3 4 key4 0 0 \N +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SELECT t1.*, t2.* FROM t1 RIGHT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 key1 a 1 1 2 key1 B 2 1 2 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index 61ad5ec0bf1..1bf5a7870e7 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -22,6 +22,17 @@ SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 {% endfor -%} {% endfor -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'RIGHT'] -%} +{% for join_strictness in ['ANY', 'SEMI', 'ANTI'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_strictness }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +{% endfor -%} +{% endfor -%} +{% endfor -%} {% for algorithm in ['hash'] -%} SET join_algorithm='{{ algorithm }}'; From 413834d04920072dd9a38b56902d799f71a476a3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 Jul 2024 10:06:11 +0800 Subject: [PATCH 0551/1722] instance template classes --- src/Interpreters/HashJoin/HashJoinMethods.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 36785c08845..17c11b202ca 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -970,9 +970,12 @@ private: /// Instantiate template class ahead in different .cpp files to avoid `too large translation unit`. extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; From 377eed20fca56c74e4987fe4038fca6f0e019090 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 Jul 2024 15:05:24 +0800 Subject: [PATCH 0552/1722] reduce the size of HashJoin.cpp.o --- src/Interpreters/HashJoin/FullHashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoinMethods.h | 834 +--------------- .../HashJoin/HashJoinMethodsImpl.h | 912 ++++++++++++++++++ src/Interpreters/HashJoin/InnerHashJoin.cpp | 2 +- src/Interpreters/HashJoin/LeftHashJoin.cpp | 2 +- src/Interpreters/HashJoin/RightHashJoin.cpp | 2 +- 6 files changed, 932 insertions(+), 822 deletions(-) create mode 100644 src/Interpreters/HashJoin/HashJoinMethodsImpl.h diff --git a/src/Interpreters/HashJoin/FullHashJoin.cpp b/src/Interpreters/HashJoin/FullHashJoin.cpp index 5d058d10fc2..4cdb2e757a4 100644 --- a/src/Interpreters/HashJoin/FullHashJoin.cpp +++ b/src/Interpreters/HashJoin/FullHashJoin.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 17c11b202ca..e3b8fbc1737 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -12,15 +12,8 @@ #include #include - namespace DB { -namespace ErrorCodes -{ - extern const int UNSUPPORTED_JOIN_KEYS; - extern const int LOGICAL_ERROR; -} - /// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN. template struct Inserter @@ -64,7 +57,6 @@ struct Inserter } }; - /// MapsTemplate is one of MapsOne, MapsAll and MapsAsof template class HashJoinMethods @@ -81,27 +73,7 @@ public: ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, - bool & is_inserted) - { - switch (type) - { - case HashJoin::Type::EMPTY: - [[fallthrough]]; - case HashJoin::Type::CROSS: - /// Do nothing. We will only save block, and it is enough - is_inserted = true; - return 0; - - #define M(TYPE) \ - case HashJoin::Type::TYPE: \ - return insertFromBlockImplTypeCase>::Type>(\ - join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ - break; - - APPLY_FOR_JOIN_VARIANTS(M) - #undef M - } - } + bool & is_inserted); using MapsTemplateVector = std::vector; @@ -110,278 +82,36 @@ public: Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, - bool is_join_get = false) - { - constexpr JoinFeatures join_features; - - std::vector join_on_keys; - const auto & onexprs = join.table_join->getClauses(); - for (size_t i = 0; i < onexprs.size(); ++i) - { - const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; - join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); - } - size_t existing_columns = block.columns(); - - /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. - * Because if they are constants, then in the "not joined" rows, they may have different values - * - default values, which can differ from the values of these constants. - */ - if constexpr (join_features.right || join_features.full) - { - materializeBlockInplace(block); - } - - /** For LEFT/INNER JOIN, the saved blocks do not contain keys. - * For FULL/RIGHT JOIN, the saved blocks contain keys; - * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. - * For ASOF, the last column is used as the ASOF column - */ - AddedColumns added_columns( - block, - block_with_columns_to_add, - join.savedBlockSample(), - join, - std::move(join_on_keys), - join.table_join->getMixedJoinExpression(), - join_features.is_asof_join, - is_join_get); - - bool has_required_right_keys = (join.required_right_keys.columns() != 0); - added_columns.need_filter = join_features.need_filter || has_required_right_keys; - added_columns.max_joined_block_rows = join.max_joined_block_rows; - if (!added_columns.max_joined_block_rows) - added_columns.max_joined_block_rows = std::numeric_limits::max(); - else - added_columns.reserve(join_features.need_replication); - - size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); - /// Do not hold memory for join_on_keys anymore - added_columns.join_on_keys.clear(); - Block remaining_block = sliceBlock(block, num_joined); - - added_columns.buildOutput(); - for (size_t i = 0; i < added_columns.size(); ++i) - block.insert(added_columns.moveColumn(i)); - - std::vector right_keys_to_replicate [[maybe_unused]]; - - if constexpr (join_features.need_filter) - { - /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. - for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1); - - /// Add join key columns from right block if needed using value from left table because of equality - for (size_t i = 0; i < join.required_right_keys.columns(); ++i) - { - const auto & right_key = join.required_right_keys.getByPosition(i); - /// asof column is already in block. - if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) - continue; - - const auto & left_column = block.getByName(join.required_right_keys_sources[i]); - const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); - auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); - block.insert(std::move(right_col)); - } - } - else if (has_required_right_keys) - { - /// Add join key columns from right block if needed. - for (size_t i = 0; i < join.required_right_keys.columns(); ++i) - { - const auto & right_key = join.required_right_keys.getByPosition(i); - auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); - /// asof column is already in block. - if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) - continue; - - const auto & left_column = block.getByName(join.required_right_keys_sources[i]); - auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); - block.insert(std::move(right_col)); - - if constexpr (join_features.need_replication) - right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); - } - } - - if constexpr (join_features.need_replication) - { - std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; - - /// If ALL ... JOIN - we replicate all the columns except the new ones. - for (size_t i = 0; i < existing_columns; ++i) - { - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); - } - - /// Replicate additional right keys - for (size_t pos : right_keys_to_replicate) - { - block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); - } - } - - return remaining_block; - } - + bool is_join_get = false); private: template - static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) - { - if constexpr (is_asof_join) - { - auto key_column_copy = key_columns; - auto key_size_copy = key_sizes; - key_column_copy.pop_back(); - key_size_copy.pop_back(); - return KeyGetter(key_column_copy, key_size_copy, nullptr); - } - else - return KeyGetter(key_columns, key_sizes, nullptr); - } + static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); template - static size_t NO_INLINE insertFromBlockImplTypeCase( + static size_t insertFromBlockImplTypeCase( HashJoin & join, HashMap & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) - { - [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; - constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; - - const IColumn * asof_column [[maybe_unused]] = nullptr; - if constexpr (is_asof_join) - asof_column = key_columns.back(); - - auto key_getter = createKeyGetter(key_columns, key_sizes); - - /// For ALL and ASOF join always insert values - is_inserted = !mapped_one || is_asof_join; - - for (size_t i = 0; i < rows; ++i) - { - if (null_map && (*null_map)[i]) - { - /// nulls are not inserted into hash table, - /// keep them for RIGHT and FULL joins - is_inserted = true; - continue; - } - - /// Check condition for right table from ON section - if (join_mask && !(*join_mask)[i]) - continue; - - if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); - else if constexpr (mapped_one) - is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); - else - Inserter::insertAll(join, map, key_getter, stored_block, i, pool); - } - return map.getBufferSizeInCells(); - } + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted); template static size_t switchJoinRightColumns( const std::vector & mapv, AddedColumns & added_columns, HashJoin::Type type, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; - switch (type) - { - case HashJoin::Type::EMPTY: { - if constexpr (!is_asof_join) - { - using KeyGetter = KeyGetterEmpty; - std::vector key_getter_vector; - key_getter_vector.emplace_back(); - - using MapTypeVal = typename KeyGetter::MappedType; - std::vector a_map_type_vector; - a_map_type_vector.emplace_back(); - return joinRightColumnsSwitchNullability( - std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); - } - throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type); - } - #define M(TYPE) \ - case HashJoin::Type::TYPE: \ - { \ - using MapTypeVal = const typename std::remove_reference_t::element_type; \ - using KeyGetter = typename KeyGetterForType::Type; \ - std::vector a_map_type_vector(mapv.size()); \ - std::vector key_getter_vector; \ - for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \ - { \ - const auto & join_on_key = added_columns.join_on_keys[d]; \ - a_map_type_vector[d] = mapv[d]->TYPE.get(); \ - key_getter_vector.push_back(std::move(createKeyGetter(join_on_key.key_columns, join_on_key.key_sizes))); \ - } \ - return joinRightColumnsSwitchNullability( \ - std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \ - } - APPLY_FOR_JOIN_VARIANTS(M) - #undef M - - default: - throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type); - } - } + JoinStuff::JoinUsedFlags & used_flags); template static size_t joinRightColumnsSwitchNullability( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - if (added_columns.need_filter) - { - return joinRightColumnsSwitchMultipleDisjuncts( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } - else - { - return joinRightColumnsSwitchMultipleDisjuncts( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } - } + JoinStuff::JoinUsedFlags & used_flags); template static size_t joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr JoinFeatures join_features; - if constexpr (join_features.is_maps_all) - { - if (added_columns.additional_filter_expression) - { - bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; - return joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), - mapv, - added_columns, - used_flags, - need_filter, - mark_per_row_used); - } - } - - if (added_columns.additional_filter_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); - - return mapv.size() > 1 ? joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } + JoinStuff::JoinUsedFlags & used_flags); /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). @@ -390,267 +120,17 @@ private: std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr JoinFeatures join_features; - - size_t rows = added_columns.rows_to_add; - if constexpr (need_filter) - added_columns.filter = IColumn::Filter(rows, 0); - - Arena pool; - - if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = std::make_unique(rows); - - IColumn::Offset current_offset = 0; - size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t i = 0; - for (; i < rows; ++i) - { - if constexpr (join_features.need_replication) - { - if (unlikely(current_offset >= max_joined_block_rows)) - { - added_columns.offsets_to_replicate->resize_assume_reserved(i); - added_columns.filter.resize_assume_reserved(i); - break; - } - } - - bool right_row_found = false; - - KnownRowsHolder known_rows; - for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) - { - const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; - if (join_keys.null_map && (*join_keys.null_map)[i]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(i); - using FindResult = typename KeyGetter::FindResult; - auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); - - if (find_result.isFound()) - { - right_row_found = true; - auto & mapped = find_result.getMapped(); - if constexpr (join_features.is_asof_join) - { - const IColumn & left_asof_key = added_columns.leftAsofKey(); - - auto row_ref = mapped->findAsof(left_asof_key, i); - if (row_ref.block) - { - setUsed(added_columns.filter, i); - if constexpr (flag_per_row) - used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); - else - used_flags.template setUsed(find_result); - - added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); - } - else - addNotFoundRow(added_columns, current_offset); - } - else if constexpr (join_features.is_all_join) - { - setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); - auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); - } - else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) - { - /// Use first appeared left key + it needs left columns replication - bool used_once = used_flags.template setUsedOnce(find_result); - if (used_once) - { - auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - setUsed(added_columns.filter, i); - addFoundRowAll( - mapped, added_columns, current_offset, known_rows, used_flags_opt); - } - } - else if constexpr (join_features.is_any_join && join_features.inner) - { - bool used_once = used_flags.template setUsedOnce(find_result); - - /// Use first appeared left key only - if (used_once) - { - setUsed(added_columns.filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); - } - - break; - } - else if constexpr (join_features.is_any_join && join_features.full) - { - /// TODO - } - else if constexpr (join_features.is_anti_join) - { - if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(find_result); - } - else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) - { - setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); - - if (join_features.is_any_or_semi_join) - { - break; - } - } - } - } - - if (!right_row_found) - { - if constexpr (join_features.is_anti_join && join_features.left) - setUsed(added_columns.filter, i); - addNotFoundRow(added_columns, current_offset); - } - - if constexpr (join_features.need_replication) - { - (*added_columns.offsets_to_replicate)[i] = current_offset; - } - } - - added_columns.applyLazyDefaults(); - return i; - } + JoinStuff::JoinUsedFlags & used_flags); template - static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]) - { - if constexpr (need_filter) - filter[pos] = 1; - } + static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]); template static ColumnPtr buildAdditionalFilter( size_t left_start_row, const std::vector & selected_rows, const std::vector & row_replicate_offset, - AddedColumns & added_columns) - { - ColumnPtr result_column; - do - { - if (selected_rows.empty()) - { - result_column = ColumnUInt8::create(); - break; - } - const Block & sample_right_block = *selected_rows.begin()->block; - if (!sample_right_block || !added_columns.additional_filter_expression) - { - auto filter = ColumnUInt8::create(); - filter->insertMany(1, selected_rows.size()); - result_column = std::move(filter); - break; - } - - auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); - if (required_cols.empty()) - { - Block block; - added_columns.additional_filter_expression->execute(block); - result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); - break; - } - NameSet required_column_names; - for (auto & col : required_cols) - required_column_names.insert(col.name); - - Block executed_block; - size_t right_col_pos = 0; - for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) - { - if (required_column_names.contains(col.name)) - { - auto new_col = col.column->cloneEmpty(); - for (const auto & selected_row : selected_rows) - { - const auto & src_col = selected_row.block->getByPosition(right_col_pos); - new_col->insertFrom(*src_col.column, selected_row.row_num); - } - executed_block.insert({std::move(new_col), col.type, col.name}); - } - right_col_pos += 1; - } - if (!executed_block) - { - result_column = ColumnUInt8::create(); - break; - } - - for (const auto & col_name : required_column_names) - { - const auto * src_col = added_columns.left_block.findByName(col_name); - if (!src_col) - continue; - auto new_col = src_col->column->cloneEmpty(); - size_t prev_left_offset = 0; - for (size_t i = 1; i < row_replicate_offset.size(); ++i) - { - const size_t & left_offset = row_replicate_offset[i]; - size_t rows = left_offset - prev_left_offset; - if (rows) - new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); - prev_left_offset = left_offset; - } - executed_block.insert({std::move(new_col), src_col->type, col_name}); - } - if (!executed_block) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", - required_cols.toString(), - sample_right_block.dumpNames(), - added_columns.left_block.dumpNames()); - } - - for (const auto & col : executed_block.getColumnsWithTypeAndName()) - if (!col.column || !col.type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); - - added_columns.additional_filter_expression->execute(executed_block); - result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); - executed_block.clear(); - } while (false); - - result_column = result_column->convertToFullIfNeeded(); - if (result_column->isNullable()) - { - /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros - /// Trying to avoid copying data, since we are the only owner of the column. - ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); - - MutableColumnPtr mutable_column; - { - ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); - result_column.reset(); - mutable_column = IColumn::mutate(std::move(nested_column)); - } - - auto & column_data = assert_cast(*mutable_column).getData(); - const auto & mask_column_data = assert_cast(*mask_column).getData(); - for (size_t i = 0; i < column_data.size(); ++i) - { - if (mask_column_data[i]) - column_data[i] = 0; - } - return mutable_column; - } - return result_column; - } + AddedColumns & added_columns); /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. template @@ -660,241 +140,10 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], bool need_filter [[maybe_unused]], - bool flag_per_row [[maybe_unused]]) - { - constexpr JoinFeatures join_features; - size_t left_block_rows = added_columns.rows_to_add; - if (need_filter) - added_columns.filter = IColumn::Filter(left_block_rows, 0); - - std::unique_ptr pool; - - if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = std::make_unique(left_block_rows); - - std::vector row_replicate_offset; - row_replicate_offset.reserve(left_block_rows); - - using FindResult = typename KeyGetter::FindResult; - size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t left_row_iter = 0; - PreSelectedRows selected_rows; - selected_rows.reserve(left_block_rows); - std::vector find_results; - find_results.reserve(left_block_rows); - bool exceeded_max_block_rows = false; - IColumn::Offset total_added_rows = 0; - IColumn::Offset current_added_rows = 0; - - auto collect_keys_matched_rows_refs = [&]() - { - pool = std::make_unique(); - find_results.clear(); - row_replicate_offset.clear(); - row_replicate_offset.push_back(0); - current_added_rows = 0; - selected_rows.clear(); - for (; left_row_iter < left_block_rows; ++left_row_iter) - { - if constexpr (join_features.need_replication) - { - if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) - { - break; - } - } - KnownRowsHolder all_flag_known_rows; - KnownRowsHolder single_flag_know_rows; - for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) - { - const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; - if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); - auto find_result = row_acceptable - ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) - : FindResult(); - - if (find_result.isFound()) - { - auto & mapped = find_result.getMapped(); - find_results.push_back(find_result); - if (flag_per_row) - addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); - else - addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); - } - } - row_replicate_offset.push_back(current_added_rows); - } - }; - - auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) - { - const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); - - size_t prev_replicated_row = 0; - auto selected_right_row_it = selected_rows.begin(); - size_t find_result_index = 0; - for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) - { - bool any_matched = false; - /// For right join, flag_per_row is true, we need mark used flags for each row. - if (flag_per_row) - { - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) - { - if (filter_flags[replicated_row]) - { - any_matched = true; - if constexpr (join_features.is_semi_join || join_features.is_any_join) - { - auto used_once = used_flags.template setUsedOnce(selected_right_row_it->block, selected_right_row_it->row_num, 0); - if (used_once) - { - total_added_rows += 1; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - } - } - else if constexpr (join_features.is_anti_join) - { - if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); - } - else - { - total_added_rows += 1; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); - } - } - ++selected_right_row_it; - } - } - else - { - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) - { - if constexpr (join_features.is_anti_join) - { - any_matched |= filter_flags[replicated_row]; - } - else if constexpr (join_features.need_replication) - { - if (filter_flags[replicated_row]) - { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - total_added_rows += 1; - } - ++selected_right_row_it; - } - else - { - if (filter_flags[replicated_row]) - { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - total_added_rows += 1; - selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; - break; - } - else - ++selected_right_row_it; - } - } - } - - - if constexpr (join_features.is_anti_join) - { - if (!any_matched) - { - if constexpr (join_features.left) - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - addNotFoundRow(added_columns, total_added_rows); - } - } - else - { - if (!any_matched) - { - addNotFoundRow(added_columns, total_added_rows); - } - else - { - if (!flag_per_row) - used_flags.template setUsed(find_results[find_result_index]); - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - if constexpr (join_features.add_missing) - added_columns.applyLazyDefaults(); - } - } - find_result_index += (prev_replicated_row != row_replicate_offset[i]); - - if constexpr (join_features.need_replication) - { - (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; - } - prev_replicated_row = row_replicate_offset[i]; - } - }; - - while (left_row_iter < left_block_rows && !exceeded_max_block_rows) - { - auto left_start_row = left_row_iter; - collect_keys_matched_rows_refs(); - if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " - "left_start_row: {}", - selected_rows.size(), - current_added_rows, - row_replicate_offset.size(), - left_row_iter, - left_start_row); - } - auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); - copy_final_matched_rows(left_start_row, filter_col); - - if constexpr (join_features.need_replication) - { - // Add a check for current_added_rows to avoid run the filter expression on too small size batch. - if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) - exceeded_max_block_rows = true; - } - } - - if constexpr (join_features.need_replication) - { - added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); - added_columns.filter.resize_assume_reserved(left_row_iter); - } - added_columns.applyLazyDefaults(); - return left_row_iter; - } + bool flag_per_row [[maybe_unused]]); /// Cut first num_rows rows from block in place and returns block with remaining rows - static Block sliceBlock(Block & block, size_t num_rows) - { - size_t total_rows = block.rows(); - if (num_rows >= total_rows) - return {}; - size_t remaining_rows = total_rows - num_rows; - Block remaining_block = block.cloneEmpty(); - for (size_t i = 0; i < block.columns(); ++i) - { - auto & col = block.getByPosition(i); - remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows); - col.column = col.column->cut(0, num_rows); - } - return remaining_block; - } + static Block sliceBlock(Block & block, size_t num_rows); /** Since we do not store right key columns, * this function is used to copy left key columns to right key columns. @@ -909,62 +158,11 @@ private: const DataTypePtr & right_key_type, const String & renamed_right_column, const ColumnWithTypeAndName & left_column, - const IColumn::Filter * null_map_filter = nullptr) - { - ColumnWithTypeAndName right_column = left_column; - right_column.name = renamed_right_column; + const IColumn::Filter * null_map_filter = nullptr); - if (null_map_filter) - right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter); + static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable); - bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type); - if (null_map_filter) - correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter); - else - correctNullabilityInplace(right_column, should_be_nullable); - - if (!right_column.type->equals(*right_key_type)) - { - right_column.column = castColumnAccurate(right_column, right_key_type); - right_column.type = right_key_type; - } - - right_column.column = right_column.column->convertToFullColumnIfConst(); - return right_column; - } - - static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable) - { - if (nullable) - { - JoinCommon::convertColumnToNullable(column); - } - else - { - /// We have to replace values masked by NULLs with defaults. - if (column.column) - if (const auto * nullable_column = checkAndGetColumn(&*column.column)) - column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true); - - JoinCommon::removeColumnNullability(column); - } - } - - static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) - { - if (nullable) - { - JoinCommon::convertColumnToNullable(column); - if (column.type->isNullable() && !negative_null_map.empty()) - { - MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column)); - assert_cast(*mutable_column).applyNegatedNullMap(negative_null_map); - column.column = std::move(mutable_column); - } - } - else - JoinCommon::removeColumnNullability(column); - } + static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map); }; /// Instantiate template class ahead in different .cpp files to avoid `too large translation unit`. diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h new file mode 100644 index 00000000000..2bf5f6aef4a --- /dev/null +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -0,0 +1,912 @@ +#pragma once +#include +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNSUPPORTED_JOIN_KEYS; + extern const int LOGICAL_ERROR; +} +template +size_t HashJoinMethods::insertFromBlockImpl( + HashJoin & join, + HashJoin::Type type, + MapsTemplate & maps, + size_t rows, + const ColumnRawPtrs & key_columns, + const Sizes & key_sizes, + Block * stored_block, + ConstNullMapPtr null_map, + UInt8ColumnDataPtr join_mask, + Arena & pool, + bool & is_inserted) +{ + switch (type) + { + case HashJoin::Type::EMPTY: + [[fallthrough]]; + case HashJoin::Type::CROSS: + /// Do nothing. We will only save block, and it is enough + is_inserted = true; + return 0; + +#define M(TYPE) \ + case HashJoin::Type::TYPE: \ + return insertFromBlockImplTypeCase< \ + typename KeyGetterForType>::Type>( \ + join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ + break; + + APPLY_FOR_JOIN_VARIANTS(M) +#undef M + } +} + +template +Block HashJoinMethods::joinBlockImpl( + const HashJoin & join, Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get) +{ + constexpr JoinFeatures join_features; + + std::vector join_on_keys; + const auto & onexprs = join.table_join->getClauses(); + for (size_t i = 0; i < onexprs.size(); ++i) + { + const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; + join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); + } + size_t existing_columns = block.columns(); + + /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. + * Because if they are constants, then in the "not joined" rows, they may have different values + * - default values, which can differ from the values of these constants. + */ + if constexpr (join_features.right || join_features.full) + { + materializeBlockInplace(block); + } + + /** For LEFT/INNER JOIN, the saved blocks do not contain keys. + * For FULL/RIGHT JOIN, the saved blocks contain keys; + * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. + * For ASOF, the last column is used as the ASOF column + */ + AddedColumns added_columns( + block, + block_with_columns_to_add, + join.savedBlockSample(), + join, + std::move(join_on_keys), + join.table_join->getMixedJoinExpression(), + join_features.is_asof_join, + is_join_get); + + bool has_required_right_keys = (join.required_right_keys.columns() != 0); + added_columns.need_filter = join_features.need_filter || has_required_right_keys; + added_columns.max_joined_block_rows = join.max_joined_block_rows; + if (!added_columns.max_joined_block_rows) + added_columns.max_joined_block_rows = std::numeric_limits::max(); + else + added_columns.reserve(join_features.need_replication); + + size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); + /// Do not hold memory for join_on_keys anymore + added_columns.join_on_keys.clear(); + Block remaining_block = sliceBlock(block, num_joined); + + added_columns.buildOutput(); + for (size_t i = 0; i < added_columns.size(); ++i) + block.insert(added_columns.moveColumn(i)); + + std::vector right_keys_to_replicate [[maybe_unused]]; + + if constexpr (join_features.need_filter) + { + /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. + for (size_t i = 0; i < existing_columns; ++i) + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1); + + /// Add join key columns from right block if needed using value from left table because of equality + for (size_t i = 0; i < join.required_right_keys.columns(); ++i) + { + const auto & right_key = join.required_right_keys.getByPosition(i); + /// asof column is already in block. + if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) + continue; + + const auto & left_column = block.getByName(join.required_right_keys_sources[i]); + const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); + block.insert(std::move(right_col)); + } + } + else if (has_required_right_keys) + { + /// Add join key columns from right block if needed. + for (size_t i = 0; i < join.required_right_keys.columns(); ++i) + { + const auto & right_key = join.required_right_keys.getByPosition(i); + auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); + /// asof column is already in block. + if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back()) + continue; + + const auto & left_column = block.getByName(join.required_right_keys_sources[i]); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter); + block.insert(std::move(right_col)); + + if constexpr (join_features.need_replication) + right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); + } + } + + if constexpr (join_features.need_replication) + { + std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; + + /// If ALL ... JOIN - we replicate all the columns except the new ones. + for (size_t i = 0; i < existing_columns; ++i) + { + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); + } + + /// Replicate additional right keys + for (size_t pos : right_keys_to_replicate) + { + block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); + } + } + + return remaining_block; +} + +template +template +KeyGetter HashJoinMethods::createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) +{ + if constexpr (is_asof_join) + { + auto key_column_copy = key_columns; + auto key_size_copy = key_sizes; + key_column_copy.pop_back(); + key_size_copy.pop_back(); + return KeyGetter(key_column_copy, key_size_copy, nullptr); + } + else + return KeyGetter(key_columns, key_sizes, nullptr); +} + +template +template +size_t HashJoinMethods::insertFromBlockImplTypeCase( + HashJoin & join, + HashMap & map, + size_t rows, + const ColumnRawPtrs & key_columns, + const Sizes & key_sizes, + Block * stored_block, + ConstNullMapPtr null_map, + UInt8ColumnDataPtr join_mask, + Arena & pool, + bool & is_inserted) +{ + [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; + constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; + + const IColumn * asof_column [[maybe_unused]] = nullptr; + if constexpr (is_asof_join) + asof_column = key_columns.back(); + + auto key_getter = createKeyGetter(key_columns, key_sizes); + + /// For ALL and ASOF join always insert values + is_inserted = !mapped_one || is_asof_join; + + for (size_t i = 0; i < rows; ++i) + { + if (null_map && (*null_map)[i]) + { + /// nulls are not inserted into hash table, + /// keep them for RIGHT and FULL joins + is_inserted = true; + continue; + } + + /// Check condition for right table from ON section + if (join_mask && !(*join_mask)[i]) + continue; + + if constexpr (is_asof_join) + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + else if constexpr (mapped_one) + is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + else + Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + } + return map.getBufferSizeInCells(); +} + +template +template +size_t HashJoinMethods::switchJoinRightColumns( + const std::vector & mapv, + AddedColumns & added_columns, + HashJoin::Type type, + JoinStuff::JoinUsedFlags & used_flags) +{ + constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; + switch (type) + { + case HashJoin::Type::EMPTY: { + if constexpr (!is_asof_join) + { + using KeyGetter = KeyGetterEmpty; + std::vector key_getter_vector; + key_getter_vector.emplace_back(); + + using MapTypeVal = typename KeyGetter::MappedType; + std::vector a_map_type_vector; + a_map_type_vector.emplace_back(); + return joinRightColumnsSwitchNullability( + std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); + } + throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type); + } +#define M(TYPE) \ + case HashJoin::Type::TYPE: { \ + using MapTypeVal = const typename std::remove_reference_t::element_type; \ + using KeyGetter = typename KeyGetterForType::Type; \ + std::vector a_map_type_vector(mapv.size()); \ + std::vector key_getter_vector; \ + for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \ + { \ + const auto & join_on_key = added_columns.join_on_keys[d]; \ + a_map_type_vector[d] = mapv[d]->TYPE.get(); \ + key_getter_vector.push_back( \ + std::move(createKeyGetter(join_on_key.key_columns, join_on_key.key_sizes))); \ + } \ + return joinRightColumnsSwitchNullability(std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \ + } + APPLY_FOR_JOIN_VARIANTS(M) +#undef M + + default: + throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type); + } +} + +template +template +size_t HashJoinMethods::joinRightColumnsSwitchNullability( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags) +{ + if (added_columns.need_filter) + { + return joinRightColumnsSwitchMultipleDisjuncts( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } + else + { + return joinRightColumnsSwitchMultipleDisjuncts( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } +} + +template +template +size_t HashJoinMethods::joinRightColumnsSwitchMultipleDisjuncts( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags) +{ + constexpr JoinFeatures join_features; + if constexpr (join_features.is_maps_all) + { + if (added_columns.additional_filter_expression) + { + bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; + return joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), mapv, added_columns, used_flags, need_filter, mark_per_row_used); + } + } + + if (added_columns.additional_filter_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); + + return mapv.size() > 1 ? joinRightColumns( + std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumns( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); +} + + +/// Joins right table columns which indexes are present in right_indexes using specified map. +/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). +template +template +size_t HashJoinMethods::joinRightColumns( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags) +{ + constexpr JoinFeatures join_features; + + size_t rows = added_columns.rows_to_add; + if constexpr (need_filter) + added_columns.filter = IColumn::Filter(rows, 0); + + Arena pool; + + if constexpr (join_features.need_replication) + added_columns.offsets_to_replicate = std::make_unique(rows); + + IColumn::Offset current_offset = 0; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t i = 0; + for (; i < rows; ++i) + { + if constexpr (join_features.need_replication) + { + if (unlikely(current_offset >= max_joined_block_rows)) + { + added_columns.offsets_to_replicate->resize_assume_reserved(i); + added_columns.filter.resize_assume_reserved(i); + break; + } + } + + bool right_row_found = false; + + KnownRowsHolder known_rows; + for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) + { + const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; + if (join_keys.null_map && (*join_keys.null_map)[i]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(i); + using FindResult = typename KeyGetter::FindResult; + auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); + + if (find_result.isFound()) + { + right_row_found = true; + auto & mapped = find_result.getMapped(); + if constexpr (join_features.is_asof_join) + { + const IColumn & left_asof_key = added_columns.leftAsofKey(); + + auto row_ref = mapped->findAsof(left_asof_key, i); + if (row_ref.block) + { + setUsed(added_columns.filter, i); + if constexpr (flag_per_row) + used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); + else + used_flags.template setUsed(find_result); + + added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); + } + else + addNotFoundRow(added_columns, current_offset); + } + else if constexpr (join_features.is_all_join) + { + setUsed(added_columns.filter, i); + used_flags.template setUsed(find_result); + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + } + else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) + { + /// Use first appeared left key + it needs left columns replication + bool used_once = used_flags.template setUsedOnce(find_result); + if (used_once) + { + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + setUsed(added_columns.filter, i); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + } + } + else if constexpr (join_features.is_any_join && join_features.inner) + { + bool used_once = used_flags.template setUsedOnce(find_result); + + /// Use first appeared left key only + if (used_once) + { + setUsed(added_columns.filter, i); + added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); + } + + break; + } + else if constexpr (join_features.is_any_join && join_features.full) + { + /// TODO + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(find_result); + } + else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) + { + setUsed(added_columns.filter, i); + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); + + if (join_features.is_any_or_semi_join) + { + break; + } + } + } + } + + if (!right_row_found) + { + if constexpr (join_features.is_anti_join && join_features.left) + setUsed(added_columns.filter, i); + addNotFoundRow(added_columns, current_offset); + } + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[i] = current_offset; + } + } + + added_columns.applyLazyDefaults(); + return i; +} + +template +template +void HashJoinMethods::setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]) +{ + if constexpr (need_filter) + filter[pos] = 1; +} + +template +template +ColumnPtr HashJoinMethods::buildAdditionalFilter( + size_t left_start_row, + const std::vector & selected_rows, + const std::vector & row_replicate_offset, + AddedColumns & added_columns) +{ + ColumnPtr result_column; + do + { + if (selected_rows.empty()) + { + result_column = ColumnUInt8::create(); + break; + } + const Block & sample_right_block = *selected_rows.begin()->block; + if (!sample_right_block || !added_columns.additional_filter_expression) + { + auto filter = ColumnUInt8::create(); + filter->insertMany(1, selected_rows.size()); + result_column = std::move(filter); + break; + } + + auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); + if (required_cols.empty()) + { + Block block; + added_columns.additional_filter_expression->execute(block); + result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); + break; + } + NameSet required_column_names; + for (auto & col : required_cols) + required_column_names.insert(col.name); + + Block executed_block; + size_t right_col_pos = 0; + for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) + { + if (required_column_names.contains(col.name)) + { + auto new_col = col.column->cloneEmpty(); + for (const auto & selected_row : selected_rows) + { + const auto & src_col = selected_row.block->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row.row_num); + } + executed_block.insert({std::move(new_col), col.type, col.name}); + } + right_col_pos += 1; + } + if (!executed_block) + { + result_column = ColumnUInt8::create(); + break; + } + + for (const auto & col_name : required_column_names) + { + const auto * src_col = added_columns.left_block.findByName(col_name); + if (!src_col) + continue; + auto new_col = src_col->column->cloneEmpty(); + size_t prev_left_offset = 0; + for (size_t i = 1; i < row_replicate_offset.size(); ++i) + { + const size_t & left_offset = row_replicate_offset[i]; + size_t rows = left_offset - prev_left_offset; + if (rows) + new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); + prev_left_offset = left_offset; + } + executed_block.insert({std::move(new_col), src_col->type, col_name}); + } + if (!executed_block) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", + required_cols.toString(), + sample_right_block.dumpNames(), + added_columns.left_block.dumpNames()); + } + + for (const auto & col : executed_block.getColumnsWithTypeAndName()) + if (!col.column || !col.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); + + added_columns.additional_filter_expression->execute(executed_block); + result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); + executed_block.clear(); + } while (false); + + result_column = result_column->convertToFullIfNeeded(); + if (result_column->isNullable()) + { + /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros + /// Trying to avoid copying data, since we are the only owner of the column. + ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); + + MutableColumnPtr mutable_column; + { + ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); + result_column.reset(); + mutable_column = IColumn::mutate(std::move(nested_column)); + } + + auto & column_data = assert_cast(*mutable_column).getData(); + const auto & mask_column_data = assert_cast(*mask_column).getData(); + for (size_t i = 0; i < column_data.size(); ++i) + { + if (mask_column_data[i]) + column_data[i] = 0; + } + return mutable_column; + } + return result_column; +} + +template +template +size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], + bool need_filter [[maybe_unused]], + bool flag_per_row [[maybe_unused]]) +{ + constexpr JoinFeatures join_features; + size_t left_block_rows = added_columns.rows_to_add; + if (need_filter) + added_columns.filter = IColumn::Filter(left_block_rows, 0); + + std::unique_ptr pool; + + if constexpr (join_features.need_replication) + added_columns.offsets_to_replicate = std::make_unique(left_block_rows); + + std::vector row_replicate_offset; + row_replicate_offset.reserve(left_block_rows); + + using FindResult = typename KeyGetter::FindResult; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t left_row_iter = 0; + PreSelectedRows selected_rows; + selected_rows.reserve(left_block_rows); + std::vector find_results; + find_results.reserve(left_block_rows); + bool exceeded_max_block_rows = false; + IColumn::Offset total_added_rows = 0; + IColumn::Offset current_added_rows = 0; + + auto collect_keys_matched_rows_refs = [&]() + { + pool = std::make_unique(); + find_results.clear(); + row_replicate_offset.clear(); + row_replicate_offset.push_back(0); + current_added_rows = 0; + selected_rows.clear(); + for (; left_row_iter < left_block_rows; ++left_row_iter) + { + if constexpr (join_features.need_replication) + { + if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) + { + break; + } + } + KnownRowsHolder all_flag_known_rows; + KnownRowsHolder single_flag_know_rows; + for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) + { + const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; + if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); + auto find_result = row_acceptable + ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) + : FindResult(); + + if (find_result.isFound()) + { + auto & mapped = find_result.getMapped(); + find_results.push_back(find_result); + if (flag_per_row) + addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); + else + addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); + } + } + row_replicate_offset.push_back(current_added_rows); + } + }; + + auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) + { + const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); + + size_t prev_replicated_row = 0; + auto selected_right_row_it = selected_rows.begin(); + size_t find_result_index = 0; + for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) + { + bool any_matched = false; + /// For right join, flag_per_row is true, we need mark used flags for each row. + if (flag_per_row) + { + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + if constexpr (join_features.is_semi_join || join_features.is_any_join) + { + auto used_once = used_flags.template setUsedOnce( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + else + { + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + used_flags.template setUsed( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + } + ++selected_right_row_it; + } + } + else + { + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if constexpr (join_features.is_anti_join) + { + any_matched |= filter_flags[replicated_row]; + } + else if constexpr (join_features.need_replication) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; + } + else + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + total_added_rows += 1; + selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; + break; + } + else + ++selected_right_row_it; + } + } + } + + + if constexpr (join_features.is_anti_join) + { + if (!any_matched) + { + if constexpr (join_features.left) + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + addNotFoundRow(added_columns, total_added_rows); + } + } + else + { + if (!any_matched) + { + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if (!flag_per_row) + used_flags.template setUsed(find_results[find_result_index]); + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + if constexpr (join_features.add_missing) + added_columns.applyLazyDefaults(); + } + } + find_result_index += (prev_replicated_row != row_replicate_offset[i]); + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; + } + prev_replicated_row = row_replicate_offset[i]; + } + }; + + while (left_row_iter < left_block_rows && !exceeded_max_block_rows) + { + auto left_start_row = left_row_iter; + collect_keys_matched_rows_refs(); + if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " + "left_start_row: {}", + selected_rows.size(), + current_added_rows, + row_replicate_offset.size(), + left_row_iter, + left_start_row); + } + auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); + copy_final_matched_rows(left_start_row, filter_col); + + if constexpr (join_features.need_replication) + { + // Add a check for current_added_rows to avoid run the filter expression on too small size batch. + if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) + exceeded_max_block_rows = true; + } + } + + if constexpr (join_features.need_replication) + { + added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); + added_columns.filter.resize_assume_reserved(left_row_iter); + } + added_columns.applyLazyDefaults(); + return left_row_iter; +} + +template +Block HashJoinMethods::sliceBlock(Block & block, size_t num_rows) +{ + size_t total_rows = block.rows(); + if (num_rows >= total_rows) + return {}; + size_t remaining_rows = total_rows - num_rows; + Block remaining_block = block.cloneEmpty(); + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = block.getByPosition(i); + remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows); + col.column = col.column->cut(0, num_rows); + } + return remaining_block; +} + +template +ColumnWithTypeAndName HashJoinMethods::copyLeftKeyColumnToRight( + const DataTypePtr & right_key_type, + const String & renamed_right_column, + const ColumnWithTypeAndName & left_column, + const IColumn::Filter * null_map_filter) +{ + ColumnWithTypeAndName right_column = left_column; + right_column.name = renamed_right_column; + + if (null_map_filter) + right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter); + + bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type); + if (null_map_filter) + correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter); + else + correctNullabilityInplace(right_column, should_be_nullable); + + if (!right_column.type->equals(*right_key_type)) + { + right_column.column = castColumnAccurate(right_column, right_key_type); + right_column.type = right_key_type; + } + + right_column.column = right_column.column->convertToFullColumnIfConst(); + return right_column; +} + +template +void HashJoinMethods::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable) +{ + if (nullable) + { + JoinCommon::convertColumnToNullable(column); + } + else + { + /// We have to replace values masked by NULLs with defaults. + if (column.column) + if (const auto * nullable_column = checkAndGetColumn(&*column.column)) + column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true); + + JoinCommon::removeColumnNullability(column); + } +} + +template +void HashJoinMethods::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) +{ + if (nullable) + { + JoinCommon::convertColumnToNullable(column); + if (column.type->isNullable() && !negative_null_map.empty()) + { + MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column)); + assert_cast(*mutable_column).applyNegatedNullMap(negative_null_map); + column.column = std::move(mutable_column); + } + } + else + JoinCommon::removeColumnNullability(column); +} +} + diff --git a/src/Interpreters/HashJoin/InnerHashJoin.cpp b/src/Interpreters/HashJoin/InnerHashJoin.cpp index 85aedf3a8e5..258e3143996 100644 --- a/src/Interpreters/HashJoin/InnerHashJoin.cpp +++ b/src/Interpreters/HashJoin/InnerHashJoin.cpp @@ -1,5 +1,5 @@ -#include +#include namespace DB { diff --git a/src/Interpreters/HashJoin/LeftHashJoin.cpp b/src/Interpreters/HashJoin/LeftHashJoin.cpp index a53ffaac0b5..4e06789570e 100644 --- a/src/Interpreters/HashJoin/LeftHashJoin.cpp +++ b/src/Interpreters/HashJoin/LeftHashJoin.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/Interpreters/HashJoin/RightHashJoin.cpp b/src/Interpreters/HashJoin/RightHashJoin.cpp index 8e304754f5c..d9d41d7d63c 100644 --- a/src/Interpreters/HashJoin/RightHashJoin.cpp +++ b/src/Interpreters/HashJoin/RightHashJoin.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { From bd4d648f63d56d11c729f719f864a8b9985c43c3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 23 Jul 2024 10:41:57 +0800 Subject: [PATCH 0553/1722] update doc --- docs/en/sql-reference/statements/select/join.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 96d9d26977d..b228f7025c4 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -186,7 +186,7 @@ Otherwise, you'll get `INVALID_JOIN_ON_EXPRESSION`. ::: -Clickhouse currently supports `ALL INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`. +Clickhouse currently supports `ALL/ANY/SEMI/ANTI INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`. **Example** From 58b7ac2264eb6d2ba83d634e1d52b874ca54d9c7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 7 Aug 2024 11:36:01 +0800 Subject: [PATCH 0554/1722] update --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoinMethods.h | 1 + src/Interpreters/HashJoin/InnerHashJoin.cpp | 1 + src/Interpreters/joinDispatch.h | 20 +++-- ..._join_on_inequal_expression_fast.reference | 78 +++++++++++++++++++ ...006_join_on_inequal_expression_fast.sql.j2 | 31 ++++++++ 6 files changed, 127 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 769cb574ed7..4033d1e3035 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1327,7 +1327,7 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona bool is_supported = ((strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind))) || ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti) - && (isLeft(kind) || isRight(kind))); + && (isLeft(kind) || isRight(kind))) || (strictness == JoinStrictness::Any && (isInner(kind))); if (!is_supported) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index e3b8fbc1737..3b7a67467e3 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -185,6 +185,7 @@ extern template class HashJoinMethods; extern template class HashJoinMethods; +extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; extern template class HashJoinMethods; diff --git a/src/Interpreters/HashJoin/InnerHashJoin.cpp b/src/Interpreters/HashJoin/InnerHashJoin.cpp index 258e3143996..69f4c620cb8 100644 --- a/src/Interpreters/HashJoin/InnerHashJoin.cpp +++ b/src/Interpreters/HashJoin/InnerHashJoin.cpp @@ -5,6 +5,7 @@ namespace DB { template class HashJoinMethods; template class HashJoinMethods; +template class HashJoinMethods; template class HashJoinMethods; template class HashJoinMethods; template class HashJoinMethods; diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 982c56e8210..5d4bd8f92e5 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -12,6 +12,15 @@ namespace DB { +/// HashJoin::MapsOne is more efficient, it only store one row for each key in the map. It is recommended to use it whenever possible. +/// When only need to match only one row from right table, use HashJoin::MapsOne. For example, LEFT ANY/SEMI/ANTI. +/// +/// HashJoin::MapsAll will store all rows for each key in the map. It is used when need to match multiple rows from right table. +/// For example, RIGHT ANY/ALL, FULL JOIN, INNER JOIN. +/// +/// prefer_use_maps_all is true when there is mixed inequal condition in the join condition. For example, `t1.a = t2.a AND t1.b > t2.b`. +/// In this case, we need to use HashJoin::MapsAll to store all rows for each key in the map. We will select all matched rows from the map +/// and filter them by `t1.b > t2.b`. template struct MapGetter; @@ -21,8 +30,9 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; -template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; @@ -33,14 +43,14 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. +/// Only ANTI LEFT and ANTI RIGHT are valid. INNER and FULL are here for templates instantiation. template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 3e413afd98e..11ac01d24d5 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -1,3 +1,4 @@ +03006_join_on_inequal_expression_fast.sql -- { echoOn } SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); @@ -596,6 +597,39 @@ SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2. 0 0 \N key4 F 1 1 1 SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); SET join_algorithm='hash'; +SELECT t1.* FROM t1 LEFT ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 b 2 3 2 +key1 c 3 2 1 +key1 d 4 7 2 +key1 e 5 5 5 +key2 a2 1 1 1 +key4 f 2 3 4 +SELECT t1.* FROM t1 LEFT SEMI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 b 2 3 2 +key1 c 3 2 1 +key1 d 4 7 2 +key2 a2 1 1 1 +key4 f 2 3 4 +SELECT t1.* FROM t1 LEFT ANTI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 e 5 5 5 +SELECT t1.* FROM t1 RIGHT ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +SELECT t1.* FROM t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +key1 a 1 1 2 +SELECT t1.* FROM t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); key1 a 1 1 2 key1 A 1 2 1 key1 a 1 1 2 key1 B 2 1 2 @@ -666,3 +700,47 @@ key2 a2 1 1 1 key1 A 1 2 1 key2 a2 1 1 1 key3 a3 1 1 1 key2 a2 1 1 1 key4 F 1 1 1 key4 f 2 3 4 key1 B 2 1 2 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 c 3 2 1 key1 B 2 1 2 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SET join_algorithm='hash'; +SELECT t1.* FROM t1 INNER ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +key1 a 1 1 2 +key1 b 2 3 2 +key1 c 3 2 1 +key1 d 4 7 2 +key2 a2 1 1 1 +key4 f 2 3 4 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index 1bf5a7870e7..b300881c562 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -34,12 +34,43 @@ SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_stri {% endfor -%} {% endfor -%} +{% for algorithm in ['hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'RIGHT'] -%} +{% for join_strictness in ['ANY', 'SEMI', 'ANTI'] -%} +SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} +{% endfor -%} + {% for algorithm in ['hash'] -%} SET join_algorithm='{{ algorithm }}'; {% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); {% endfor -%} {% endfor -%} + +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['INNER'] -%} +{% for join_strictness in ['ANY'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_strictness }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +{% endfor -%} +{% endfor -%} +{% endfor -%} + +{% for algorithm in ['hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['INNER'] -%} +{% for join_strictness in ['ANY'] -%} +SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} +{% endfor -%} + -- { echoOff } -- test error messages From 5eb896b9f1976feaa423071919e65d22e09da4ea Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 10:43:41 +0200 Subject: [PATCH 0555/1722] Add documentation for toDecimal32 and variants --- .../functions/type-conversion-functions.md | 471 ++++++++++++------ 1 file changed, 305 insertions(+), 166 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 1e618b8cdab..24055bb99b7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -95,7 +95,7 @@ SELECT toInt8(-8), toInt8(-8.8), toInt8('-8') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -156,7 +156,7 @@ Query: SELECT toInt8OrZero('-8'), toInt8OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -216,7 +216,7 @@ Query: SELECT toInt8OrNull('-8'), toInt8OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -281,7 +281,7 @@ Query: SELECT toInt8OrDefault('-8', CAST('-1', 'Int8')), toInt8OrDefault('abc', CAST('-1', 'Int8')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -345,7 +345,7 @@ SELECT toInt16(-16), toInt16(-16.16), toInt16('-16') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -406,7 +406,7 @@ Query: SELECT toInt16OrZero('-16'), toInt16OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -466,7 +466,7 @@ Query: SELECT toInt16OrNull('-16'), toInt16OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -531,7 +531,7 @@ Query: SELECT toInt16OrDefault('-16', CAST('-1', 'Int16')), toInt16OrDefault('abc', CAST('-1', 'Int16')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -595,7 +595,7 @@ SELECT toInt32(-32), toInt32(-32.32), toInt32('-32') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -656,7 +656,7 @@ Query: SELECT toInt32OrZero('-32'), toInt32OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -715,7 +715,7 @@ Query: SELECT toInt32OrNull('-32'), toInt32OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -780,7 +780,7 @@ Query: SELECT toInt32OrDefault('-32', CAST('-1', 'Int32')), toInt32OrDefault('abc', CAST('-1', 'Int32')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -844,7 +844,7 @@ SELECT toInt64(-64), toInt64(-64.64), toInt64('-64') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -905,7 +905,7 @@ Query: SELECT toInt64OrZero('-64'), toInt64OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -965,7 +965,7 @@ Query: SELECT toInt64OrNull('-64'), toInt64OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1030,7 +1030,7 @@ Query: SELECT toInt64OrDefault('-64', CAST('-1', 'Int64')), toInt64OrDefault('abc', CAST('-1', 'Int64')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1093,7 +1093,7 @@ SELECT toInt128(-128), toInt128(-128.8), toInt128('-128') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1154,7 +1154,7 @@ Query: SELECT toInt128OrZero('-128'), toInt128OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1214,7 +1214,7 @@ Query: SELECT toInt128OrNull('-128'), toInt128OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1279,7 +1279,7 @@ Query: SELECT toInt128OrDefault('-128', CAST('-1', 'Int128')), toInt128OrDefault('abc', CAST('-1', 'Int128')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1342,7 +1342,7 @@ SELECT toInt256(-256), toInt256(-256.256), toInt256('-256') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1403,7 +1403,7 @@ Query: SELECT toInt256OrZero('-256'), toInt256OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1463,7 +1463,7 @@ Query: SELECT toInt256OrNull('-256'), toInt256OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1528,7 +1528,7 @@ Query: SELECT toInt256OrDefault('-256', CAST('-1', 'Int256')), toInt256OrDefault('abc', CAST('-1', 'Int256')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1592,7 +1592,7 @@ SELECT toUInt8(8), toUInt8(8.8), toUInt8('8') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1653,7 +1653,7 @@ Query: SELECT toUInt8OrZero('-8'), toUInt8OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1713,7 +1713,7 @@ Query: SELECT toUInt8OrNull('8'), toUInt8OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1778,7 +1778,7 @@ Query: SELECT toUInt8OrDefault('8', CAST('0', 'UInt8')), toUInt8OrDefault('abc', CAST('0', 'UInt8')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1842,7 +1842,7 @@ SELECT toUInt16(16), toUInt16(16.16), toUInt16('16') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1903,7 +1903,7 @@ Query: SELECT toUInt16OrZero('16'), toUInt16OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -1963,7 +1963,7 @@ Query: SELECT toUInt16OrNull('16'), toUInt16OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2028,7 +2028,7 @@ Query: SELECT toUInt16OrDefault('16', CAST('0', 'UInt16')), toUInt16OrDefault('abc', CAST('0', 'UInt16')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2092,7 +2092,7 @@ SELECT toUInt32(32), toUInt32(32.32), toUInt32('32') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2154,7 +2154,7 @@ Query: SELECT toUInt32OrZero('32'), toUInt32OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2214,7 +2214,7 @@ Query: SELECT toUInt32OrNull('32'), toUInt32OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2279,7 +2279,7 @@ Query: SELECT toUInt32OrDefault('32', CAST('0', 'UInt32')), toUInt32OrDefault('abc', CAST('0', 'UInt32')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2343,7 +2343,7 @@ SELECT toUInt64(64), toUInt64(64.64), toUInt64('64') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2404,7 +2404,7 @@ Query: SELECT toUInt64OrZero('64'), toUInt64OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2464,7 +2464,7 @@ Query: SELECT toUInt64OrNull('64'), toUInt64OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2529,7 +2529,7 @@ Query: SELECT toUInt64OrDefault('64', CAST('0', 'UInt64')), toUInt64OrDefault('abc', CAST('0', 'UInt64')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2592,7 +2592,7 @@ SELECT toUInt128(128), toUInt128(128.8), toUInt128('128') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2653,7 +2653,7 @@ Query: SELECT toUInt128OrZero('128'), toUInt128OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2713,7 +2713,7 @@ Query: SELECT toUInt128OrNull('128'), toUInt128OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2778,7 +2778,7 @@ Query: SELECT toUInt128OrDefault('128', CAST('0', 'UInt128')), toUInt128OrDefault('abc', CAST('0', 'UInt128')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2841,7 +2841,7 @@ SELECT toUInt256(256), toUInt256(256.256), toUInt256('256') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2902,7 +2902,7 @@ Query: SELECT toUInt256OrZero('256'), toUInt256OrZero('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -2962,7 +2962,7 @@ Query: SELECT toUInt256OrNull('256'), toUInt256OrNull('abc') -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3027,7 +3027,7 @@ Query: SELECT toUInt256OrDefault('-256', CAST('0', 'UInt256')), toUInt256OrDefault('abc', CAST('0', 'UInt256')) -FORMAT vertical; +FORMAT Vertical; ``` Result: @@ -3542,173 +3542,312 @@ SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Asia/Istanbul') AS value, toTypeN ## toDateTime64OrDefault -## toDecimal(32\|64\|128\|256) +## toDecimal32 -Converts `value` to the [Decimal](../data-types/decimal.md) data type with precision of `S`. The `value` can be a number or a string. The `S` (scale) parameter specifies the number of decimal places. +Converts an input value to a value of type [`Decimal(9, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. -- `toDecimal32(value, S)` -- `toDecimal64(value, S)` -- `toDecimal128(value, S)` -- `toDecimal256(value, S)` +**Syntax** -## toDecimal(32\|64\|128\|256)OrNull - -Converts an input string to a [Nullable(Decimal(P,S))](../data-types/decimal.md) data type value. This family of functions includes: - -- `toDecimal32OrNull(expr, S)` — Results in `Nullable(Decimal32(S))` data type. -- `toDecimal64OrNull(expr, S)` — Results in `Nullable(Decimal64(S))` data type. -- `toDecimal128OrNull(expr, S)` — Results in `Nullable(Decimal128(S))` data type. -- `toDecimal256OrNull(expr, S)` — Results in `Nullable(Decimal256(S))` data type. - -These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `NULL` value instead of an exception in the event of an input value parsing error. +```sql +toDecimal32(expr, S) +``` **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions), returns a value in the [String](../data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: **Returned value** -A value in the `Nullable(Decimal(P,S))` data type. The value contains: +- Value of type `Decimal(9, S)`. [Decimal32(S)](../data-types/int-uint.md). -- Number with `S` decimal places, if ClickHouse interprets the input string as a number. -- `NULL`, if ClickHouse can’t interpret the input string as a number or if the input number contains more than `S` decimal places. - -**Examples** +**Example** Query: -``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); +```sql +SELECT + toDecimal32(2, 1) AS a, toTypeName(a) AS type_a, + toDecimal32(4.2, 2) AS b, toTypeName(b) AS type_b, + toDecimal32('4.2', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; ``` Result: ```response -┌────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┠-│ -1.111 │ Nullable(Decimal(9, 5)) │ -└────────┴────────────────────────────────────────────────────┘ +Row 1: +────── +a: 2 +type_a: Decimal(9, 1) +b: 4.2 +type_b: Decimal(9, 2) +c: 4.2 +type_c: Decimal(9, 3) ``` -Query: +**See also** -``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); +- [`toDecimal32OrZero`](#todecimal32orzero). +- [`toDecimal32OrNull`](#todecimal32ornull). +- [`toDecimal32OrDefault`](#todecimal32ordefault). + +## toDecimal32OrZero + +Like [`toDecimal32`](#todecimal32), this function converts an input value to a value of type [Decimal(9, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal32OrZero(expr, S) ``` -Result: - -```response -┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┠-│ á´ºáµá´¸á´¸ │ Nullable(Decimal(9, 2)) │ -└──────┴────────────────────────────────────────────────────┘ -``` - - -## toDecimal(32\|64\|128\|256)OrDefault - -Converts an input string to a [Decimal(P,S)](../data-types/decimal.md) data type value. This family of functions includes: - -- `toDecimal32OrDefault(expr, S)` — Results in `Decimal32(S)` data type. -- `toDecimal64OrDefault(expr, S)` — Results in `Decimal64(S)` data type. -- `toDecimal128OrDefault(expr, S)` — Results in `Decimal128(S)` data type. -- `toDecimal256OrDefault(expr, S)` — Results in `Decimal256(S)` data type. - -These functions should be used instead of `toDecimal*()` functions, if you prefer to get a default value instead of an exception in the event of an input value parsing error. - **Arguments** -- `expr` — [Expression](../syntax.md/#syntax-expressions), returns a value in the [String](../data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrZero('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: **Returned value** -A value in the `Decimal(P,S)` data type. The value contains: - -- Number with `S` decimal places, if ClickHouse interprets the input string as a number. -- Default `Decimal(P,S)` data type value, if ClickHouse can’t interpret the input string as a number or if the input number contains more than `S` decimal places. - -**Examples** - -Query: - -``` sql -SELECT toDecimal32OrDefault(toString(-1.111), 5) AS val, toTypeName(val); -``` - -Result: - -```response -┌────val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 5))─┠-│ -1.111 │ Decimal(9, 5) │ -└────────┴───────────────────────────────────────────────────────┘ -``` - -Query: - -``` sql -SELECT toDecimal32OrDefault(toString(-1.111), 2) AS val, toTypeName(val); -``` - -Result: - -```response -┌─val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 2))─┠-│ 0 │ Decimal(9, 2) │ -└─────┴───────────────────────────────────────────────────────┘ -``` - -## toDecimal(32\|64\|128\|256)OrZero - -Converts an input value to the [Decimal(P,S)](../data-types/decimal.md) data type. This family of functions includes: - -- `toDecimal32OrZero( expr, S)` — Results in `Decimal32(S)` data type. -- `toDecimal64OrZero( expr, S)` — Results in `Decimal64(S)` data type. -- `toDecimal128OrZero( expr, S)` — Results in `Decimal128(S)` data type. -- `toDecimal256OrZero( expr, S)` — Results in `Decimal256(S)` data type. - -These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `0` value instead of an exception in the event of an input value parsing error. - -**Arguments** - -- `expr` — [Expression](../syntax.md/#syntax-expressions), returns a value in the [String](../data-types/string.md) data type. ClickHouse expects the textual representation of the decimal number. For example, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. - -**Returned value** - -A value in the `Nullable(Decimal(P,S))` data type. The value contains: - -- Number with `S` decimal places, if ClickHouse interprets the input string as a number. -- 0 with `S` decimal places, if ClickHouse can’t interpret the input string as a number or if the input number contains more than `S` decimal places. +- Value of type `Decimal(9, S)` if successful, otherwise `0` with `S` decimal places. [Decimal32(S)](../data-types/decimal.md). **Example** Query: ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); +SELECT + toDecimal32OrZero(toString(-1.111), 5) AS val, + toTypeName(val) +FORMAT Vertical; ``` Result: ```response -┌────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┠-│ -1.111 │ Decimal(9, 5) │ -└────────┴────────────────────────────────────────────────────┘ +Row 1: +────── +val: -1.111 +toTypeName(val): Decimal(9, 5) ``` Query: ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); +SELECT + toDecimal32OrZero(toString(-1.111), 2) AS val, + toTypeName(val) +FORMAT Vertical; ``` Result: ```response -┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┠-│ 0.00 │ Decimal(9, 2) │ -└──────┴────────────────────────────────────────────────────┘ +Row 1: +────── +val: -1.11 +toTypeName(val): Decimal(9, 2) +``` + +**See also** + +- [`toDecimal32`](#todecimal32). +- [`toDecimal32OrNull`](#todecimal32ornull). +- [`toDecimal32OrDefault`](#todecimal32ordefault). + +## toDecimal32OrNull + +Like [`toDecimal32`](#todecimal32), this function converts an input value to a value of type [Nullable(Decimal(9, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal32OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrNull('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(9, S))` if successful, otherwise value `NULL` of the same type. [Decimal32(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal32OrNull(toString(-1.111), 5) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.111 +toTypeName(val): Nullable(Decimal(9, 5)) +``` + +Query: + +``` sql +SELECT + toDecimal32OrNull(toString(-1.111), 2) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.11 +toTypeName(val): Nullable(Decimal(9, 2)) +``` + +**See also** + +- [`toDecimal32`](#todecimal32). +- [`toDecimal32OrZero`](#todecimal32orzero). +- [`toDecimal32OrDefault`](#todecimal32ordefault). + +## toDecimal32OrDefault + +Like [`toDecimal32`](#todecimal32), this function converts an input value to a value of type [Decimal(9, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal32OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal32(S)` is unsuccessful. [Decimal32(S](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrDefault('0xc0fe', 1);`. + +:::note +Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(9, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal32(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal32OrDefault(toString(-1.111), 5) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.111 +toTypeName(val): Decimal(9, 5) +``` + +Query: + +``` sql +SELECT + toDecimal32OrDefault(toString(-1.111), 2) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: -1.11 +toTypeName(val): Decimal(9, 2) +``` + +Query: + +``` sql +SELECT + toDecimal32OrDefault('Inf', 2, CAST('0', 'Decimal32(2)')) AS val, + toTypeName(val) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +val: 0 +toTypeName(val): Decimal(9, 2) ``` ## toString From 4f2b1c36b7115143a23462282dc5474ed5b90afd Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 11:12:09 +0200 Subject: [PATCH 0556/1722] Fix typo from previous PR --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 24055bb99b7..5db44da3e2d 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -2362,7 +2362,7 @@ toUInt64('64'): 64 - [`toUInt64OrNull`](#touint64ornull). - [`toUInt64OrDefault`](#touint64ordefault). -## toInt64OrZero +## toUInt64OrZero Like [`toUInt64`](#touint64), this function converts an input value to a value of type [UInt64](../data-types/int-uint.md) but returns `0` in case of an error. From b76e4acbc0a260f5222249a250066c77d2fcaff8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 7 Aug 2024 11:13:56 +0200 Subject: [PATCH 0557/1722] fix another typo --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 5db44da3e2d..8e72fea7fdb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1546,7 +1546,7 @@ toInt256OrDefault('abc', CAST('-1', 'Int256')): -1 - [`toInt256OrZero`](#toint256orzero). - [`toInt256OrNull`](#toint256ornull). -# toUInt8 +## toUInt8 Converts an input value to a value of type [`UInt8`](../data-types/int-uint.md). Throws an exception in case of an error. From e4134f5a51a1ad6d46c60337b9a3b5f8695d8020 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 7 Aug 2024 09:16:19 +0000 Subject: [PATCH 0558/1722] catch exception in destructor of `LocalFileHolder` Signed-off-by: Duc Canh Le --- src/Storages/Cache/ExternalDataSourceCache.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index cffb1dc9ca3..8c778fd511a 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -57,8 +57,15 @@ LocalFileHolder::~LocalFileHolder() { if (original_readbuffer) { - assert_cast(original_readbuffer.get())->seek(0, SEEK_SET); - file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool); + try + { + assert_cast(original_readbuffer.get())->seek(0, SEEK_SET); + file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool); + } + catch (...) + { + tryLogCurrentException(getLogger("LocalFileHolder"), "Exception during destructor of LocalFileHolder."); + } } } From 6172c56c1fd27f39d11914542f9e2dcb94fffd36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 09:48:38 +0000 Subject: [PATCH 0559/1722] Split tests to separate vaguely correlated tests --- ...3217_filtering_in_storage_merge.reference} | 6 +---- .../03217_filtering_in_storage_merge.sql | 16 +++++++++++ ...03217_filtering_in_system_tables.reference | 4 +++ ...l => 03217_filtering_in_system_tables.sql} | 27 +++++++------------ 4 files changed, 30 insertions(+), 23 deletions(-) rename tests/queries/0_stateless/{03217_read_rows_in_system_tables.reference => 03217_filtering_in_storage_merge.reference} (54%) create mode 100644 tests/queries/0_stateless/03217_filtering_in_storage_merge.sql create mode 100644 tests/queries/0_stateless/03217_filtering_in_system_tables.reference rename tests/queries/0_stateless/{03217_read_rows_in_system_tables.sql => 03217_filtering_in_system_tables.sql} (55%) diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_storage_merge.reference similarity index 54% rename from tests/queries/0_stateless/03217_read_rows_in_system_tables.reference rename to tests/queries/0_stateless/03217_filtering_in_storage_merge.reference index b21ead49b1e..d366ad04c39 100644 --- a/tests/queries/0_stateless/03217_read_rows_in_system_tables.reference +++ b/tests/queries/0_stateless/03217_filtering_in_storage_merge.reference @@ -1,10 +1,6 @@ -information_schema tables -default test_replica_1 r1 Expression ((Project names + Projection)) Aggregating Expression (Before GROUP BY) ReadFromMerge Filter (( + ( + ))) - ReadFromMergeTree (default.test_replica_1) -1 1 -1 1 + ReadFromMergeTree (default.test_03217_merge_replica_1) diff --git a/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql new file mode 100644 index 00000000000..5ecc1e7c672 --- /dev/null +++ b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql @@ -0,0 +1,16 @@ +CREATE TABLE test_03217_merge_replica_1(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_merge_replica', 'r1') + ORDER BY x; +CREATE TABLE test_03217_merge_replica_2(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_merge_replica', 'r2') + ORDER BY x; + + +CREATE TABLE test_03217_all_replicas (x UInt32) + ENGINE = Merge(currentDatabase(), 'test_03217_merge_replica_*'); + +INSERT INTO test_03217_merge_replica_1 SELECT number AS x FROM numbers(10); +SYSTEM SYNC REPLICA test_03217_merge_replica_2; + +-- If the filter on _table is not applied, then the plan will show both replicas +EXPLAIN SELECT _table, count() FROM test_03217_all_replicas WHERE _table = 'test_03217_merge_replica_1' AND x >= 0 GROUP BY _table; diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference new file mode 100644 index 00000000000..218fddf92e0 --- /dev/null +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference @@ -0,0 +1,4 @@ +information_schema tables +default test_03217_system_tables_replica_1 r1 +1 1 +1 1 diff --git a/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql similarity index 55% rename from tests/queries/0_stateless/03217_read_rows_in_system_tables.sql rename to tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 3bea04ccccf..bbc755e478d 100644 --- a/tests/queries/0_stateless/03217_read_rows_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -1,27 +1,18 @@ +-- If filtering is not done correctly on databases, then this query report to read 3 rows, which are: `system.tables`, `information_schema.tables` and `INFORMATION_SCHEMA.tables` SELECT database, table FROM system.tables WHERE database = 'information_schema' AND table = 'tables'; --- To verify StorageSystemReplicas applies the filter properly -CREATE TABLE test_replica_1(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r1') +CREATE TABLE test_03217_system_tables_replica_1(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_system_tables_replica', 'r1') ORDER BY x; -CREATE TABLE test_replica_2(x UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217/test_replica', 'r2') +CREATE TABLE test_03217_system_tables_replica_2(x UInt32) + ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_system_tables_replica', 'r2') ORDER BY x; -SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_replica_1' AND replica_name = 'r1'; - - --- To verify StorageMerge -CREATE TABLE all_replicas (x UInt32) - ENGINE = Merge(currentDatabase(), 'test_replica_*'); - -INSERT INTO test_replica_1 SELECT number AS x FROM numbers(10); -SYSTEM SYNC REPLICA test_replica_2; --- If the filter not applied, then the plan will show both replicas -EXPLAIN SELECT _table, count() FROM all_replicas WHERE _table = 'test_replica_1' AND x >= 0 GROUP BY _table; +-- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables +SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; SYSTEM FLUSH LOGS; --- argMin-argMax make the test repeatable +-- argMin-argMax is necessary to make the test repeatable -- StorageSystemTables SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 @@ -30,5 +21,5 @@ SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_ -- StorageSystemReplicas SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 - AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_replica_1\' AND replica_name = \'r1\';' + AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' AND type = 'QueryFinish'; From bcc5201c99e00998beab8088a988a66f921415b6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 17:17:07 +0200 Subject: [PATCH 0560/1722] init --- src/Common/SystemLogBase.cpp | 138 ++++++++++++-------- src/Common/SystemLogBase.h | 58 +++++--- src/Interpreters/InterpreterSystemQuery.cpp | 9 +- src/Interpreters/SystemLog.cpp | 38 +++--- 4 files changed, 148 insertions(+), 95 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 7d2c15714e2..748cf4744ae 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -27,12 +27,15 @@ #include #include +#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) + namespace DB { namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; + extern const int ABORTED; } ISystemLog::~ISystemLog() = default; @@ -86,9 +89,8 @@ void SystemLogQueue::push(LogElement&& element) // by one, under exclusive lock, so we will see each message count. // It is enough to only wake the flushing thread once, after the message // count increases past half available size. - const uint64_t queue_end = queue_front_index + queue.size(); - requested_flush_up_to = std::max(requested_flush_up_to, queue_end); - + const auto last_log_index = queue_front_index + queue.size(); + requested_flush_index = std::max(requested_flush_index, last_log_index); flush_event.notify_all(); } @@ -127,20 +129,46 @@ template void SystemLogQueue::handleCrash() { if (settings.notify_flush_on_crash) - notifyFlush(/* force */ true); + { + notifyFlush(getLastLogIndex(), /* should_prepare_tables_anyway */ true); + } } template -void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) +void SystemLogQueue::notifyFlush(SystemLogQueue::Index expected_flushed_index, bool should_prepare_tables_anyway) { + std::unique_lock lock(mutex); + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + force_prepare_tables_requested |= should_prepare_tables_anyway; + requested_flush_index = std::max(requested_flush_index, expected_flushed_index); + flush_event.notify_all(); +} + +template +void SystemLogQueue::waitFlush(SystemLogQueue::Index expected_flushed_index, bool should_prepare_tables_anyway) +{ + LOG_DEBUG(log, "Requested flush up to offset {}", expected_flushed_index); + // Use an arbitrary timeout to avoid endless waiting. 60s proved to be // too fast for our parallel functional tests, probably because they // heavily load the disk. const int timeout_seconds = 180; + std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + + // there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered + force_prepare_tables_requested |= should_prepare_tables_anyway; + if (requested_flush_index < expected_flushed_index) { - return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; + requested_flush_index = expected_flushed_index; + flush_event.notify_all(); + } + + auto result = confirm_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + { + const bool if_should_prepare_then_it_is_done = LOGICAL_IF_THEN(should_prepare_tables_anyway, prepare_tables_done); + return (flushed_index >= expected_flushed_index && if_should_prepare_then_it_is_done) || is_shutdown; }); if (!result) @@ -148,67 +176,54 @@ void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", toString(timeout_seconds), demangle(typeid(*this).name())); } -} - -template -uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) -{ - uint64_t this_thread_requested_offset; + if (is_shutdown) { - std::lock_guard lock(mutex); - if (is_shutdown) - return uint64_t(-1); - - this_thread_requested_offset = queue_front_index + queue.size(); - - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= should_prepare_tables_anyway; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); - - flush_event.notify_all(); + throw Exception(ErrorCodes::ABORTED, "Shutdown has been called while flushing system log '{}'. Aborting.", + demangle(typeid(*this).name())); } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; } template -void SystemLogQueue::confirm(uint64_t to_flush_end) +SystemLogQueue::Index SystemLogQueue::getLastLogIndex() { std::lock_guard lock(mutex); - flushed_up_to = to_flush_end; - is_force_prepare_tables = false; - flush_event.notify_all(); + return queue_front_index + queue.size(); } template -typename SystemLogQueue::Index SystemLogQueue::pop(std::vector & output, - bool & should_prepare_tables_anyway, - bool & exit_this_thread) +void SystemLogQueue::confirm(SystemLogQueue::Index last_flashed_index) { - /// Call dtors and deallocate strings without holding the global lock - output.resize(0); + std::lock_guard lock(mutex); + prepare_tables_done = true; + flushed_index = std::max(flushed_index, last_flashed_index); + confirm_event.notify_all(); +} +template +typename SystemLogQueue::PopResult SystemLogQueue::pop() +{ std::unique_lock lock(mutex); - flush_event.wait_for(lock, - std::chrono::milliseconds(settings.flush_interval_milliseconds), - [&] () - { - return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; - } - ); + + flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] () + { + const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; + return requested_flush_index > flushed_index || if_prepare_requested_and_it_is_not_done || is_shutdown; + }); + + if (is_shutdown) + return PopResult{.is_shutdown = true}; queue_front_index += queue.size(); - // Swap with existing array from previous flush, to save memory - // allocations. - queue.swap(output); - should_prepare_tables_anyway = is_force_prepare_tables; + PopResult result; + result.logs_index = queue_front_index; + result.logs_elemets.swap(queue); - exit_this_thread = is_shutdown; - return queue_front_index; + const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; + result.create_table_force = if_prepare_requested_and_it_is_not_done; + + return result; } template @@ -229,13 +244,21 @@ SystemLogBase::SystemLogBase( } template -void SystemLogBase::flush(bool force) +SystemLogBase::Index SystemLogBase::getLastLogIndex() { - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; + return queue->getLastLogIndex(); +} - queue->waitFlush(this_thread_requested_offset); +template +void SystemLogBase::notifyFlush(Index expected_flushed_index) +{ + queue->notifyFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); +} + +template +void SystemLogBase::flush(Index expected_flushed_index) +{ + queue->waitFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); } template @@ -257,9 +280,6 @@ void SystemLogBase::add(LogElement element) queue->push(std::move(element)); } -template -void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } - #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) @@ -267,3 +287,5 @@ SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) } + +#undef LOGICAL_IF_THEN diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index b87fcf419d3..6c60ffafc4c 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -54,10 +54,19 @@ struct StorageID; class ISystemLog { public: + using Index = uint64_t; + virtual String getName() const = 0; - //// force -- force table creation (used for SYSTEM FLUSH LOGS) - virtual void flush(bool force = false) = 0; /// NOLINT + /// Return the index of the lastest added log element. That index no less than the flashed index. + /// The flashed index is the index of the last log element which has been flushed successfully. + /// Thereby all the records whose index is less than the flashed index are flushed already. + virtual Index getLastLogIndex() = 0; + /// Call this method to wake up the flush thread and flush the data in the background. It is non blocking call + virtual void notifyFlush(Index expected_flushed_index) = 0; + /// Call this method to wait intill the logs are flushed up to expected_flushed_index. It is blocking call. + virtual void flush(Index expected_flushed_index) = 0; + virtual void prepareTable() = 0; /// Start the background thread. @@ -97,24 +106,34 @@ struct SystemLogQueueSettings template class SystemLogQueue { - using Index = uint64_t; - public: + using Index = ISystemLog::Index; + explicit SystemLogQueue(const SystemLogQueueSettings & settings_); void shutdown(); // producer methods void push(LogElement && element); - Index notifyFlush(bool should_prepare_tables_anyway); - void waitFlush(Index expected_flushed_up_to); + + Index getLastLogIndex(); + void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway); + void waitFlush(Index expected_flushed_index, bool should_prepare_tables_anyway); /// Handles crash, flushes log without blocking if notify_flush_on_crash is set void handleCrash(); + struct PopResult + { + Index logs_index = 0; + std::vector logs_elemets = {}; + bool create_table_force = false; + bool is_shutdown = false; + }; + // consumer methods - Index pop(std::vector& output, bool & should_prepare_tables_anyway, bool & exit_this_thread); - void confirm(Index to_flush_end); + PopResult pop(); + void confirm(Index last_flashed_index); private: /// Data shared between callers of add()/flush()/shutdown(), and the saving thread @@ -124,22 +143,29 @@ private: // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; + // An always-incrementing index of the first message currently in the queue. // We use it to give a global sequential index to every message, so that we // can wait until a particular message is flushed. This is used to implement // synchronous log flushing for SYSTEM FLUSH LOGS. Index queue_front_index = 0; + // A flag that says we must create the tables even if the queue is empty. - bool is_force_prepare_tables = false; + bool force_prepare_tables_requested = false; + bool prepare_tables_done = false; + // Requested to flush logs up to this index, exclusive - Index requested_flush_up_to = 0; + Index requested_flush_index = 0; + // Flushed log up to this index, exclusive - Index flushed_up_to = 0; + Index flushed_index = 0; + // Logged overflow message at this queue front index Index logged_queue_full_at_index = -1; bool is_shutdown = false; + std::condition_variable confirm_event; std::condition_variable flush_event; const SystemLogQueueSettings settings; @@ -150,6 +176,7 @@ template class SystemLogBase : public ISystemLog { public: + using Index = ISystemLog::Index; using Self = SystemLogBase; explicit SystemLogBase( @@ -163,15 +190,16 @@ public: */ void add(LogElement element); + Index getLastLogIndex() override; + + void notifyFlush(Index expected_flushed_index) override; + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; + void flush(Index expected_flushed_index) override; /// Handles crash, flushes log without blocking if notify_flush_on_crash is set void handleCrash() override; - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - String getName() const override { return LogElement::name(); } static const char * getDefaultOrderBy() { return "event_date, event_time"; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c284acfa308..9b483bac25c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -712,11 +712,18 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); auto logs = getContext()->getSystemLogs(); + std::vector> commands; commands.reserve(logs.size()); for (auto * system_log : logs) - commands.emplace_back([system_log] { system_log->flush(true); }); + { + auto current_index = system_log->getLastLogIndex(); + /// The data is started to being flushed in the background after notifyFlush call + system_log->notifyFlush(current_index); + commands.emplace_back([system_log, current_index] { system_log->flush(current_index); }); + } + /// The data is flashing in the background, we need to wait until it is done executeCommandsAndThrowIfError(commands); break; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 572481e6b12..7042564799a 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -1,6 +1,7 @@ #include #include +#include "Common/SystemLogBase.h" #include #include #include @@ -462,33 +463,26 @@ void SystemLog::savingThreadFunction() { setThreadName("SystemLogFlush"); - std::vector to_flush; - bool exit_this_thread = false; - while (!exit_this_thread) + while (true) { try { - // The end index (exclusive, like std end()) of the messages we are - // going to flush. - uint64_t to_flush_end = 0; - // Should we prepare table even if there are no new messages. - bool should_prepare_tables_anyway = false; + auto result = queue->pop(); - to_flush_end = queue->pop(to_flush, should_prepare_tables_anyway, exit_this_thread); - - if (to_flush.empty()) + if (result.is_shutdown) { - if (should_prepare_tables_anyway) - { - prepareTable(); - LOG_TRACE(log, "Table created (force)"); - - queue->confirm(to_flush_end); - } + LOG_TRACE(log, "Terminating"); + return; } - else + + if (!result.logs_elemets.empty()) { - flushImpl(to_flush, to_flush_end); + flushImpl(result.logs_elemets, result.logs_index); + } + else if (result.create_table_force) + { + prepareTable(); + queue->confirm(/* last_flashed_index */ 0); } } catch (...) @@ -496,7 +490,6 @@ void SystemLog::savingThreadFunction() tryLogCurrentException(__PRETTY_FUNCTION__); } } - LOG_TRACE(log, "Terminating"); } @@ -579,6 +572,9 @@ StoragePtr SystemLog::getStorage() const template void SystemLog::prepareTable() { + if (is_prepared) + return; + String description = table_id.getNameForLogs(); auto table = getStorage(); From aa42ccf0531aa416c5525b6d8c01c057f781b0e3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 19:04:22 +0200 Subject: [PATCH 0561/1722] move LOGICAL_IF_THEN to base/defines.h --- base/base/defines.h | 2 ++ src/Common/SystemLogBase.cpp | 3 --- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 5685a6d9833..7860cebd359 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -164,3 +164,5 @@ template constexpr void UNUSED(Args &&... args [[maybe_unused]]) // NOLINT(cppcoreguidelines-missing-std-forward) { } + +#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 748cf4744ae..6840c461ce6 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -27,7 +27,6 @@ #include #include -#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) namespace DB { @@ -287,5 +286,3 @@ SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) } - -#undef LOGICAL_IF_THEN From abd5dfe1d0c7b88212a2f0dc4ed1a8b470dcedb1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 19:12:13 +0200 Subject: [PATCH 0562/1722] fix typo --- src/Common/SystemLogBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 6c60ffafc4c..3915b99f8aa 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -58,7 +58,7 @@ public: virtual String getName() const = 0; - /// Return the index of the lastest added log element. That index no less than the flashed index. + /// Return the index of the latest added log element. That index no less than the flashed index. /// The flashed index is the index of the last log element which has been flushed successfully. /// Thereby all the records whose index is less than the flashed index are flushed already. virtual Index getLastLogIndex() = 0; From 8e5577ad8f8cb4e4d3ccb977af8536957088b8ca Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 Jul 2024 19:18:03 +0200 Subject: [PATCH 0563/1722] fix includes --- src/Interpreters/SystemLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 7042564799a..c236b524c60 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -1,7 +1,7 @@ #include #include -#include "Common/SystemLogBase.h" +#include #include #include #include From 86267418f9a74915d5089770d658b328684b9189 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 31 Jul 2024 23:28:03 +0200 Subject: [PATCH 0564/1722] fix tests, rework recreation tables conditions, add log about ignored logs --- base/base/defines.h | 2 - src/Backups/BackupsWorker.cpp | 2 + src/Common/SystemLogBase.cpp | 104 +++++++++--------- src/Common/SystemLogBase.h | 28 +++-- src/Interpreters/SystemLog.cpp | 9 +- .../test_system_flush_logs/test.py | 25 ++++- .../test_system_logs_recreate/test.py | 13 ++- 7 files changed, 105 insertions(+), 78 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 7860cebd359..5685a6d9833 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -164,5 +164,3 @@ template constexpr void UNUSED(Args &&... args [[maybe_unused]]) // NOLINT(cppcoreguidelines-missing-std-forward) { } - -#define LOGICAL_IF_THEN(A, B) (!(A) || !!(B)) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 0b93ae6d547..363aaae9c8d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1171,6 +1171,8 @@ void BackupsWorker::waitAll() for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); + backup_log->flush(backup_log->getLastLogIndex()); + LOG_INFO(log, "Backups and restores finished"); } diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 6840c461ce6..a35a46c49cc 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -88,31 +88,18 @@ void SystemLogQueue::push(LogElement&& element) // by one, under exclusive lock, so we will see each message count. // It is enough to only wake the flushing thread once, after the message // count increases past half available size. + const auto last_log_index = queue_front_index + queue.size(); - requested_flush_index = std::max(requested_flush_index, last_log_index); - flush_event.notify_all(); + notifyFlushUnlocked(last_log_index, /* should_prepare_tables_anyway */ false); } if (queue.size() >= settings.max_size_rows) { + chassert(queue.size() == settings.max_size_rows); + // Ignore all further entries until the queue is flushed. - // Log a message about that. Don't spam it -- this might be especially - // problematic in case of trace log. Remember what the front index of the - // queue was when we last logged the message. If it changed, it means the - // queue was flushed, and we can log again. - if (queue_front_index != logged_queue_full_at_index) - { - logged_queue_full_at_index = queue_front_index; - - // TextLog sets its logger level to 0, so this log is a noop and - // there is no recursive logging. - lock.unlock(); - LOG_ERROR(log, "Queue is full for system log '{}' at {}. max_size_rows {}", - demangle(typeid(*this).name()), - queue_front_index, - settings.max_size_rows); - } - + // To the next batch we add a log message about how much we have lost + ++ignored_logs; return; } @@ -133,15 +120,22 @@ void SystemLogQueue::handleCrash() } } +template +void SystemLogQueue::notifyFlushUnlocked(Index expected_flushed_index, bool should_prepare_tables_anyway) +{ + if (should_prepare_tables_anyway) + requested_prepare_tables = std::max(requested_prepare_tables, expected_flushed_index); + + requested_flush_index = std::max(requested_flush_index, expected_flushed_index); + + flush_event.notify_all(); +} + template void SystemLogQueue::notifyFlush(SystemLogQueue::Index expected_flushed_index, bool should_prepare_tables_anyway) { - std::unique_lock lock(mutex); - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - force_prepare_tables_requested |= should_prepare_tables_anyway; - requested_flush_index = std::max(requested_flush_index, expected_flushed_index); - flush_event.notify_all(); + std::lock_guard lock(mutex); + notifyFlushUnlocked(expected_flushed_index, should_prepare_tables_anyway); } template @@ -156,18 +150,15 @@ void SystemLogQueue::waitFlush(SystemLogQueue::Index exp std::unique_lock lock(mutex); - // there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered - force_prepare_tables_requested |= should_prepare_tables_anyway; - if (requested_flush_index < expected_flushed_index) - { - requested_flush_index = expected_flushed_index; - flush_event.notify_all(); - } + // there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered before we wait the result + notifyFlushUnlocked(expected_flushed_index, should_prepare_tables_anyway); auto result = confirm_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { - const bool if_should_prepare_then_it_is_done = LOGICAL_IF_THEN(should_prepare_tables_anyway, prepare_tables_done); - return (flushed_index >= expected_flushed_index && if_should_prepare_then_it_is_done) || is_shutdown; + if (should_prepare_tables_anyway) + return (flushed_index >= expected_flushed_index && prepared_tables >= requested_prepare_tables) || is_shutdown; + else + return (flushed_index >= expected_flushed_index) || is_shutdown; }); if (!result) @@ -194,7 +185,7 @@ template void SystemLogQueue::confirm(SystemLogQueue::Index last_flashed_index) { std::lock_guard lock(mutex); - prepare_tables_done = true; + prepared_tables = std::max(prepared_tables, last_flashed_index); flushed_index = std::max(flushed_index, last_flashed_index); confirm_event.notify_all(); } @@ -202,25 +193,34 @@ void SystemLogQueue::confirm(SystemLogQueue::Index last_ template typename SystemLogQueue::PopResult SystemLogQueue::pop() { - std::unique_lock lock(mutex); - - flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] () - { - const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; - return requested_flush_index > flushed_index || if_prepare_requested_and_it_is_not_done || is_shutdown; - }); - - if (is_shutdown) - return PopResult{.is_shutdown = true}; - - queue_front_index += queue.size(); - PopResult result; - result.logs_index = queue_front_index; - result.logs_elemets.swap(queue); + size_t prev_ignored_logs = 0; - const bool if_prepare_requested_and_it_is_not_done = force_prepare_tables_requested && !prepare_tables_done; - result.create_table_force = if_prepare_requested_and_it_is_not_done; + { + std::unique_lock lock(mutex); + + flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] () + { + return requested_flush_index > flushed_index || requested_prepare_tables > prepared_tables || is_shutdown; + }); + + if (is_shutdown) + return PopResult{.is_shutdown = true}; + + queue_front_index += queue.size(); + prev_ignored_logs = ignored_logs; + ignored_logs = 0; + + result.last_log_index = queue_front_index; + result.logs.swap(queue); + result.create_table_force = requested_prepare_tables > prepared_tables; + } + + if (prev_ignored_logs) + LOG_ERROR(log, "Queue had been full at {}, accepted {} logs, ignored {} logs.", + result.last_log_index - result.logs.size(), + result.logs.size(), + prev_ignored_logs); return result; } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 3915b99f8aa..c359287a73f 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -54,7 +55,7 @@ struct StorageID; class ISystemLog { public: - using Index = uint64_t; + using Index = int64_t; virtual String getName() const = 0; @@ -125,8 +126,8 @@ public: struct PopResult { - Index logs_index = 0; - std::vector logs_elemets = {}; + Index last_log_index = 0; + std::vector logs = {}; bool create_table_force = false; bool is_shutdown = false; }; @@ -136,6 +137,8 @@ public: void confirm(Index last_flashed_index); private: + void notifyFlushUnlocked(Index expected_flushed_index, bool should_prepare_tables_anyway); + /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; @@ -150,18 +153,21 @@ private: // synchronous log flushing for SYSTEM FLUSH LOGS. Index queue_front_index = 0; - // A flag that says we must create the tables even if the queue is empty. - bool force_prepare_tables_requested = false; - bool prepare_tables_done = false; - // Requested to flush logs up to this index, exclusive - Index requested_flush_index = 0; - + Index requested_flush_index = std::numeric_limits::min(); // Flushed log up to this index, exclusive Index flushed_index = 0; - // Logged overflow message at this queue front index - Index logged_queue_full_at_index = -1; + // The same logic for the prepare tables: if requested_prepar_tables > prepared_tables we need to do prepare + // except that initial prepared_tables is -1 + // it is due to the difference: when no logs have been written and we call flush logs + // it becomes in the state: requested_flush_index = 0 and flushed_index = 0 -- we do not want to do anything + // but if we need to prepare tables it becomes requested_prepare_tables = 0 and prepared_tables = -1 + // we trigger background thread and do prepare + Index requested_prepare_tables = std::numeric_limits::min(); + Index prepared_tables = -1; + + size_t ignored_logs = 0; bool is_shutdown = false; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index c236b524c60..9d07184a0e5 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -475,14 +475,14 @@ void SystemLog::savingThreadFunction() return; } - if (!result.logs_elemets.empty()) + if (!result.logs.empty()) { - flushImpl(result.logs_elemets, result.logs_index); + flushImpl(result.logs, result.last_log_index); } else if (result.create_table_force) { prepareTable(); - queue->confirm(/* last_flashed_index */ 0); + queue->confirm(result.last_log_index); } } catch (...) @@ -572,9 +572,6 @@ StoragePtr SystemLog::getStorage() const template void SystemLog::prepareTable() { - if (is_prepared) - return; - String description = table_id.getNameForLogs(); auto table = getStorage(); diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 713b327eb76..0399122406a 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -4,7 +4,7 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain_with_retry, TSV cluster = ClickHouseCluster(__file__) node = cluster.add_instance( @@ -75,6 +75,8 @@ def test_system_suspend(): def test_log_max_size(start_cluster): + # we do misconfiguration here: buffer_size_rows_flush_threshold > max_size_rows, flush_interval_milliseconds is huge + # no auto flush by size not by time has a chance node.exec_in_container( [ "bash", @@ -83,6 +85,7 @@ def test_log_max_size(start_cluster): 1000000 + 1000000 10 10 @@ -91,11 +94,23 @@ def test_log_max_size(start_cluster): """, ] ) - node.restart_clickhouse() - for i in range(10): - node.query(f"select {i}") - assert node.query("select count() >= 10 from system.query_log") == "1\n" + node.query(f"TRUNCATE TABLE IF EXISTS system.query_log") + node.restart_clickhouse() + + # all logs records above max_size_rows are lost + # The accepted logs records are never flushed until system flush logs is called by us + for i in range(21): + node.query(f"select {i}") + node.query("system flush logs") + + assert_logs_contain_with_retry( + node, "Queue had been full at 0, accepted 10 logs, ignored 34 logs." + ) + assert node.query( + "select count() >= 10, count() < 20 from system.query_log" + ) == TSV([[1, 1]]) + node.exec_in_container( ["rm", f"/etc/clickhouse-server/config.d/yyy-override-query_log.xml"] ) diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index 1bdb1fe3261..1a4ed31278d 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -173,11 +173,20 @@ def test_drop_system_log(): node.query("system flush logs") node.query("select 2") node.query("system flush logs") - assert node.query("select count() > 0 from system.query_log") == "1\n" + assert node.query("select count() >= 2 from system.query_log") == "1\n" + node.query("drop table system.query_log sync") node.query("select 3") node.query("system flush logs") - assert node.query("select count() > 0 from system.query_log") == "1\n" + assert node.query("select count() >= 1 from system.query_log") == "1\n" + + node.query("drop table system.query_log sync") + node.restart_clickhouse() + node.query("system flush logs") + assert ( + node.query("select count() >= 0 from system.query_log") == "1\n" + ) # we check that query_log just exists + node.exec_in_container( ["rm", f"/etc/clickhouse-server/config.d/yyy-override-query_log.xml"] ) From 00efd5fc0da7124cb1671487250476185101a488 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 22:23:16 +0000 Subject: [PATCH 0565/1722] Automatic style fix --- tests/integration/test_system_logs_recreate/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index 1a4ed31278d..d5347ae7dea 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -185,7 +185,7 @@ def test_drop_system_log(): node.query("system flush logs") assert ( node.query("select count() >= 0 from system.query_log") == "1\n" - ) # we check that query_log just exists + ) # we check that query_log just exists node.exec_in_container( ["rm", f"/etc/clickhouse-server/config.d/yyy-override-query_log.xml"] From e3290c782066aaf7d1891d865547706387bf773c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Aug 2024 16:58:17 +0200 Subject: [PATCH 0566/1722] rework Context::getSystemLogs, add system logs flush at shutdown --- src/Backups/BackupsWorker.cpp | 4 +- src/Common/SystemLogBase.cpp | 8 +- src/Common/SystemLogBase.h | 8 +- src/Interpreters/Context.cpp | 7 +- src/Interpreters/Context.h | 4 +- src/Interpreters/InterpreterSystemQuery.cpp | 17 +--- src/Interpreters/SystemLog.cpp | 86 ++++++++-------- src/Interpreters/SystemLog.h | 104 +++++++++----------- 8 files changed, 109 insertions(+), 129 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 363aaae9c8d..106aa89082d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1171,7 +1171,7 @@ void BackupsWorker::waitAll() for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); - backup_log->flush(backup_log->getLastLogIndex()); + backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); LOG_INFO(log, "Backups and restores finished"); } @@ -1223,6 +1223,8 @@ void BackupsWorker::cancelAll(bool wait_) for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); + backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); + LOG_INFO(log, "Backups and restores finished or stopped"); } diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index a35a46c49cc..127c8862a35 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -249,15 +249,15 @@ SystemLogBase::Index SystemLogBase::getLastLogIndex() } template -void SystemLogBase::notifyFlush(Index expected_flushed_index) +void SystemLogBase::notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) { - queue->notifyFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); + queue->notifyFlush(expected_flushed_index, should_prepare_tables_anyway); } template -void SystemLogBase::flush(Index expected_flushed_index) +void SystemLogBase::flush(Index expected_flushed_index, bool should_prepare_tables_anyway) { - queue->waitFlush(expected_flushed_index, /* should_prepare_tables_anyway */ true); + queue->waitFlush(expected_flushed_index, should_prepare_tables_anyway); } template diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index c359287a73f..0d7b04d5c57 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -64,9 +64,9 @@ public: /// Thereby all the records whose index is less than the flashed index are flushed already. virtual Index getLastLogIndex() = 0; /// Call this method to wake up the flush thread and flush the data in the background. It is non blocking call - virtual void notifyFlush(Index expected_flushed_index) = 0; + virtual void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) = 0; /// Call this method to wait intill the logs are flushed up to expected_flushed_index. It is blocking call. - virtual void flush(Index expected_flushed_index) = 0; + virtual void flush(Index expected_flushed_index, bool should_prepare_tables_anyway) = 0; virtual void prepareTable() = 0; @@ -198,10 +198,10 @@ public: Index getLastLogIndex() override; - void notifyFlush(Index expected_flushed_index) override; + void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) override; /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(Index expected_flushed_index) override; + void flush(Index expected_flushed_index, bool should_prepare_tables_anyway) override; /// Handles crash, flushes log without blocking if notify_flush_on_crash is set void handleCrash() override; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5413b568068..3051ed3e567 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -99,6 +99,7 @@ #include #include #include +#include #include #include #include @@ -618,7 +619,7 @@ struct ContextSharedPart : boost::noncopyable /** After system_logs have been shut down it is guaranteed that no system table gets created or written to. * Note that part changes at shutdown won't be logged to part log. */ - SHUTDOWN(log, "system logs", system_logs, shutdown()); + SHUTDOWN(log, "system logs", system_logs, flushAndShutdown()); LOG_TRACE(log, "Shutting down database catalog"); DatabaseCatalog::shutdown(); @@ -4312,13 +4313,13 @@ std::shared_ptr Context::getBlobStorageLog() const return shared->system_logs->blob_storage_log; } -std::vector Context::getSystemLogs() const +SystemLogs Context::getSystemLogs() const { SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; - return shared->system_logs->logs; + return *shared->system_logs; } std::optional Context::getDashboards() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5e35c3e4b3..3da4f124553 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -48,6 +48,8 @@ namespace DB class ASTSelectQuery; +class SystemLogs; + struct ContextSharedPart; class ContextAccess; class ContextAccessWrapper; @@ -1150,7 +1152,7 @@ public: std::shared_ptr getBackupLog() const; std::shared_ptr getBlobStorageLog() const; - std::vector getSystemLogs() const; + SystemLogs getSystemLogs() const; using Dashboards = std::vector>; std::optional getDashboards() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 9b483bac25c..ef6d1040c5e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -710,21 +710,8 @@ BlockIO InterpreterSystemQuery::execute() case Type::FLUSH_LOGS: { getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); - - auto logs = getContext()->getSystemLogs(); - - std::vector> commands; - commands.reserve(logs.size()); - for (auto * system_log : logs) - { - auto current_index = system_log->getLastLogIndex(); - /// The data is started to being flushed in the background after notifyFlush call - system_log->notifyFlush(current_index); - commands.emplace_back([system_log, current_index] { system_log->flush(current_index); }); - } - - /// The data is flashing in the background, we need to wait until it is done - executeCommandsAndThrowIfError(commands); + auto system_logs = getContext()->getSystemLogs(); + system_logs.flush(true); break; } case Type::STOP_LISTEN: diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 9d07184a0e5..9b58da3f545 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -50,6 +50,7 @@ #include + namespace DB { @@ -312,56 +313,13 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf azure_queue_log = createSystemLog(global_context, "system", "azure_queue_log", config, "azure_queue_log", "Contains logging entries with the information files processes by S3Queue engine."); blob_storage_log = createSystemLog(global_context, "system", "blob_storage_log", config, "blob_storage_log", "Contains logging entries with information about various blob storage operations such as uploads and deletes."); - if (query_log) - logs.emplace_back(query_log.get()); - if (query_thread_log) - logs.emplace_back(query_thread_log.get()); - if (part_log) - logs.emplace_back(part_log.get()); - if (trace_log) - logs.emplace_back(trace_log.get()); - if (crash_log) - logs.emplace_back(crash_log.get()); - if (text_log) - logs.emplace_back(text_log.get()); - if (metric_log) - logs.emplace_back(metric_log.get()); - if (error_log) - logs.emplace_back(error_log.get()); - if (asynchronous_metric_log) - logs.emplace_back(asynchronous_metric_log.get()); - if (opentelemetry_span_log) - logs.emplace_back(opentelemetry_span_log.get()); - if (query_views_log) - logs.emplace_back(query_views_log.get()); - if (zookeeper_log) - logs.emplace_back(zookeeper_log.get()); if (session_log) - { - logs.emplace_back(session_log.get()); global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); - } - if (transactions_info_log) - logs.emplace_back(transactions_info_log.get()); - if (processors_profile_log) - logs.emplace_back(processors_profile_log.get()); - if (filesystem_cache_log) - logs.emplace_back(filesystem_cache_log.get()); - if (filesystem_read_prefetches_log) - logs.emplace_back(filesystem_read_prefetches_log.get()); - if (asynchronous_insert_log) - logs.emplace_back(asynchronous_insert_log.get()); - if (backup_log) - logs.emplace_back(backup_log.get()); - if (s3_queue_log) - logs.emplace_back(s3_queue_log.get()); - if (blob_storage_log) - logs.emplace_back(blob_storage_log.get()); bool should_prepare = global_context->getServerSettings().prepare_system_log_tables_on_startup; try { - for (auto & log : logs) + for (auto & log : getAllLogs()) { log->startup(); if (should_prepare) @@ -395,20 +353,56 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf } } - -SystemLogs::~SystemLogs() +std::vector SystemLogs::getAllLogs() const { +/// NOLINTBEGIN(bugprone-macro-parentheses) +#define GET_RAW_POINTERS(log_type, member, descr) \ + member.get(), \ + + std::vector result = { + LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) + }; +#undef GET_RAW_POINTERS +/// NOLINTEND(bugprone-macro-parentheses) + + auto last_it = std::remove(result.begin(), result.end(), nullptr); + result.erase(last_it, result.end()); + + return result; +} + +void SystemLogs::flush(bool should_prepare_tables_anyway) +{ + auto logs = getAllLogs(); + std::vector logs_indexes(logs.size(), 0); + + for (size_t i = 0; i < logs.size(); ++i) + { + auto last_log_index = logs[i]->getLastLogIndex(); + logs_indexes[i] = last_log_index; + logs[i]->notifyFlush(last_log_index, should_prepare_tables_anyway); + } + + for (size_t i = 0; i < logs.size(); ++i) + logs[i]->flush(logs_indexes[i], should_prepare_tables_anyway); +} + +void SystemLogs::flushAndShutdown() +{ + flush(/* should_prepare_tables_anyway */ false); shutdown(); } void SystemLogs::shutdown() { + auto logs = getAllLogs(); for (auto & log : logs) log->shutdown(); } void SystemLogs::handleCrash() { + auto logs = getAllLogs(); for (auto & log : logs) log->handleCrash(); } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 0ac468b15ec..093be203282 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -5,6 +5,32 @@ #include #include +#include + +#define LIST_OF_ALL_SYSTEM_LOGS(M) \ + M(QueryLog, query_log, "Used to log queries.") \ + M(QueryThreadLog, query_thread_log, "Used to log query threads.") \ + M(PartLog, part_log, "Used to log operations with parts.") \ + M(TraceLog, trace_log, "Used to log traces from query profiler.") \ + M(CrashLog, crash_log, "Used to log server crashes.") \ + M(TextLog, text_log, "Used to log all text messages.") \ + M(MetricLog, metric_log, "Used to log all metrics.") \ + M(ErrorLog, error_log, "Used to log errors.") \ + M(FilesystemCacheLog, filesystem_cache_log, "") \ + M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "") \ + M(ObjectStorageQueueLog, s3_queue_log, "") \ + M(ObjectStorageQueueLog, azure_queue_log, "") \ + M(AsynchronousMetricLog, asynchronous_metric_log, "Metrics from system.asynchronous_metrics") \ + M(OpenTelemetrySpanLog, opentelemetry_span_log, "OpenTelemetry trace spans.") \ + M(QueryViewsLog, query_views_log, "Used to log queries of materialized and live views.") \ + M(ZooKeeperLog, zookeeper_log, "Used to log all actions of ZooKeeper client.") \ + M(SessionLog, session_log, "Login, LogOut and Login failure events.") \ + M(TransactionsInfoLog, transactions_info_log, "Events related to transactions.") \ + M(ProcessorsProfileLog, processors_profile_log, "Used to log processors profiling") \ + M(AsynchronousInsertLog, asynchronous_insert_log, "") \ + M(BackupLog, backup_log, "Backup and restore events") \ + M(BlobStorageLog, blob_storage_log, "Log blob storage operations") \ + namespace DB { @@ -34,71 +60,39 @@ namespace DB }; */ -class QueryLog; -class QueryThreadLog; -class PartLog; -class TextLog; -class TraceLog; -class CrashLog; -class ErrorLog; -class MetricLog; -class AsynchronousMetricLog; -class OpenTelemetrySpanLog; -class QueryViewsLog; -class ZooKeeperLog; -class SessionLog; -class TransactionsInfoLog; -class ProcessorsProfileLog; -class FilesystemCacheLog; -class FilesystemReadPrefetchesLog; -class AsynchronousInsertLog; -class BackupLog; -class ObjectStorageQueueLog; -class BlobStorageLog; +/// NOLINTBEGIN(bugprone-macro-parentheses) +#define FORWARD_DECLARATION(log_type, member, descr) \ + class log_type; \ + +LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) +#undef FORWARD_DECLARATION +/// NOLINTEND(bugprone-macro-parentheses) + /// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables -struct SystemLogs +class SystemLogs { +public: + SystemLogs() = default; SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config); - ~SystemLogs(); + SystemLogs(const SystemLogs & other) = default; + void flush(bool should_prepare_tables_anyway); + void flushAndShutdown(); void shutdown(); void handleCrash(); - std::shared_ptr query_log; /// Used to log queries. - std::shared_ptr query_thread_log; /// Used to log query threads. - std::shared_ptr part_log; /// Used to log operations with parts - std::shared_ptr trace_log; /// Used to log traces from query profiler - std::shared_ptr crash_log; /// Used to log server crashes. - std::shared_ptr text_log; /// Used to log all text messages. - std::shared_ptr metric_log; /// Used to log all metrics. - std::shared_ptr error_log; /// Used to log errors. - std::shared_ptr filesystem_cache_log; - std::shared_ptr filesystem_read_prefetches_log; - std::shared_ptr s3_queue_log; - std::shared_ptr azure_queue_log; - /// Metrics from system.asynchronous_metrics. - std::shared_ptr asynchronous_metric_log; - /// OpenTelemetry trace spans. - std::shared_ptr opentelemetry_span_log; - /// Used to log queries of materialized and live views - std::shared_ptr query_views_log; - /// Used to log all actions of ZooKeeper client - std::shared_ptr zookeeper_log; - /// Login, LogOut and Login failure events - std::shared_ptr session_log; - /// Events related to transactions - std::shared_ptr transactions_info_log; - /// Used to log processors profiling - std::shared_ptr processors_profile_log; - std::shared_ptr asynchronous_insert_log; - /// Backup and restore events - std::shared_ptr backup_log; - /// Log blob storage operations - std::shared_ptr blob_storage_log; +/// NOLINTBEGIN(bugprone-macro-parentheses) +#define PUBLIC_MEMBERS(log_type, member, descr) \ + std::shared_ptr member; \ - std::vector logs; + LIST_OF_ALL_SYSTEM_LOGS(PUBLIC_MEMBERS) +#undef PUBLIC_MEMBERS +/// NOLINTEND(bugprone-macro-parentheses) + +private: + std::vector getAllLogs() const; }; struct SystemLogSettings From 633f700df60b878e59fe17d6204faf4e984b1009 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Aug 2024 12:14:58 +0200 Subject: [PATCH 0567/1722] adjust tests --- .../test_system_flush_logs/test.py | 63 +++-- .../test_system_logs_recreate/test.py | 225 +++++++++--------- 2 files changed, 146 insertions(+), 142 deletions(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 0399122406a..44269883d1b 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -12,18 +12,6 @@ node = cluster.add_instance( stay_alive=True, ) -system_logs = [ - # enabled by default - ("system.text_log", 1), - ("system.query_log", 1), - ("system.query_thread_log", 1), - ("system.part_log", 1), - ("system.trace_log", 1), - ("system.metric_log", 1), - ("system.error_log", 1), -] - - @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -33,22 +21,29 @@ def start_cluster(): cluster.shutdown() -@pytest.fixture(scope="function") -def flush_logs(): +def test_system_logs_exists(): + system_logs = [ + # disabled by default + ("system.text_log", 0), + ("system.query_log", 1), + ("system.query_thread_log", 1), + ("system.part_log", 1), + ("system.trace_log", 1), + ("system.metric_log", 1), + ("system.error_log", 1), + ] + node.query("SYSTEM FLUSH LOGS") - - -@pytest.mark.parametrize("table,exists", system_logs) -def test_system_logs(flush_logs, table, exists): - q = "SELECT * FROM {}".format(table) - if exists: - node.query(q) - else: - response = node.query_and_get_error(q) - assert ( - "Table {} does not exist".format(table) in response - or "Unknown table expression identifier '{}'".format(table) in response - ) + for table, exists in system_logs: + q = "SELECT * FROM {}".format(table) + if exists: + node.query(q) + else: + response = node.query_and_get_error(q) + assert ( + "Table {} does not exist".format(table) in response + or "Unknown table expression identifier '{}'".format(table) in response + ) # Logic is tricky, let's check that there is no hang in case of message queue @@ -67,11 +62,14 @@ def test_system_logs_non_empty_queue(): def test_system_suspend(): - node.query("CREATE TABLE t (x DateTime) ENGINE=Memory;") - node.query("INSERT INTO t VALUES (now());") - node.query("SYSTEM SUSPEND FOR 1 SECOND;") - node.query("INSERT INTO t VALUES (now());") - assert "1\n" == node.query("SELECT max(x) - min(x) >= 1 FROM t;") + try: + node.query("CREATE TABLE t (x DateTime) ENGINE=Memory;") + node.query("INSERT INTO t VALUES (now());") + node.query("SYSTEM SUSPEND FOR 1 SECOND;") + node.query("INSERT INTO t VALUES (now());") + assert "1\n" == node.query("SELECT max(x) - min(x) >= 1 FROM t;") + finally: + node.query("DROP TABLE IF EXISTS t;") def test_log_max_size(start_cluster): @@ -95,6 +93,7 @@ def test_log_max_size(start_cluster): ] ) + node.query("SYSTEM FLUSH LOGS") node.query(f"TRUNCATE TABLE IF EXISTS system.query_log") node.restart_clickhouse() diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index d5347ae7dea..c6d5861904c 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -33,124 +33,129 @@ def test_system_logs_recreate(): "error_log", ] - node.query("SYSTEM FLUSH LOGS") - for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + try: + node.query("SYSTEM FLUSH LOGS") + for table in system_logs: + assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 1 ) - == 1 - ) - # NOTE: we use zzz- prefix to make it the last file, - # so that it will be applied last. - for table in system_logs: - node.exec_in_container( - [ - "bash", - "-c", - f"""echo " - - <{table}> - ENGINE = Null - - - - " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml - """, - ] - ) - - node.restart_clickhouse() - node.query("SYSTEM FLUSH LOGS") - for table in system_logs: - assert "ENGINE = MergeTree" not in node.query( - f"SHOW CREATE TABLE system.{table}" - ) - assert "ENGINE = Null" in node.query(f"SHOW CREATE TABLE system.{table}") - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + # NOTE: we use zzz- prefix to make it the last file, + # so that it will be applied last. + for table in system_logs: + node.exec_in_container( + [ + "bash", + "-c", + f"""echo " + + <{table}> + ENGINE = Null + + + + " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml + """, + ] ) - == 2 - ) - # apply only storage_policy for all system tables - for table in system_logs: - node.exec_in_container( - [ - "bash", - "-c", - f"""echo " - - <{table}> - system_tables - - - " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml - """, - ] - ) - node.restart_clickhouse() - node.query("SYSTEM FLUSH LOGS") - import logging - - for table in system_logs: - create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw") - logging.debug( - "With storage policy, SHOW CREATE TABLE system.%s is: %s", - table, - create_table_sql, - ) - assert "ENGINE = MergeTree" in create_table_sql - assert "ENGINE = Null" not in create_table_sql - assert "SETTINGS storage_policy = 'system_tables'" in create_table_sql - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + node.restart_clickhouse() + node.query("SYSTEM FLUSH LOGS") + for table in system_logs: + assert "ENGINE = MergeTree" not in node.query( + f"SHOW CREATE TABLE system.{table}" ) - == 3 - ) - - for table in system_logs: - node.exec_in_container( - ["rm", f"/etc/clickhouse-server/config.d/zzz-override-{table}.xml"] - ) - - node.restart_clickhouse() - node.query("SYSTEM FLUSH LOGS") - for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + assert "ENGINE = Null" in node.query(f"SHOW CREATE TABLE system.{table}") + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 2 ) - == 4 - ) - node.query("SYSTEM FLUSH LOGS") - # Ensure that there was no superfluous RENAME's - # IOW that the table created only when the structure is indeed different. - for table in system_logs: - assert ( - len( - node.query(f"SHOW TABLES FROM system LIKE '{table}%'") - .strip() - .split("\n") + # apply only storage_policy for all system tables + for table in system_logs: + node.exec_in_container( + [ + "bash", + "-c", + f"""echo " + + <{table}> + system_tables + + + " > /etc/clickhouse-server/config.d/zzz-override-{table}.xml + """, + ] ) - == 4 - ) + node.restart_clickhouse() + node.query("SYSTEM FLUSH LOGS") + import logging + + for table in system_logs: + create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw") + logging.debug( + "With storage policy, SHOW CREATE TABLE system.%s is: %s", + table, + create_table_sql, + ) + assert "ENGINE = MergeTree" in create_table_sql + assert "ENGINE = Null" not in create_table_sql + assert "SETTINGS storage_policy = 'system_tables'" in create_table_sql + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 3 + ) + + for table in system_logs: + node.exec_in_container( + ["rm", f"/etc/clickhouse-server/config.d/zzz-override-{table}.xml"] + ) + + node.restart_clickhouse() + node.query("SYSTEM FLUSH LOGS") + for table in system_logs: + assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 4 + ) + + node.query("SYSTEM FLUSH LOGS") + # Ensure that there was no superfluous RENAME's + # IOW that the table created only when the structure is indeed different. + for table in system_logs: + assert ( + len( + node.query(f"SHOW TABLES FROM system LIKE '{table}%'") + .strip() + .split("\n") + ) + == 4 + ) + finally: + for table in system_logs: + for syffix in range(3): + node.query(f"DROP TABLE IF EXISTS system.{table}_{syffix} sync") def test_drop_system_log(): From 08f8d94856841254957e1e746685d32de2074dc6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Aug 2024 12:16:44 +0200 Subject: [PATCH 0568/1722] no flush backup logs at shutdown, flush all logs --- src/Backups/BackupsWorker.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 106aa89082d..0b93ae6d547 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1171,8 +1171,6 @@ void BackupsWorker::waitAll() for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); - backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); - LOG_INFO(log, "Backups and restores finished"); } @@ -1223,8 +1221,6 @@ void BackupsWorker::cancelAll(bool wait_) for (const auto & id : current_operations) wait(id, /* rethrow_exception= */ false); - backup_log->flush(backup_log->getLastLogIndex(), /* should_prepare_tables_anyway */ false); - LOG_INFO(log, "Backups and restores finished or stopped"); } From e2e9ae776d0e0795a565fa7ed6a29671fcda377e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 09:03:29 +0000 Subject: [PATCH 0569/1722] Follow-up to #63898, pt. II --- .../0_stateless/00366_multi_statements.sh | 20 +++--- .../00443_preferred_block_size_bytes.sh | 4 +- ...ess_to_temporary_table_in_readonly_mode.sh | 20 +++--- ...4_performance_introspection_and_logging.sh | 4 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 6 +- ...d_optimize_skip_select_on_unused_shards.sh | 30 ++++----- ...p_select_on_unused_shards_with_prewhere.sh | 26 ++++---- ...ated_minimalistic_part_header_zookeeper.sh | 4 +- .../queries/0_stateless/00837_minmax_index.sh | 2 +- .../queries/0_stateless/00838_unique_index.sh | 2 +- .../0_stateless/00907_set_index_max_rows.sh | 2 +- .../0_stateless/00908_bloom_filter_index.sh | 10 +-- .../queries/0_stateless/00942_mutate_index.sh | 2 +- .../0_stateless/00943_materialize_index.sh | 4 +- .../00944_clear_index_in_partition.sh | 2 +- .../00964_bloom_index_string_functions.sh | 2 +- .../00965_set_index_string_functions.sh | 2 +- .../00974_primary_key_for_lowCardinality.sh | 4 +- tests/queries/0_stateless/00990_hasToken.sh | 2 +- .../01013_sync_replica_timeout_zookeeper.sh | 6 +- ...th_nondeterministic_functions_zookeeper.sh | 4 +- .../01037_polygon_dicts_correctness_all.sh | 6 +- .../01037_polygon_dicts_correctness_fast.sh | 6 +- .../01037_polygon_dicts_simple_functions.sh | 6 +- .../01055_minmax_index_compact_parts.sh | 2 +- .../01077_mutations_index_consistency.sh | 10 +-- .../01187_set_profile_as_setting.sh | 8 +-- ..._block_size_rows_for_materialized_views.sh | 2 +- .../01307_multiple_leaders_zookeeper.sh | 10 +-- .../0_stateless/01415_sticking_mutations.sh | 2 +- tests/queries/0_stateless/01451_dist_logs.sh | 2 +- .../01459_manual_write_to_replicas.sh | 4 +- .../01459_manual_write_to_replicas_quorum.sh | 4 +- ..._write_to_replicas_quorum_detach_attach.sh | 4 +- ...house_server_start_with_embedded_config.sh | 2 +- .../0_stateless/01508_format_regexp_raw.sh | 4 +- .../01509_dictionary_preallocate.sh | 2 +- ...01510_format_regexp_raw_low_cardinality.sh | 4 +- .../0_stateless/01526_initial_query_id.sh | 2 +- .../01599_mutation_query_params.sh | 2 +- .../01600_quota_by_forwarded_ip.sh | 4 +- .../01684_ssd_cache_dictionary_simple_key.sh | 2 +- .../01685_ssd_cache_dictionary_complex_key.sh | 2 +- .../01691_parser_data_type_exponential.sh | 2 +- ...ojections_optimize_aggregation_in_order.sh | 2 +- ...s_partial_optimize_aggregation_in_order.sh | 2 +- .../01753_optimize_aggregation_in_order.sh | 2 +- .../01758_optimize_skip_unused_shards_once.sh | 2 +- ...91_dist_INSERT_block_structure_mismatch.sh | 2 +- .../01814_distributed_push_down_limit.sh | 6 +- .../01853_dictionary_cache_duplicates.sh | 8 +-- .../01872_initial_query_start_time.sh | 2 +- ...75_ssd_cache_dictionary_decimal256_type.sh | 2 +- .../01890_materialized_distributed_join.sh | 2 +- .../01903_ssd_cache_dictionary_array_type.sh | 2 +- ..._cache_dictionary_default_nullable_type.sh | 2 +- ...1927_query_views_log_matview_exceptions.sh | 4 +- .../0_stateless/01947_multiple_pipe_read.sh | 4 +- .../02003_memory_limit_in_client.sh | 32 ++++----- .../02021_create_database_with_comment.sh | 2 +- .../02050_client_profile_events.sh | 8 +-- .../02221_parallel_replicas_bug.sh | 2 +- .../02221_system_zookeeper_unrestricted.sh | 4 +- ...2221_system_zookeeper_unrestricted_like.sh | 4 +- ...parallel_distributed_insert_select_view.sh | 6 +- ...arallel_reading_from_replicas_benchmark.sh | 4 +- .../02232_allow_only_replicated_engine.sh | 6 +- .../0_stateless/02250_ON_CLUSTER_grant.sh | 4 +- tests/queries/0_stateless/02262_column_ttl.sh | 4 +- .../0_stateless/02286_parallel_final.sh | 4 +- ..._distinct_in_order_optimization_explain.sh | 66 +++++++++---------- ..._column_ttl_expired_column_optimization.sh | 2 +- .../0_stateless/02361_fsync_profile_events.sh | 4 +- ...7_extend_protocol_with_query_parameters.sh | 10 +-- ...ting_by_input_stream_properties_explain.sh | 2 +- .../0_stateless/02417_load_marks_async.sh | 2 +- 76 files changed, 229 insertions(+), 229 deletions(-) diff --git a/tests/queries/0_stateless/00366_multi_statements.sh b/tests/queries/0_stateless/00366_multi_statements.sh index 0b2e80fe457..8546e547581 100755 --- a/tests/queries/0_stateless/00366_multi_statements.sh +++ b/tests/queries/0_stateless/00366_multi_statements.sh @@ -14,22 +14,22 @@ $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT -n --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2" -$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2;" -$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT -n --query="DROP TABLE IF EXISTS t_00366; CREATE TABLE t_00366 (x UInt64) ENGINE = TinyLog;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t_00366; CREATE TABLE t_00366 (x UInt64) ENGINE = TinyLog;" $CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES (1),(2),(3);" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" $CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT -n --query="INSERT INTO t_00366 VALUES (1),(2),(3);" -$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT -n --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" -$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t_00366" +$CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES (1),(2),(3);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" +$CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" +$CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1;" @@ -48,4 +48,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+t_00366+VALUES" -d "(7),(8),(9)" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT -n --query="DROP TABLE t_00366;" +$CLICKHOUSE_CLIENT --query="DROP TABLE t_00366;" diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 27b9f5c00c7..0635fbc2a57 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -43,10 +43,10 @@ popd > /dev/null #SCRIPTDIR=`dirname "$SCRIPTPATH"` SCRIPTDIR=$SCRIPTPATH -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED rm "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout diff --git a/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh b/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh index 560b97a1d1b..5550fa69d3d 100755 --- a/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh +++ b/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS test_readonly; CREATE TABLE test_readonly ( ID Int @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT -n --query=" ################ # Try to create temporary table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 1; CREATE TEMPORARY TABLE readonly ( ID Int @@ -26,7 +26,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 1; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -34,7 +34,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 1; DROP TABLE test_readonly; " 2> /dev/null; @@ -46,7 +46,7 @@ CODE=$?; ################ # Try to create temporary table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 2; CREATE TEMPORARY TABLE readonly ( ID Int @@ -58,7 +58,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 2; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -66,7 +66,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 2; DROP TABLE test_readonly; " 2> /dev/null; @@ -78,7 +78,7 @@ CODE=$?; ################ # Try to create temporary table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 0; CREATE TEMPORARY TABLE readonly ( ID Int @@ -90,7 +90,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 0; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -98,7 +98,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 0; DROP TABLE test_readonly; " 2> /dev/null; diff --git a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index 93fd0c4a977..e9a4369a5bf 100755 --- a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -19,13 +19,13 @@ settings="$server_logs --log_queries=1 --log_query_threads=1 --log_profile_event # Test insert logging on each block and checkPacket() method -$CLICKHOUSE_CLIENT $settings -n -q " +$CLICKHOUSE_CLIENT $settings -q " DROP TABLE IF EXISTS null_00634; CREATE TABLE null_00634 (i UInt8) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();" head -c 1000 /dev/zero | $CLICKHOUSE_CLIENT $settings --max_insert_block_size=10 --min_insert_block_size_rows=1 --min_insert_block_size_bytes=1 -q "INSERT INTO null_00634 FORMAT RowBinary" -$CLICKHOUSE_CLIENT $settings -n -q " +$CLICKHOUSE_CLIENT $settings -q " SELECT count() FROM null_00634; DROP TABLE null_00634;" diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index 96d5764780f..d69e14bdbb9 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS fetches_r1 SYNC; DROP TABLE IF EXISTS fetches_r2 SYNC" @@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate SETTINGS prefer_fetch_merged_part_time_threshold=0, \ prefer_fetch_merged_part_size_threshold=0" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); @@ -51,6 +51,6 @@ ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA fetches_r2" ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutated part ***'" ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE fetches_r1 SYNC; DROP TABLE fetches_r2 SYNC" diff --git a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh index 09f20284402..989096a26d6 100755 --- a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh +++ b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh @@ -25,83 +25,83 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM distributed WHERE a = 0 AND b | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complext expressions for constant folding - all should pass. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 1 AND a = 0 AND b = 0; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a IN (0, 1) AND b IN (0, 1); " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR a = 1 AND b = 1; " # TODO: should pass one day. -#${CLICKHOUSE_CLIENT} -n --query=" +#${CLICKHOUSE_CLIENT} --query=" # SET optimize_skip_unused_shards = 1; # SELECT count(*) FROM distributed WHERE a = 0 AND b >= 0 AND b <= 1; #" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND c = 0; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND c != 10; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND (a+b)*b != 12; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE (a = 0 OR a = 1) AND (b = 0 OR b = 1); " # These ones should fail. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b <= 1; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND c = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR c = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh index 035907bddd7..b3dff2ea69a 100755 --- a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh +++ b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh @@ -30,73 +30,73 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM distributed_00754 PREWHERE a | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complex expressions for constant folding - all should pass. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 1 AND a = 0 WHERE b = 0; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 1 WHERE b = 1 AND length(c) = 5; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a IN (0, 1) AND b IN (0, 1) WHERE c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a IN (0, 1) WHERE b IN (0, 1) AND c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR a = 1 AND b = 1 WHERE c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE (a = 0 OR a = 1) WHERE (b = 0 OR b = 1); " # These should fail. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b <= 1; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 WHERE c LIKE '%l%'; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR c LIKE '%l%'; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 12d889a7137..8f7a1a9ae98 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; @@ -62,7 +62,7 @@ do [[ $count1 == 1 && $count2 == 1 ]] && break done -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; diff --git a/tests/queries/0_stateless/00837_minmax_index.sh b/tests/queries/0_stateless/00837_minmax_index.sh index e4de0b9ebfc..ff487f50ee0 100755 --- a/tests/queries/0_stateless/00837_minmax_index.sh +++ b/tests/queries/0_stateless/00837_minmax_index.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00838_unique_index.sh b/tests/queries/0_stateless/00838_unique_index.sh index b267b6a8eb3..a3aba4f26b6 100755 --- a/tests/queries/0_stateless/00838_unique_index.sh +++ b/tests/queries/0_stateless/00838_unique_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE set_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00907_set_index_max_rows.sh b/tests/queries/0_stateless/00907_set_index_max_rows.sh index 3707aaf2ca6..bdd0f36346f 100755 --- a/tests/queries/0_stateless/00907_set_index_max_rows.sh +++ b/tests/queries/0_stateless/00907_set_index_max_rows.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE set_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00908_bloom_filter_index.sh b/tests/queries/0_stateless/00908_bloom_filter_index.sh index 25a6567b894..3bd169dd6df 100755 --- a/tests/queries/0_stateless/00908_bloom_filter_index.sh +++ b/tests/queries/0_stateless/00908_bloom_filter_index.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx3;" # NGRAM BF -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx ( k UInt64, @@ -22,7 +22,7 @@ CREATE TABLE bloom_filter_idx ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx2 ( k UInt64, @@ -109,7 +109,7 @@ $CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT count() FROM bloom # TOKEN BF -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx3 ( k UInt64, @@ -147,7 +147,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx2" $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx3" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx_na;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx_na ( na Array(Array(String)), @@ -156,7 +156,7 @@ CREATE TABLE bloom_filter_idx_na ORDER BY na" 2>&1 | grep -c 'DB::Exception: Unexpected type Array(Array(String)) of bloom filter index' # NGRAM BF with IPv6 -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_ipv6_idx ( foo IPv6, diff --git a/tests/queries/0_stateless/00942_mutate_index.sh b/tests/queries/0_stateless/00942_mutate_index.sh index 6ebb30c25b9..e1e23639e85 100755 --- a/tests/queries/0_stateless/00942_mutate_index.sh +++ b/tests/queries/0_stateless/00942_mutate_index.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00943_materialize_index.sh b/tests/queries/0_stateless/00943_materialize_index.sh index 6ff7d34a9d7..e4a585fce97 100755 --- a/tests/queries/0_stateless/00943_materialize_index.sh +++ b/tests/queries/0_stateless/00943_materialize_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, @@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES $CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 SETTINGS mutations_sync = 2;" $CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1 SETTINGS mutations_sync = 2;" diff --git a/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 4655077960f..a12536da239 100755 --- a/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh index e2ec7fd42e4..9e410f09b13 100755 --- a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh +++ b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx;" # NGRAM BF -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx ( k UInt64, diff --git a/tests/queries/0_stateless/00965_set_index_string_functions.sh b/tests/queries/0_stateless/00965_set_index_string_functions.sh index 8892fb11752..0f29c3dd2f2 100755 --- a/tests/queries/0_stateless/00965_set_index_string_functions.sh +++ b/tests/queries/0_stateless/00965_set_index_string_functions.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE set_idx ( k UInt64, diff --git a/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh b/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh index 389d433c7e2..ba260042f47 100755 --- a/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh +++ b/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS lowString;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS string;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" create table lowString ( a LowCardinality(String), @@ -18,7 +18,7 @@ ENGINE = MergeTree() PARTITION BY toYYYYMM(b) ORDER BY (a)" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" create table string ( a String, diff --git a/tests/queries/0_stateless/00990_hasToken.sh b/tests/queries/0_stateless/00990_hasToken.sh index 6a1d4ff5ccf..d79472aa5a5 100755 --- a/tests/queries/0_stateless/00990_hasToken.sh +++ b/tests/queries/0_stateless/00990_hasToken.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -nm +python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -m diff --git a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh index 55bbfb3ff11..54f1bbe29dc 100755 --- a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh +++ b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) R1=table_1013_1 R2=table_1013_2 -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS $R1; DROP TABLE IF EXISTS $R2; @@ -19,13 +19,13 @@ ${CLICKHOUSE_CLIENT} -n -q " INSERT INTO $R1 VALUES (1) " -timeout 10s ${CLICKHOUSE_CLIENT} -n -q " +timeout 10s ${CLICKHOUSE_CLIENT} -q " SET receive_timeout=1; SYSTEM SYNC REPLICA $R2 " 2>&1 | grep -F -q "Code: 159. DB::Exception" && echo 'OK' || echo 'Failed!' # By dropping tables all related SYNC REPLICA queries would be terminated as well -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS $R2; DROP TABLE IF EXISTS $R1; " diff --git a/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh b/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh index 4f35b69da0b..053fd9d9d49 100755 --- a/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh +++ b/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh @@ -9,7 +9,7 @@ R1=table_1017_1 R2=table_1017_2 T1=table_1017_merge -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP DICTIONARY IF EXISTS dict1; DROP TABLE IF EXISTS $R1; DROP TABLE IF EXISTS $R2; @@ -68,7 +68,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE dictHas('${CLICKHOUSE ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE dictHas('${CLICKHOUSE_DATABASE}.dict1', toUInt64(x))" --allow_nondeterministic_mutations=1 2>&1 \ && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP DICTIONARY IF EXISTS dict1; DROP TABLE IF EXISTS $R2; DROP TABLE IF EXISTS $R1; diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh index 9a26f78a8ee..5c67fe08fbf 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh @@ -14,7 +14,7 @@ declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${DATA_DIR}" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS points; CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_si rm "${DATA_DIR}"/01037_point_data -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array @@ -43,7 +43,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index 47f7a5c1c4f..591978d1129 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -14,7 +14,7 @@ declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL") tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${DATA_DIR}" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " @@ -22,7 +22,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for rm "${DATA_DIR}"/01037_point_data -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array @@ -42,7 +42,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array diff --git a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh index d1ee3f283bc..ac033ff4eb8 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) TMP_DIR=${CLICKHOUSE_TMP}/tmp mkdir -p $TMP_DIR -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory; @@ -53,7 +53,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array ( @@ -106,7 +106,7 @@ do diff -q "${CURDIR}/01037_polygon_dicts_simple_functions.ans" "$outputFile" done -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP DICTIONARY dict_array; DROP DICTIONARY dict_tuple; DROP TABLE polygons_array; diff --git a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh index 0b14ef8f6fa..29ce4da02ed 100755 --- a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh +++ b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/01077_mutations_index_consistency.sh b/tests/queries/0_stateless/01077_mutations_index_consistency.sh index ffbe3692b64..f103692de56 100755 --- a/tests/queries/0_stateless/01077_mutations_index_consistency.sh +++ b/tests/queries/0_stateless/01077_mutations_index_consistency.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS movement" -$CLICKHOUSE_CLIENT -n --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';" +$CLICKHOUSE_CLIENT --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query "insert into movement select toDateTime('2020-01-22 00:00:00', 'Asia/Istanbul') + number%(23*3600) from numbers(1000000);" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE movement FINAL" -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour @@ -26,7 +26,7 @@ ORDER BY Hour DESC $CLICKHOUSE_CLIENT --query "alter table movement delete where date >= toDateTime('2020-01-22T16:00:00', 'Asia/Istanbul') and date < toDateTime('2020-01-22T17:00:00', 'Asia/Istanbul') SETTINGS mutations_sync = 2" -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour @@ -37,7 +37,7 @@ ORDER BY Hour DESC " | grep "16:00:00" | wc -l -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour @@ -48,7 +48,7 @@ ORDER BY Hour DESC " | grep "22:00:00" | cut -f1 -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index 42f596c45d6..f6c6fd0be34 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -7,11 +7,11 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" -$CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" -$CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -m -q "select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') -$CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&profile=default&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh index 5f82731c54e..1ec53399958 100755 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh @@ -10,7 +10,7 @@ set -o pipefail # shellcheck disable=SC2120 function execute() { - ${CLICKHOUSE_CLIENT} -n "$@" + ${CLICKHOUSE_CLIENT} "$@" } # diff --git a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh index db986e74b6b..02aa0f76be5 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh @@ -12,8 +12,8 @@ DATA_SIZE=200 SEQ=$(seq 0 $(($NUM_REPLICAS - 1))) -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE IF EXISTS r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done function thread() { @@ -30,6 +30,6 @@ done wait -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "SYSTEM SYNC REPLICA r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "SELECT count(), sum(x) FROM r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "SELECT count(), sum(x) FROM r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "DROP TABLE r$REPLICA"; done diff --git a/tests/queries/0_stateless/01415_sticking_mutations.sh b/tests/queries/0_stateless/01415_sticking_mutations.sh index b7c8768a65d..97467c3ce9d 100755 --- a/tests/queries/0_stateless/01415_sticking_mutations.sh +++ b/tests/queries/0_stateless/01415_sticking_mutations.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS sticking_mutations" function check_sticky_mutations() { - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations ( + $CLICKHOUSE_CLIENT --query "CREATE TABLE sticking_mutations ( date Date, key UInt64, value1 String, diff --git a/tests/queries/0_stateless/01451_dist_logs.sh b/tests/queries/0_stateless/01451_dist_logs.sh index 23dee7a827d..e281e232bb5 100755 --- a/tests/queries/0_stateless/01451_dist_logs.sh +++ b/tests/queries/0_stateless/01451_dist_logs.sh @@ -10,4 +10,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # triggered not for the first query for _ in {1..20}; do echo "select * from remote('127.{2,3}', system.numbers) where number = 10 limit 1;" -done | ${CLICKHOUSE_CLIENT} -n 2>/dev/null +done | ${CLICKHOUSE_CLIENT} 2>/dev/null diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 56620d848a3..cc574557438 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -31,7 +31,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 91a73471557..24ea3ba3835 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -16,7 +16,7 @@ unset CLICKHOUSE_WRITE_COVERAGE NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -39,7 +39,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh index 1f76a2efc6b..a2ef0d52328 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh @@ -11,7 +11,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -36,7 +36,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh index 29593ea4fb5..6954fef7314 100755 --- a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh +++ b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh @@ -34,7 +34,7 @@ done # Check access rights -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP DATABASE IF EXISTS test; CREATE DATABASE test; USE test; diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.sh b/tests/queries/0_stateless/01508_format_regexp_raw.sh index 8cf1bd73566..52613c28b2f 100755 --- a/tests/queries/0_stateless/01508_format_regexp_raw.sh +++ b/tests/queries/0_stateless/01508_format_regexp_raw.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a String, b String) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a String, b String) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t; DROP TABLE t; " diff --git a/tests/queries/0_stateless/01509_dictionary_preallocate.sh b/tests/queries/0_stateless/01509_dictionary_preallocate.sh index 2a22a307a08..0459f69b0ad 100755 --- a/tests/queries/0_stateless/01509_dictionary_preallocate.sh +++ b/tests/queries/0_stateless/01509_dictionary_preallocate.sh @@ -15,7 +15,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # PREALLOCATE attribute (and also for the history/greppability, that it was # such). -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS data_01509; DROP DICTIONARY IF EXISTS dict_01509; CREATE TABLE data_01509 diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh index 594caca7d04..dc178d081bf 100755 --- a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t; DROP TABLE t; " diff --git a/tests/queries/0_stateless/01526_initial_query_id.sh b/tests/queries/0_stateless/01526_initial_query_id.sh index e77764ee34e..8ba27a04d60 100755 --- a/tests/queries/0_stateless/01526_initial_query_id.sh +++ b/tests/queries/0_stateless/01526_initial_query_id.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CURL} \ --get \ --data-urlencode "query=select 1 format Null" -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " system flush logs; select interface, initial_query_id = query_id from system.query_log diff --git a/tests/queries/0_stateless/01599_mutation_query_params.sh b/tests/queries/0_stateless/01599_mutation_query_params.sh index 52b0131a9c2..5b604c96028 100755 --- a/tests/queries/0_stateless/01599_mutation_query_params.sh +++ b/tests/queries/0_stateless/01599_mutation_query_params.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS test; CREATE TABLE test diff --git a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh index 1d768c8b027..834eba8f25c 100755 --- a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh +++ b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " CREATE USER quoted_by_ip_${CLICKHOUSE_DATABASE}; CREATE USER quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; @@ -57,7 +57,7 @@ ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 5.6.7.8, 1.2.3.4' -sS "${CLICKHOUSE_URL} ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4, 5.6.7.8' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP QUOTA IF EXISTS quota_by_ip_${CLICKHOUSE_DATABASE}; DROP QUOTA IF EXISTS quota_by_forwarded_ip; diff --git a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh index 0e5c2862066..6a7eb975c87 100755 --- a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh +++ b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary; CREATE DATABASE 01684_database_for_cache_dictionary; diff --git a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh index 55061b9a643..c2d222a86ea 100755 --- a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh +++ b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE complex_key_simple_attributes_source_table ( id UInt64, diff --git a/tests/queries/0_stateless/01691_parser_data_type_exponential.sh b/tests/queries/0_stateless/01691_parser_data_type_exponential.sh index f8004f9350d..5d115e09a79 100755 --- a/tests/queries/0_stateless/01691_parser_data_type_exponential.sh +++ b/tests/queries/0_stateless/01691_parser_data_type_exponential.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Check that DataType parser does not have exponential complexity in the case found by fuzzer. -for _ in {1..10}; do ${CLICKHOUSE_CLIENT} -n --query "SELECT CAST(1 AS A2222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222220000000000000000000000000000000000000000000000000000000000000000000000000000002260637443813394204 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggre222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 22222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 2222222222222eFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222, 222222222222222ggregateFuncpion(groupBitmap222222222222222222222222222222222222222222222222222222222222222222222222000000000000000000001788596394540167623 222222222222222222ggregateFu22222222222222222222222222 222222222, UInt33)); -- { clientError 62 }"; done +for _ in {1..10}; do ${CLICKHOUSE_CLIENT} --query "SELECT CAST(1 AS A2222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222220000000000000000000000000000000000000000000000000000000000000000000000000000002260637443813394204 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggre222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 22222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 2222222222222eFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222, 222222222222222ggregateFuncpion(groupBitmap222222222222222222222222222222222222222222222222222222222222222222222222000000000000000000001788596394540167623 222222222222222222ggregateFu22222222222222222222222222 222222222, UInt33)); -- { clientError 62 }"; done diff --git a/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh index a166837e01a..f38e53f898a 100755 --- a/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS in_order_agg_01710; CREATE TABLE in_order_agg_01710 diff --git a/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh index ee73974e8a4..01537524730 100755 --- a/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS in_order_agg_partial_01710; CREATE TABLE in_order_agg_partial_01710 diff --git a/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh index 2a7345f4865..f9681ebe4f5 100755 --- a/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -nm -q " +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -m -q " drop table if exists data_01753; create table data_01753 (key Int) engine=MergeTree() order by key as select * from numbers(8); select * from data_01753 group by key settings max_block_size=1; diff --git a/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh b/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh index b963f3a618f..3c9e12f780b 100755 --- a/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh +++ b/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --optimize_skip_unused_shards=1 -nm -q " +$CLICKHOUSE_CLIENT --optimize_skip_unused_shards=1 -m -q " create table dist_01758 as system.one engine=Distributed(test_cluster_two_shards, system, one, dummy); select * from dist_01758 where dummy = 0 format Null; " |& grep -o "StorageDistributed (dist_01758).*" diff --git a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh index 9c51b82282c..ee46f8194b9 100755 --- a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh +++ b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 -nm -q " +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 -m -q " DROP TABLE IF EXISTS tmp_01683; DROP TABLE IF EXISTS dist_01683; diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh index 4b75102e9cf..f3e8ceffff6 100755 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -13,7 +13,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function setup() { - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " drop table if exists data_01814; drop table if exists dist_01814; @@ -24,7 +24,7 @@ function setup() function cleanup() { - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " drop table data_01814; drop table dist_01814; " @@ -67,7 +67,7 @@ function test_distributed_push_down_limit_with_query_log() $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from $table group by key limit $offset, 10" - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " system flush logs; select read_rows from system.query_log where diff --git a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh index 218320772c9..2b122debff5 100755 --- a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh +++ b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_test_once() { - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS simple_key_source_table_01863; CREATE TABLE simple_key_source_table_01863 ( @@ -29,13 +29,13 @@ function run_test_once() LIFETIME(MIN 0 MAX 1000); " - prev=$($CLICKHOUSE_CLIENT -nm -q "SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1") - curr=$($CLICKHOUSE_CLIENT -nm -q " + prev=$($CLICKHOUSE_CLIENT -m -q "SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1") + curr=$($CLICKHOUSE_CLIENT -m -q " SELECT toUInt64(1) as key, dictGet('simple_key_cache_dictionary_01863', 'value', key) FORMAT Null; SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1 ") - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " DROP DICTIONARY simple_key_cache_dictionary_01863; DROP TABLE simple_key_source_table_01863; " diff --git a/tests/queries/0_stateless/01872_initial_query_start_time.sh b/tests/queries/0_stateless/01872_initial_query_start_time.sh index 6a935602ea4..ff3e1954c75 100755 --- a/tests/queries/0_stateless/01872_initial_query_start_time.sh +++ b/tests/queries/0_stateless/01872_initial_query_start_time.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} -q "create table m (dummy UInt8) ENGINE = Distributed('test query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") ${CLICKHOUSE_CLIENT} -q "select * from m format Null" "--query_id=$query_id" -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " system flush logs; select anyIf(initial_query_start_time, is_initial_query) = anyIf(initial_query_start_time, not is_initial_query), diff --git a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh index 8336229a643..1294ba53e82 100755 --- a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh +++ b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET allow_experimental_bigint_types = 1; DROP TABLE IF EXISTS dictionary_decimal_source_table; diff --git a/tests/queries/0_stateless/01890_materialized_distributed_join.sh b/tests/queries/0_stateless/01890_materialized_distributed_join.sh index 88f7dcf9a69..5c04ee8b214 100755 --- a/tests/queries/0_stateless/01890_materialized_distributed_join.sh +++ b/tests/queries/0_stateless/01890_materialized_distributed_join.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists test_distributed; drop table if exists test_source; drop table if exists test_shard; diff --git a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh index 853445daf3f..a44106414ea 100755 --- a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh +++ b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS dictionary_array_source_table; CREATE TABLE dictionary_array_source_table ( diff --git a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh index 0b555cf82c2..a5c65ca87a7 100755 --- a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh +++ b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS dictionary_nullable_source_table; CREATE TABLE dictionary_nullable_source_table ( diff --git a/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh b/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh index 47d5e733480..608107c76d6 100755 --- a/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh +++ b/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - ${CLICKHOUSE_CLIENT} -n -q " + ${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS matview_exception_a_to_c; DROP TABLE IF EXISTS matview_exception_a_to_b; DROP TABLE IF EXISTS table_exception_c; @@ -17,7 +17,7 @@ function cleanup() function setup() { - ${CLICKHOUSE_CLIENT} -n -q " + ${CLICKHOUSE_CLIENT} -q " CREATE TABLE table_exception_a (a String, b Int64) ENGINE = MergeTree ORDER BY b; CREATE TABLE table_exception_b (a Float64, b Int64) ENGINE = MergeTree ORDER BY tuple(); CREATE TABLE table_exception_c (a Float64) ENGINE = MergeTree ORDER BY a; diff --git a/tests/queries/0_stateless/01947_multiple_pipe_read.sh b/tests/queries/0_stateless/01947_multiple_pipe_read.sh index 06a18a55e6e..51709eb574e 100755 --- a/tests/queries/0_stateless/01947_multiple_pipe_read.sh +++ b/tests/queries/0_stateless/01947_multiple_pipe_read.sh @@ -11,7 +11,7 @@ cat "$SAMPLE_FILE" echo '******************' echo 'Read twice from a regular file' -${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' --file "$SAMPLE_FILE" +${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' --file "$SAMPLE_FILE" echo '---' ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' --file "$SAMPLE_FILE" echo '---' @@ -19,7 +19,7 @@ ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table UNI echo '******************' echo 'Read twice from file descriptor that corresponds to a regular file' -${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' < "$SAMPLE_FILE" +${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' < "$SAMPLE_FILE" echo '---' ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' < "$SAMPLE_FILE" echo '---' diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 96028f4847a..15cacbff8c5 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -4,21 +4,21 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2P' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2.1p' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10E' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10.2e' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1.1T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1m' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='14g' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11t' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2P' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2.1p' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10E' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10.2e' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1.1T' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1m' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='14g' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11t' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.sh b/tests/queries/0_stateless/02021_create_database_with_comment.sh index f77397dc482..d87b0794c91 100755 --- a/tests/queries/0_stateless/02021_create_database_with_comment.sh +++ b/tests/queries/0_stateless/02021_create_database_with_comment.sh @@ -20,7 +20,7 @@ function test_db_comments() local ENGINE_NAME="$1" echo "engine : ${ENGINE_NAME}" - $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm <& /dev/null +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' >& /dev/null echo $? echo 'regression test for overlap profile events snapshots between queries' -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'regression test for overlap profile events snapshots between queries (clickhouse-local)' -$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'print everything' profile_events="$( @@ -35,5 +35,5 @@ profile_events="$( test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" echo 'check that ProfileEvents is new for each query' -sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') +sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') test "$sleep_function_calls" -eq 1 && echo OK || echo "FAIL ($sleep_function_calls)" diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index 3c44a2a7ba7..a382b3859f3 100755 --- a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh +++ b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -m < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh index db94c59d2de..e23a272a4e8 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2" -${CLICKHOUSE_CLIENT} -n -q" +${CLICKHOUSE_CLIENT} -q" CREATE TABLE sample_table ( key UInt64 ) @@ -16,7 +16,7 @@ ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_ ORDER BY tuple(); " -${CLICKHOUSE_CLIENT} -n -q" +${CLICKHOUSE_CLIENT} -q" CREATE TABLE sample_table_2 ( key UInt64 ) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh index c62ec14b340..6381d811d5d 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" -${CLICKHOUSE_CLIENT} -n --query="CREATE TABLE sample_table ( +${CLICKHOUSE_CLIENT} --query="CREATE TABLE sample_table ( key UInt64 ) ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like', '1') @@ -16,7 +16,7 @@ ORDER BY tuple(); DROP TABLE IF EXISTS sample_table SYNC;" -${CLICKHOUSE_CLIENT} -n --query "CREATE TABLE sample_table_2 ( +${CLICKHOUSE_CLIENT} --query "CREATE TABLE sample_table_2 ( key UInt64 ) ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like_2', '1') diff --git a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh index 376a49fd820..63111cc32e4 100755 --- a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh +++ b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists dst_02225; drop table if exists src_02225; create table dst_02225 (key Int) engine=Memory(); @@ -14,7 +14,7 @@ create table src_02225 (key Int) engine=Memory(); insert into src_02225 values (1); " -$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -nm -q " +$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -m -q " truncate table dst_02225; insert into function remote('127.{1,2}', currentDatabase(), dst_02225, key) select * from remote('127.{1,2}', view(select * from {database:Identifier}.src_02225), key) @@ -29,7 +29,7 @@ settings parallel_distributed_insert_select=2, max_distributed_depth=1; select * from dst_02225; " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table src_02225; drop table dst_02225; " diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index bc90f4b2c11..177b373641f 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data_02226; create table data_02226 (key Int) engine=MergeTree() order by key as select * from numbers(1); @@ -24,7 +24,7 @@ opts=( $CLICKHOUSE_BENCHMARK --query "select * from remote('127.1', $CLICKHOUSE_DATABASE, data_02226)" "${opts[@]}" >& /dev/null ret=$? -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table data_02226; " diff --git a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh index d1a3825d286..e47a3033681 100755 --- a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh +++ b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh @@ -12,9 +12,9 @@ ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO us ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON Memory, TABLE ENGINE ON MergeTree, TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_memory (x UInt32) engine = Memory;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" diff --git a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh index 66417e9694a..09f9c0c8a98 100755 --- a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh +++ b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - $CLICKHOUSE_CLIENT -nmq " + $CLICKHOUSE_CLIENT -mq " DROP USER IF EXISTS with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; DROP USER IF EXISTS without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; DROP DATABASE IF EXISTS db_with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; @@ -15,7 +15,7 @@ function cleanup() cleanup trap cleanup EXIT -$CLICKHOUSE_CLIENT -nmq " +$CLICKHOUSE_CLIENT -mq " CREATE USER with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; CREATE USER without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; diff --git a/tests/queries/0_stateless/02262_column_ttl.sh b/tests/queries/0_stateless/02262_column_ttl.sh index b5e29c9b2a1..c620d3b6d9c 100755 --- a/tests/queries/0_stateless/02262_column_ttl.sh +++ b/tests/queries/0_stateless/02262_column_ttl.sh @@ -14,7 +14,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # note, that this should be written in .sh since we need $CLICKHOUSE_DATABASE # not 'default' to catch text_log -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists ttl_02262; drop table if exists this_text_log; @@ -31,7 +31,7 @@ $CLICKHOUSE_CLIENT -nm -q " ttl_02262_uuid=$($CLICKHOUSE_CLIENT -q "select uuid from system.tables where database = '$CLICKHOUSE_DATABASE' and name = 'ttl_02262'") -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " -- OPTIMIZE TABLE x FINAL will be done in background -- attach to it's log, via table UUID in query_id (see merger/mutator code). create materialized view this_text_log engine=Memory() as diff --git a/tests/queries/0_stateless/02286_parallel_final.sh b/tests/queries/0_stateless/02286_parallel_final.sh index 0ac510208f3..47dfad42e11 100755 --- a/tests/queries/0_stateless/02286_parallel_final.sh +++ b/tests/queries/0_stateless/02286_parallel_final.sh @@ -9,7 +9,7 @@ echo "Test intersecting ranges" test_random_values() { layers=$1 - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " drop table if exists tbl_8parts_${layers}granules_rnd; create table tbl_8parts_${layers}granules_rnd (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 % 8); insert into tbl_8parts_${layers}granules_rnd select number, 1 from numbers_mt($((layers * 8 * 8192))); @@ -29,7 +29,7 @@ echo "Test non intersecting ranges" test_sequential_values() { layers=$1 - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " drop table if exists tbl_8parts_${layers}granules_seq; create table tbl_8parts_${layers}granules_seq (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 / $((layers * 8192)))::UInt64; insert into tbl_8parts_${layers}granules_seq select number, 1 from numbers_mt($((layers * 8 * 8192))); diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index bd7e6be3987..953485c3a1f 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -23,99 +23,99 @@ $CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % num $CLICKHOUSE_CLIENT -q "select '-- disable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- enable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct with all primary key columns -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by the same columns -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by columns are prefix of distinct columns -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column _not_ in distinct -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column in distinct -> final distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column _not_ in distinct -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by _const_ column in distinct -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT echo "-- Check reading in order for distinct" echo "-- disabled, distinct columns match sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -q "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "-- enabled, distinct columns match sorting key" # read_in_order_two_level_merge_threshold is set here to avoid repeating MergeTreeInOrder in output -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns DON't form prefix of sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "-- enabled, distinct columns contains constant columns, non-const columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns contains constant columns, non-const columns match prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, b, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, b, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, only part of distinct columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "=== disable new analyzer ===" DISABLE_ANALYZER="set enable_analyzer=0" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES echo "=== enable new analyzer ===" ENABLE_ANALYZER="set enable_analyzer=1" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" diff --git a/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh b/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh index 96f80d65878..490f8361682 100755 --- a/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh +++ b/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) data_path="$CLICKHOUSE_TMP/local" -$CLICKHOUSE_LOCAL --path "$data_path" -nm -q " +$CLICKHOUSE_LOCAL --path "$data_path" -m -q " create table ttl_02335 ( date Date, key Int, diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 98c9cf9b7b4..73bf3fa120a 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data_fsync_pe; create table data_fsync_pe (key Int) engine=MergeTree() @@ -27,7 +27,7 @@ for i in {1..100}; do $CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data_fsync_pe values (1)" read -r FileSync FileOpen DirectorySync FileSyncElapsedMicroseconds DirectorySyncElapsedMicroseconds <<<"$( - $CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + $CLICKHOUSE_CLIENT -m --param_query_id "$query_id" -q " system flush logs; select diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh index 71e3b6961f8..46396d38747 100755 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT \ table_name="t_02377_extend_protocol_with_query_parameters_$RANDOM$RANDOM" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " create table $table_name( id Int64, arr Array(UInt8), @@ -57,17 +57,17 @@ $CLICKHOUSE_CLIENT \ # it is possible to set parameter for the current session -$CLICKHOUSE_CLIENT -n -q "set param_n = 42; select {n: UInt8}" +$CLICKHOUSE_CLIENT -q "set param_n = 42; select {n: UInt8}" # and it will not be visible to other sessions -$CLICKHOUSE_CLIENT -n -q "select {n: UInt8} -- { serverError 456 }" +$CLICKHOUSE_CLIENT -q "select {n: UInt8} -- { serverError 456 }" # the same parameter could be set multiple times within one session (new value overrides the previous one) -$CLICKHOUSE_CLIENT -n -q "set param_n = 12; set param_n = 13; select {n: UInt8}" +$CLICKHOUSE_CLIENT -q "set param_n = 12; set param_n = 13; select {n: UInt8}" # multiple different parameters could be defined within each session -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " set param_a = 13, param_b = 'str'; set param_c = '2022-08-04 18:30:53'; set param_d = '{\'10\': [11, 12], \'13\': [14, 15]}'; diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index 4b9793da5bb..974f10e2f24 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -15,7 +15,7 @@ FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES" function explain_sorting { echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTING + $CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -q "$1" | eval $FIND_SORTING } function explain_sortmode { diff --git a/tests/queries/0_stateless/02417_load_marks_async.sh b/tests/queries/0_stateless/02417_load_marks_async.sh index 950656e7ab6..bcede9e4f5e 100755 --- a/tests/queries/0_stateless/02417_load_marks_async.sh +++ b/tests/queries/0_stateless/02417_load_marks_async.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test;" -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " CREATE TABLE test ( n0 UInt64, From b416764585e7e03382f66c99636b4bf0b51bd79f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 13:02:38 +0200 Subject: [PATCH 0570/1722] put description of system log table in one place --- src/Interpreters/SystemLog.cpp | 35 ++++------------------ src/Interpreters/SystemLog.h | 54 ++++++++++++++++------------------ 2 files changed, 30 insertions(+), 59 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 9b58da3f545..d4403b72583 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -284,34 +284,11 @@ ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context) SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config) { - query_log = createSystemLog(global_context, "system", "query_log", config, "query_log", "Contains information about executed queries, for example, start time, duration of processing, error messages."); - query_thread_log = createSystemLog(global_context, "system", "query_thread_log", config, "query_thread_log", "Contains information about threads that execute queries, for example, thread name, thread start time, duration of query processing."); - part_log = createSystemLog(global_context, "system", "part_log", config, "part_log", "This table contains information about events that occurred with data parts in the MergeTree family tables, such as adding or merging data."); - trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log", "Contains stack traces collected by the sampling query profiler."); - crash_log = createSystemLog(global_context, "system", "crash_log", config, "crash_log", "Contains information about stack traces for fatal errors. The table does not exist in the database by default, it is created only when fatal errors occur."); - text_log = createSystemLog(global_context, "system", "text_log", config, "text_log", "Contains logging entries which are normally written to a log file or to stdout."); - metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log", "Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk."); - error_log = createSystemLog(global_context, "system", "error_log", config, "error_log", "Contains history of error values from table system.errors, periodically flushed to disk."); - filesystem_cache_log = createSystemLog(global_context, "system", "filesystem_cache_log", config, "filesystem_cache_log", "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem."); - filesystem_read_prefetches_log = createSystemLog( - global_context, "system", "filesystem_read_prefetches_log", config, "filesystem_read_prefetches_log", "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem."); - asynchronous_metric_log = createSystemLog( - global_context, "system", "asynchronous_metric_log", config, - "asynchronous_metric_log", "Contains the historical values for system.asynchronous_metrics, once per time interval (one second by default)."); - opentelemetry_span_log = createSystemLog( - global_context, "system", "opentelemetry_span_log", config, - "opentelemetry_span_log", "Contains information about trace spans for executed queries."); - query_views_log = createSystemLog(global_context, "system", "query_views_log", config, "query_views_log", "Contains information about the dependent views executed when running a query, for example, the view type or the execution time."); - zookeeper_log = createSystemLog(global_context, "system", "zookeeper_log", config, "zookeeper_log", "This table contains information about the parameters of the request to the ZooKeeper server and the response from it."); - session_log = createSystemLog(global_context, "system", "session_log", config, "session_log", "Contains information about all successful and failed login and logout events."); - transactions_info_log = createSystemLog( - global_context, "system", "transactions_info_log", config, "transactions_info_log", "Contains information about all transactions executed on a current server."); - processors_profile_log = createSystemLog(global_context, "system", "processors_profile_log", config, "processors_profile_log", "Contains profiling information on processors level (building blocks for a pipeline for query execution."); - asynchronous_insert_log = createSystemLog(global_context, "system", "asynchronous_insert_log", config, "asynchronous_insert_log", "Contains a history for all asynchronous inserts executed on current server."); - backup_log = createSystemLog(global_context, "system", "backup_log", config, "backup_log", "Contains logging entries with the information about BACKUP and RESTORE operations."); - s3_queue_log = createSystemLog(global_context, "system", "s3queue_log", config, "s3queue_log", "Contains logging entries with the information files processes by S3Queue engine."); - azure_queue_log = createSystemLog(global_context, "system", "azure_queue_log", config, "azure_queue_log", "Contains logging entries with the information files processes by S3Queue engine."); - blob_storage_log = createSystemLog(global_context, "system", "blob_storage_log", config, "blob_storage_log", "Contains logging entries with information about various blob storage operations such as uploads and deletes."); +#define CREATE_PUBLIC_MEMBERS(log_type, member, descr) \ + member = createSystemLog(global_context, "system", #member, config, #member, descr); \ + + LIST_OF_ALL_SYSTEM_LOGS(CREATE_PUBLIC_MEMBERS) +#undef CREATE_PUBLIC_MEMBERS if (session_log) global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); @@ -355,7 +332,6 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf std::vector SystemLogs::getAllLogs() const { -/// NOLINTBEGIN(bugprone-macro-parentheses) #define GET_RAW_POINTERS(log_type, member, descr) \ member.get(), \ @@ -363,7 +339,6 @@ std::vector SystemLogs::getAllLogs() const LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) }; #undef GET_RAW_POINTERS -/// NOLINTEND(bugprone-macro-parentheses) auto last_it = std::remove(result.begin(), result.end(), nullptr); result.erase(last_it, result.end()); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 093be203282..6682829c0c6 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -8,28 +8,28 @@ #include #define LIST_OF_ALL_SYSTEM_LOGS(M) \ - M(QueryLog, query_log, "Used to log queries.") \ - M(QueryThreadLog, query_thread_log, "Used to log query threads.") \ - M(PartLog, part_log, "Used to log operations with parts.") \ - M(TraceLog, trace_log, "Used to log traces from query profiler.") \ - M(CrashLog, crash_log, "Used to log server crashes.") \ - M(TextLog, text_log, "Used to log all text messages.") \ - M(MetricLog, metric_log, "Used to log all metrics.") \ - M(ErrorLog, error_log, "Used to log errors.") \ - M(FilesystemCacheLog, filesystem_cache_log, "") \ - M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "") \ - M(ObjectStorageQueueLog, s3_queue_log, "") \ - M(ObjectStorageQueueLog, azure_queue_log, "") \ - M(AsynchronousMetricLog, asynchronous_metric_log, "Metrics from system.asynchronous_metrics") \ - M(OpenTelemetrySpanLog, opentelemetry_span_log, "OpenTelemetry trace spans.") \ - M(QueryViewsLog, query_views_log, "Used to log queries of materialized and live views.") \ - M(ZooKeeperLog, zookeeper_log, "Used to log all actions of ZooKeeper client.") \ - M(SessionLog, session_log, "Login, LogOut and Login failure events.") \ - M(TransactionsInfoLog, transactions_info_log, "Events related to transactions.") \ - M(ProcessorsProfileLog, processors_profile_log, "Used to log processors profiling") \ - M(AsynchronousInsertLog, asynchronous_insert_log, "") \ - M(BackupLog, backup_log, "Backup and restore events") \ - M(BlobStorageLog, blob_storage_log, "Log blob storage operations") \ + M(QueryLog, query_log, "Contains information about executed queries, for example, start time, duration of processing, error messages.") \ + M(QueryThreadLog, query_thread_log, "Contains information about threads that execute queries, for example, thread name, thread start time, duration of query processing.") \ + M(PartLog, part_log, "This table contains information about events that occurred with data parts in the MergeTree family tables, such as adding or merging data.") \ + M(TraceLog, trace_log, "Contains stack traces collected by the sampling query profiler.") \ + M(CrashLog, crash_log, "Contains information about stack traces for fatal errors. The table does not exist in the database by default, it is created only when fatal errors occur.") \ + M(TextLog, text_log, "Contains logging entries which are normally written to a log file or to stdout.") \ + M(MetricLog, metric_log, "Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.") \ + M(ErrorLog, error_log, "Contains history of error values from table system.errors, periodically flushed to disk.") \ + M(FilesystemCacheLog, filesystem_cache_log, "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem.") \ + M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem.") \ + M(ObjectStorageQueueLog, s3_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ + M(ObjectStorageQueueLog, azure_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ + M(AsynchronousMetricLog, asynchronous_metric_log, "Contains the historical values for system.asynchronous_metrics, once per time interval (one second by default).") \ + M(OpenTelemetrySpanLog, opentelemetry_span_log, "Contains information about trace spans for executed queries.") \ + M(QueryViewsLog, query_views_log, "Contains information about the dependent views executed when running a query, for example, the view type or the execution time.") \ + M(ZooKeeperLog, zookeeper_log, "This table contains information about the parameters of the request to the ZooKeeper server and the response from it.") \ + M(SessionLog, session_log, "Contains information about all successful and failed login and logout events.") \ + M(TransactionsInfoLog, transactions_info_log, "Contains information about all transactions executed on a current server.") \ + M(ProcessorsProfileLog, processors_profile_log, "Contains profiling information on processors level (building blocks for a pipeline for query execution.") \ + M(AsynchronousInsertLog, asynchronous_insert_log, "Contains a history for all asynchronous inserts executed on current server.") \ + M(BackupLog, backup_log, "Contains logging entries with the information about BACKUP and RESTORE operations.") \ + M(BlobStorageLog, blob_storage_log, "Contains logging entries with information about various blob storage operations such as uploads and deletes.") \ namespace DB @@ -60,13 +60,11 @@ namespace DB }; */ -/// NOLINTBEGIN(bugprone-macro-parentheses) #define FORWARD_DECLARATION(log_type, member, descr) \ class log_type; \ LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) #undef FORWARD_DECLARATION -/// NOLINTEND(bugprone-macro-parentheses) /// System logs should be destroyed in destructor of the last Context and before tables, @@ -83,13 +81,11 @@ public: void shutdown(); void handleCrash(); -/// NOLINTBEGIN(bugprone-macro-parentheses) -#define PUBLIC_MEMBERS(log_type, member, descr) \ +#define DECLARE_PUBLIC_MEMBERS(log_type, member, descr) \ std::shared_ptr member; \ - LIST_OF_ALL_SYSTEM_LOGS(PUBLIC_MEMBERS) -#undef PUBLIC_MEMBERS -/// NOLINTEND(bugprone-macro-parentheses) + LIST_OF_ALL_SYSTEM_LOGS(DECLARE_PUBLIC_MEMBERS) +#undef DECLARE_PUBLIC_MEMBERS private: std::vector getAllLogs() const; From b9f564f6f4fa98cc385acd6ce3c8e9f736bc55e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 7 Aug 2024 11:12:11 +0000 Subject: [PATCH 0571/1722] Automatic style fix --- .../test_system_flush_logs/test.py | 1 + .../test_system_logs_recreate/test.py | 20 ++++++++++++++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 44269883d1b..dd48ef055f5 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -12,6 +12,7 @@ node = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index c6d5861904c..8b84734ed02 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -36,8 +36,12 @@ def test_system_logs_recreate(): try: node.query("SYSTEM FLUSH LOGS") for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = MergeTree" in node.query( + f"SHOW CREATE TABLE system.{table}" + ) + assert "ENGINE = Null" not in node.query( + f"SHOW CREATE TABLE system.{table}" + ) assert ( len( node.query(f"SHOW TABLES FROM system LIKE '{table}%'") @@ -103,7 +107,9 @@ def test_system_logs_recreate(): import logging for table in system_logs: - create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw") + create_table_sql = node.query( + f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw" + ) logging.debug( "With storage policy, SHOW CREATE TABLE system.%s is: %s", table, @@ -129,8 +135,12 @@ def test_system_logs_recreate(): node.restart_clickhouse() node.query("SYSTEM FLUSH LOGS") for table in system_logs: - assert "ENGINE = MergeTree" in node.query(f"SHOW CREATE TABLE system.{table}") - assert "ENGINE = Null" not in node.query(f"SHOW CREATE TABLE system.{table}") + assert "ENGINE = MergeTree" in node.query( + f"SHOW CREATE TABLE system.{table}" + ) + assert "ENGINE = Null" not in node.query( + f"SHOW CREATE TABLE system.{table}" + ) assert ( len( node.query(f"SHOW TABLES FROM system LIKE '{table}%'") From 3a89b7034fc9f6b6012a8eb7c277bca72b7b17c2 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Wed, 7 Aug 2024 19:16:22 +0800 Subject: [PATCH 0572/1722] modify integration test --- .../configs/settings.xml | 4 -- .../test.py | 38 +++++++++---------- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml index c637fe8eead..16caee9ba20 100644 --- a/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml +++ b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml @@ -1,10 +1,6 @@ - 1 - 1 - 0 - 0 diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/test.py b/tests/integration/test_replicated_database_alter_modify_order_by/test.py index 1ea126ace10..22355817ee6 100644 --- a/tests/integration/test_replicated_database_alter_modify_order_by/test.py +++ b/tests/integration/test_replicated_database_alter_modify_order_by/test.py @@ -1,15 +1,14 @@ -import re import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry, assert_logs_contain +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -main_node = cluster.add_instance( - "main_node", +shard1_node = cluster.add_instance( + "shard1_node", main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, @@ -17,18 +16,19 @@ main_node = cluster.add_instance( macros={"shard": 1, "replica": 1}, ) -snapshotting_node = cluster.add_instance( - "snapshotting_node", +shard2_node = cluster.add_instance( + "shard2_node", main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, + stay_alive=True, macros={"shard": 2, "replica": 1}, ) all_nodes = [ - main_node, - snapshotting_node, + shard1_node, + shard2_node, ] @@ -43,28 +43,28 @@ def started_cluster(): def test_alter_modify_order_by(started_cluster): - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard1_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard2_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - main_node.query( + shard1_node.query( "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" ) - main_node.query( + shard1_node.query( "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" ) - main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") - snapshotting_node.query( + shard1_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") + shard2_node.query( "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" ) query = ( "select count() from system.tables where database = 'alter_modify_order_by';" ) - expected = main_node.query(query) - assert_eq_with_retry(snapshotting_node, query, expected) + expected = shard1_node.query(query) + assert_eq_with_retry(shard2_node, query, expected) query = "show create table alter_modify_order_by.t1;" - assert main_node.query(query) == snapshotting_node.query(query) + assert shard1_node.query(query) == shard2_node.query(query) - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard1_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard2_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") From 49871bacc1d56fb82b78c70dbfc92d52003e2e99 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 7 Aug 2024 12:37:39 +0100 Subject: [PATCH 0573/1722] fix test --- .../poco/Net/include/Poco/Net/HTTPServerSession.h | 1 - base/poco/Net/src/HTTPServerSession.cpp | 1 - tests/integration/test_server_keep_alive/test.py | 15 ++++++++++++--- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index 54e7f2c8c50..b0659ca405c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -66,7 +66,6 @@ namespace Net bool _firstRequest; Poco::Timespan _keepAliveTimeout; int _maxKeepAliveRequests; - HTTPServerParams::Ptr _params; }; diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index 3093f215952..8eec3e14872 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -24,7 +24,6 @@ HTTPServerSession::HTTPServerSession(const StreamSocket & socket, HTTPServerPara , _firstRequest(true) , _keepAliveTimeout(pParams->getKeepAliveTimeout()) , _maxKeepAliveRequests(pParams->getMaxKeepAliveRequests()) - , _params(pParams) { setTimeout(pParams->getTimeout()); } diff --git a/tests/integration/test_server_keep_alive/test.py b/tests/integration/test_server_keep_alive/test.py index 96f08a37adb..e550319b6df 100644 --- a/tests/integration/test_server_keep_alive/test.py +++ b/tests/integration/test_server_keep_alive/test.py @@ -1,5 +1,6 @@ import logging import pytest +import random import requests from helpers.cluster import ClickHouseCluster @@ -24,19 +25,27 @@ def test_max_keep_alive_requests_on_user_side(start_cluster): # In this test we have `keep_alive_timeout` set to one hour to never trigger connection reset by timeout, `max_keep_alive_requests` is set to 5. # We expect server to close connection after each 5 requests. We detect connection reset by change in src port. # So the first 5 requests should come from the same port, the following 5 requests should come from another port. + + log_comments = [] + for _ in range(10): + rand_id = random.randint(0, 1000000) + log_comment = f"test_requests_with_keep_alive_{rand_id}" + log_comments.append(log_comment) + log_comments = sorted(log_comments) + session = requests.Session() for i in range(10): session.get( - f"http://{node.ip_address}:8123/?query=select%201&log_comment=test_requests_with_keep_alive_{i}" + f"http://{node.ip_address}:8123/?query=select%201&log_comment={log_comments[i]}" ) ports = node.query( - """ + f""" SYSTEM FLUSH LOGS; SELECT port FROM system.query_log - WHERE log_comment like 'test_requests_with_keep_alive_%' AND type = 'QueryFinish' + WHERE log_comment IN ({", ".join(f"'{comment}'" for comment in log_comments)}) AND type = 'QueryFinish' ORDER BY log_comment """ ).split("\n")[:-1] From ddc058aa6ff6780fa67bc5c59d9d7ff9a71d4ee1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Aug 2024 12:51:06 +0200 Subject: [PATCH 0574/1722] Update minio in stateless tests --- docker/test/stateless/Dockerfile | 4 ++-- docker/test/stateless/setup_minio.sh | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index a0e5513a3a2..d8eb072328f 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -69,8 +69,8 @@ ENV MAX_RUN_TIME=0 # Unrelated to vars in setup_minio.sh, but should be the same there # to have the same binaries for local running scenario -ARG MINIO_SERVER_VERSION=2022-01-03T18-22-58Z -ARG MINIO_CLIENT_VERSION=2022-01-05T23-52-51Z +ARG MINIO_SERVER_VERSION=2024-08-03T04-33-23Z +ARG MINIO_CLIENT_VERSION=2024-07-31T15-58-33Z ARG TARGETARCH # Download Minio-related binaries diff --git a/docker/test/stateless/setup_minio.sh b/docker/test/stateless/setup_minio.sh index 2b9433edd20..d8310d072b8 100755 --- a/docker/test/stateless/setup_minio.sh +++ b/docker/test/stateless/setup_minio.sh @@ -59,8 +59,8 @@ find_os() { download_minio() { local os local arch - local minio_server_version=${MINIO_SERVER_VERSION:-2022-09-07T22-25-02Z} - local minio_client_version=${MINIO_CLIENT_VERSION:-2022-08-28T20-08-11Z} + local minio_server_version=${MINIO_SERVER_VERSION:-2024-08-03T04-33-23Z} + local minio_client_version=${MINIO_CLIENT_VERSION:-2024-07-31T15-58-33Z} os=$(find_os) arch=$(find_arch) @@ -82,10 +82,10 @@ setup_minio() { local test_type=$1 ./mc alias set clickminio http://localhost:11111 clickhouse clickhouse ./mc admin user add clickminio test testtest - ./mc admin policy set clickminio readwrite user=test + ./mc admin policy attach clickminio readwrite --user=test ./mc mb --ignore-existing clickminio/test if [ "$test_type" = "stateless" ]; then - ./mc policy set public clickminio/test + ./mc anonymous set public clickminio/test fi } @@ -148,4 +148,4 @@ main() { setup_aws_credentials } -main "$@" \ No newline at end of file +main "$@" From 6afff5824e3bd3e4eca2c50d7cce10dda5678433 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Aug 2024 12:14:05 +0000 Subject: [PATCH 0575/1722] Fix 03130_convert_outer_join_to_inner_join --- .../03130_convert_outer_join_to_inner_join.sql | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index 28362f1f469..168177a06d7 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -6,14 +6,18 @@ CREATE TABLE test_table_1 ( id UInt64, value String -) ENGINE=MergeTree ORDER BY id; +) ENGINE=MergeTree ORDER BY id +SETTINGS index_granularity = 16 # We have number of granules in the `EXPLAIN` output in reference file +; DROP TABLE IF EXISTS test_table_2; CREATE TABLE test_table_2 ( id UInt64, value String -) ENGINE=MergeTree ORDER BY id; +) ENGINE=MergeTree ORDER BY id +SETTINGS index_granularity = 16 +; INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); From 6882e8ad79bcc1ce0b2c80e82aee38f57a4504bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 14:34:33 +0200 Subject: [PATCH 0576/1722] Revert "Merge pull request #67130 from rschu1ze/unflake-win-view-tests2" This reverts commit 48e61a295cf3e2a7ab93dc99531d14701b90004c, reversing changes made to da24aa06fac26bf1516320cc6e49c8927b1f600a. --- .../queries/0_stateless/01052_window_view_proc_tumble_to_now.sh | 1 - tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh | 1 - tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh | 1 - tests/queries/0_stateless/01055_window_view_proc_hop_to.sh | 1 - .../0_stateless/01075_window_view_proc_tumble_to_now_populate.sh | 1 - 5 files changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh index c473bf766b0..a235b60cee9 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh index ca89dd9daf1..b4a647c9864 100755 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh index 67cac226de6..af9a47714b5 100755 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh index e44fc2e363c..5a7e92e6a1b 100755 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh b/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh index 859b9a86a2a..220bb39602b 100755 --- a/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh +++ b/tests/queries/0_stateless/01075_window_view_proc_tumble_to_now_populate.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4a537874cad725227c847850b2da42d0ed86ccaf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 14:35:05 +0200 Subject: [PATCH 0577/1722] adjust tests --- src/Disks/DiskFomAST.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 22 +++++++++---------- ...2808_custom_disk_with_user_defined_name.sh | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 2a5e7368de9..35cb124acfd 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -144,7 +144,7 @@ std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, Conte if (result->isCustomDisk()) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` is a custom disk that is used in other table." + "Disk name `{}` is a custom disk that is used in other table. " "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", disk_name); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index f72b24e3270..e11af43ed23 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -59,19 +59,19 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte CustomType custom; if (name == "disk") { + ASTPtr value_as_custom_ast = nullptr; if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) + value_as_custom_ast = dynamic_cast(custom.getImpl()).ast; + + if (value_as_custom_ast && isDiskFunction(value_as_custom_ast)) { - auto ast = dynamic_cast(custom.getImpl()).ast; - if (ast && isDiskFunction(ast)) - { - auto disk_name = DiskFomAST::createCustomDisk(ast, context, is_attach); - LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); - value = disk_name; - } - else - { - value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); - } + auto disk_name = DiskFomAST::createCustomDisk(value_as_custom_ast, context, is_attach); + LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name); + value = disk_name; + } + else + { + value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); } if (has("storage_policy")) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 333bc1bc25d..b62adea5683 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS disk = disk(name = 's3_disk', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); -""" 2>&1 | grep -q "Disk with name \`s3_disk\` already exist" && echo 'OK' || echo 'FAIL' +""" 2>&1 | grep -q "The disk \`s3_disk\` is already exist and described by the config" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" From cfeb20681d3af5f2a5d538096e5117be06e7c624 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 7 Aug 2024 14:42:42 +0200 Subject: [PATCH 0578/1722] Fix style check --- tests/clickhouse-test | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c3b1d4d907c..5946e561949 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2068,10 +2068,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] - + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( + all_tags_and_random_settings_limits[1] + ) self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From 5f3417153439b56b68a34be890b75884941800dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 15:12:06 +0200 Subject: [PATCH 0579/1722] Fix window view missing blocks due to slow flush to view --- src/Storages/WindowView/StorageWindowView.cpp | 25 ++++++++++++++++--- .../01056_window_view_proc_hop_watch.py | 7 +++--- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 5830c844582..94eed575ca8 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1051,17 +1051,27 @@ void StorageWindowView::threadFuncFireProc() if (shutdown_called) return; + /// Acquiring the lock can take seconds (depends on how long it takes to push) so we keep a reference to remember + /// what's the starting point where we want to push from + UInt32 timestamp_start = now(); + std::lock_guard lock(fire_signal_mutex); /// TODO: consider using time_t instead (for every timestamp in this class) UInt32 timestamp_now = now(); - LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}", timestamp_now, next_fire_signal, max_watermark); + LOG_TRACE( + log, + "Start: {}, now: {}, next fire signal: {}, max watermark: {}", + timestamp_start, + timestamp_now, + next_fire_signal, + max_watermark); while (next_fire_signal <= timestamp_now) { try { - if (max_watermark >= timestamp_now) + if (max_watermark >= timestamp_start) fire(next_fire_signal); } catch (...) @@ -1075,8 +1085,15 @@ void StorageWindowView::threadFuncFireProc() slide_interval *= 86400; next_fire_signal += slide_interval; - LOG_TRACE(log, "Now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}", - timestamp_now, next_fire_signal, max_watermark, max_fired_watermark, slide_interval); + LOG_TRACE( + log, + "Start: {}, now: {}, next fire signal: {}, max watermark: {}, max fired watermark: {}, slide interval: {}", + timestamp_start, + timestamp_now, + next_fire_signal, + max_watermark, + max_fired_watermark, + slide_interval); } if (max_watermark >= timestamp_now) diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index 6bb8db2c38e..bc4ec16f1b1 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -20,7 +20,7 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.expect(prompt) - client1.send("SET enable_analyzer = 0") + client1.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") client1.expect(prompt) client1.send("SET allow_experimental_window_view = 1") client1.expect(prompt) @@ -28,7 +28,7 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) - client2.send("SET enable_analyzer = 0") + client2.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch") @@ -42,8 +42,9 @@ with client(name="client1>", log=log) as client1, client( "CREATE TABLE 01056_window_view_proc_hop_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()" ) client1.expect(prompt) + # Introduce a sleep call to verify that even if the push to view is slow WATCH will work client1.send( - "CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;" + "CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(sleep(5)) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;" ) client1.expect(prompt) From 28c8d158635ef2564a60c25a6f971aa324030dcc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 13:37:56 +0000 Subject: [PATCH 0580/1722] Fix style --- tests/queries/0_stateless/03217_filtering_in_system_tables.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index bbc755e478d..72ca7c8684d 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -16,10 +16,12 @@ SYSTEM FLUSH LOGS; -- StorageSystemTables SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND current_database = currentDatabase() AND query LIKE '%SELECT database, table FROM system.tables WHERE database = \'information_schema\' AND table = \'tables\';' AND type = 'QueryFinish'; -- StorageSystemReplicas SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 + AND current_database = currentDatabase() AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' AND type = 'QueryFinish'; From aca7e6734ceb5a0ec7a8d6f8f3bfa5bde05c860e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Aug 2024 16:24:56 +0200 Subject: [PATCH 0581/1722] Style --- .../0_stateless/01056_window_view_proc_hop_watch.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index bc4ec16f1b1..e0f969050b5 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -20,7 +20,9 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.expect(prompt) - client1.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") + client1.send( + "SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000" + ) client1.expect(prompt) client1.send("SET allow_experimental_window_view = 1") client1.expect(prompt) @@ -28,7 +30,9 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) - client2.send("SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000") + client2.send( + "SET enable_analyzer = 0, function_sleep_max_microseconds_per_block=10000000" + ) client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch") From d2e9833dba4038999e7aeb540ea6b56cc1c59449 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Aug 2024 17:00:41 +0200 Subject: [PATCH 0582/1722] Add minio audit logs --- docker/test/stateless/run.sh | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ea32df23af0..b33c261dacc 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -176,6 +176,25 @@ done setup_logs_replication attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 +# create minio log webhooks for both audit and server logs +clickhouse-client --query "CREATE TABLE minio_audit_logs +( + log String, + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') +) +ENGINE = MergeTree +ORDER BY tuple()" +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" + +clickhouse-client --query "CREATE TABLE minio_server_logs +( + log String, + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') +) +ENGINE = MergeTree +ORDER BY tuple()" +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" + function fn_exists() { declare -F "$1" > /dev/null; } @@ -328,6 +347,11 @@ do fi done + +# collect minio audit and server logs +clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" +clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" + # Stop server so we can safely read data with clickhouse-local. # Why do we read data with clickhouse-local? # Because it's the simplest way to read it when server has crashed. From ad678cb5a8d4533a88273b8244dc0844c83e641c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2024 18:24:03 +0200 Subject: [PATCH 0583/1722] Ignore disappeared projections on start --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 33 +++++++++++- .../MergeTree/MergeTreeDataPartChecksum.cpp | 6 --- .../MergeTree/MergeTreeDataPartChecksum.h | 3 -- .../test_broken_projections/test.py | 50 +++++++++++++++++++ 4 files changed, 81 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3a44359b537..918a4cda714 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -72,6 +72,7 @@ namespace ErrorCodes extern const int BAD_TTL_FILE; extern const int NOT_IMPLEMENTED; extern const int NO_SUCH_COLUMN_IN_TABLE; + extern const int FILE_DOESNT_EXIST; } @@ -749,8 +750,16 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks /// Probably there is something wrong with files of this part. /// So it can be helpful to add to the error message some information about those files. String files_in_part; + for (auto it = getDataPartStorage().iterate(); it->isValid(); it->next()) - files_in_part += fmt::format("{}{} ({} bytes)", (files_in_part.empty() ? "" : ", "), it->name(), getDataPartStorage().getFileSize(it->name())); + { + std::string file_info; + if (!getDataPartStorage().isDirectory(it->name())) + file_info = fmt::format(" ({} bytes)", getDataPartStorage().getFileSize(it->name())); + + files_in_part += fmt::format("{}{}{}", (files_in_part.empty() ? "" : ", "), it->name(), file_info); + + } if (!files_in_part.empty()) e->addMessage("Part contains files: {}", files_in_part); if (isEmpty()) @@ -2141,7 +2150,27 @@ void IMergeTreeDataPart::checkConsistencyBase() const } } - checksums.checkSizes(getDataPartStorage()); + const auto & data_part_storage = getDataPartStorage(); + for (const auto & [filename, checksum] : checksums.files) + { + try + { + checksum.checkSize(data_part_storage, filename); + } + catch (const Exception & ex) + { + /// For projection parts check will mark them broken in loadProjections + if (!parent_part && filename.ends_with(".proj")) + { + std::string projection_name = fs::path(filename).stem(); + LOG_INFO(storage.log, "Projection {} doesn't exist on start for part {}, marking it as broken", projection_name, name); + if (hasProjection(projection_name)) + markProjectionPartAsBroken(projection_name, ex.message(), ex.code()); + } + else + throw; + } + } } else { diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index b327480fa92..3ef36ce364c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -100,12 +100,6 @@ void MergeTreeDataPartChecksums::checkEqual(const MergeTreeDataPartChecksums & r } } -void MergeTreeDataPartChecksums::checkSizes(const IDataPartStorage & storage) const -{ - for (const auto & [name, checksum] : files) - checksum.checkSize(storage, name); -} - UInt64 MergeTreeDataPartChecksums::getTotalSizeOnDisk() const { UInt64 res = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h index 05178dc3a60..dc52f1ada2b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h @@ -65,9 +65,6 @@ struct MergeTreeDataPartChecksums static bool isBadChecksumsErrorCode(int code); - /// Checks that the directory contains all the needed files of the correct size. Does not check the checksum. - void checkSizes(const IDataPartStorage & storage) const; - /// Returns false if the checksum is too old. bool read(ReadBuffer & in); /// Assume that header with version (the first line) is read diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 162c0dbaa2f..578ff42369c 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -4,6 +4,7 @@ import logging import string import random from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) @@ -18,6 +19,12 @@ def cluster(): stay_alive=True, with_zookeeper=True, ) + cluster.add_instance( + "node_restart", + main_configs=["config.d/dont_start_broken.xml"], + stay_alive=True, + with_zookeeper=True, + ) logging.info("Starting cluster...") cluster.start() @@ -632,6 +639,49 @@ def test_broken_on_start(cluster): check(node, table_name, 0) +def test_disappeared_projection_on_start(cluster): + node = cluster.instances["node_restart"] + + table_name = "test_disapperead_projection" + create_table(node, table_name, 1) + + node.query(f"SYSTEM STOP MERGES {table_name}") + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + def drop_projection(): + node.query( + f"ALTER TABLE {table_name} DROP PROJECTION proj2", + settings={"mutations_sync": "0"}, + ) + + p = Pool(2) + p.apply_async(drop_projection) + + for i in range(30): + create_query = node.query(f"SHOW CREATE TABLE {table_name}") + if "proj2" not in create_query: + break + time.sleep(0.5) + + assert "proj2" not in create_query + + # Remove 'proj2' for part all_2_2_0 + break_projection(node, table_name, "proj2", "all_2_2_0", "part") + + node.restart_clickhouse() + + # proj2 is not broken, it doesn't exist, but ok + check(node, table_name, 0, expect_broken_part="proj2", do_check_command=0) + + def test_mutation_with_broken_projection(cluster): node = cluster.instances["node"] From eee3811e89ed229ffedb26e97200b80edf399dcd Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Thu, 8 Aug 2024 00:43:37 +0800 Subject: [PATCH 0584/1722] empty commit From f5e993df2a3ed28505a6fdc22db16306c0308733 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 7 Aug 2024 16:46:33 +0000 Subject: [PATCH 0585/1722] Add tests --- .../DataLakes/IStorageDataLake.h | 49 +- .../StorageObjectStorageSource.cpp | 2 + .../registerDataLakeTableFunctions.cpp | 20 +- tests/integration/helpers/cloud_tools.py | 13 +- .../test_storage_azure_blob_storage/test.py | 2 + .../configs/config.d/named_collections.xml | 4 + .../integration/test_storage_iceberg/test.py | 424 +++++++++--------- 7 files changed, 291 insertions(+), 223 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index c8603fccb86..123a629f395 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -48,10 +48,20 @@ public: ConfigurationPtr configuration = base_configuration->clone(); + try { + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start1"); metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - configuration->setPaths(metadata->getDataFiles()); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish1"); + + auto data_files = metadata->getDataFiles(); + LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data file name: {}", data_file); + } + configuration->setPaths(data_files); if (use_schema_from_metadata) schema_from_metadata = metadata->getTableSchema(); } @@ -80,9 +90,20 @@ public: const std::optional & format_settings_, ContextPtr local_context) { + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start2"); + auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish2"); + + auto schema_from_metadata = metadata->getTableSchema(); + auto data_files = metadata->getDataFiles(); + LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data file name: {}", data_file); + } if (!schema_from_metadata.empty()) { return ColumnsDescription(std::move(schema_from_metadata)); @@ -90,7 +111,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(metadata->getDataFiles()); + configuration->setPaths(data_files); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } @@ -99,14 +120,33 @@ public: void updateConfiguration(ContextPtr local_context) override { Storage::updateConfiguration(local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start3"); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish3"); + if (current_metadata && *current_metadata == *new_metadata) return; + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 1"); + + current_metadata = std::move(new_metadata); + + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 2"); + + auto data_files = current_metadata->getDataFiles(); + + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 3"); + + LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data file name: {}", data_file); + } auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(current_metadata->getDataFiles()); + updated_configuration->setPaths(data_files); updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); Storage::configuration = updated_configuration; @@ -148,7 +188,10 @@ private: if (!current_metadata) { Storage::updateConfiguration(local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start4"); + current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish4"); } auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); if (!column_mapping.empty()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index e760098f10f..399e4a56ba8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -438,6 +438,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + LOG_DEBUG(&Poco::Logger::get("Get path"), "Path: {}", object_info.getPath()); + auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 40561a1e075..8de852f4fba 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -17,13 +17,25 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); # endif # if USE_AZURE_BLOB_STORAGE + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", + .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); # endif + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", + .examples{{"icebergLocal", "SELECT * FROM icebergLocal(filename)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); } #endif @@ -62,11 +74,13 @@ void registerTableFunctionHudi(TableFunctionFactory & factory) void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); -#if USE_AWS_S3 #if USE_AVRO +# if USE_AWS_S3 registerTableFunctionIceberg(factory); #endif -#if USE_PARQUET +#endif +#if USE_AWS_S3 +# if USE_PARQUET registerTableFunctionDeltaLake(factory); #endif registerTableFunctionHudi(factory); diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/cloud_tools.py index 534791b8bc5..0d009b5f3fe 100644 --- a/tests/integration/helpers/cloud_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -52,15 +52,16 @@ class LocalUploader(CloudUploader): class AzureUploader(CloudUploader): - def __init__(self, blob_service_client, container_name): - self.blob_service_client = blob_service_client - self.container_name = container_name + + def __init__(self, container_client): + self.container_client = container_client def upload_file(self, local_path, remote_blob_path): - blob_client = self.blob_service_client.get_blob_client( - container=self.container_name, blob=remote_blob_path - ) + # print("Local path", local_path) + # print("Remote blob path", remote_blob_path) + blob_client = self.container_client.get_blob_client(remote_blob_path) with open(local_path, "rb") as data: + # print("Data", data) blob_client.upload_blob(data, overwrite=True) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6fbe7634642..c5599d20519 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -550,6 +550,7 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] + print("Account URL: ", cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " @@ -559,6 +560,7 @@ def test_simple_write_account_string_table_function(cluster): ) print(get_azure_file_content("test_simple_write_tf.csv", port)) assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' + assert 0 == 1 def test_simple_write_connection_string_table_function(cluster): diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index d4c54e2d13d..3d80fa2ecf9 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -5,5 +5,9 @@ minio minio123 + + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 7f83846bd89..b8a67600785 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -36,6 +36,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject +from azure.storage.blob import BlobServiceClient from helpers.cloud_tools import ( prepare_s3_bucket, @@ -92,12 +93,27 @@ def started_cluster(): cluster.minio_client, cluster.minio_bucket ) - container_name = "my_container" + cluster.azure_container_name = "mycontainer" - cluster.default_azurite_uploader = AzureUploader( - cluster.blob_service_client, container_name + # connection_string = ( + # f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + # f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + # f"BlobEndpoint=http://azurite1:{cluster.env_variables['AZURITE_PORT']}/devstoreaccount1;" + # ) + # local_blob_service_client = BlobServiceClient.from_connection_string( + # cluster.env_variables["AZURITE_CONNECTION_STRING"] + # ) + + local_blob_service_client = cluster.blob_service_client + + container_client = local_blob_service_client.create_container( + cluster.azure_container_name ) + cluster.container_client = container_client + + cluster.default_azure_uploader = AzureUploader(container_client) + yield cluster finally: @@ -162,10 +178,24 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwargs): +def create_iceberg_table( + storage_type, + node, + table_name, + cluster, + format="Parquet", + table_function=False, + **kwargs, +): if storage_type == "local": pass elif storage_type == "s3": + if "bucket" in kwargs: + bucket = kwargs["bucket"] + else: + bucket = cluster.minio_bucket + if table_function: + return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" node.query( f""" DROP TABLE IF EXISTS {table_name}; @@ -173,11 +203,15 @@ def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwa ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" ) elif storage_type == "azure": + if table_function: + return f""" + icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) + """ node.query( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=IcebergAzure(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" ) else: raise Exception("Unknown iceberg storage type: {}", storage_type) @@ -218,11 +252,11 @@ def default_upload_directory(started_cluster, storage_type, local_path, remote_p @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_single_iceberg_file_" + format_version + TABLE_NAME = "test_single_iceberg_file_" + format_version + "_" + storage_type write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) @@ -233,96 +267,7 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): "", ) - # for bucket in minio_client.list_buckets(): - # for object in minio_client.list_objects(bucket.name, recursive=True): - # print("Object: ", object.object_name) - # extension = object.object_name.split(".")[-1] - # print("File extension: ", extension) - # try: - # response = minio_client.get_object( - # object.bucket_name, object.object_name - # ) - - # if extension == "avro": - # avro_bytes = response.read() - - # # Use BytesIO to create a file-like object from the byte string - # avro_file = io.BytesIO(avro_bytes) - - # # Read the Avro data - # reader = avro.datafile.DataFileReader( - # avro_file, avro.io.DatumReader() - # ) - # records = [record for record in reader] - - # # Close the reader - # reader.close() - - # # Now you can work with the records - # for record in records: - # # print(json.dumps(record, indent=4, sort_keys=True)) - # print(str(record)) - # # my_json = ( - # # str(record) - # # .replace("'", '"') - # # .replace("None", "null") - # # .replace('b"', '"') - # # ) - # # print(my_json) - # # data = json.loads(my_json) - # # s = json.dumps(data, indent=4, sort_keys=True) - # # print(s) - # elif extension == "json": - # my_bytes_value = response.read() - # my_json = my_bytes_value.decode("utf8").replace("'", '"') - # data = json.loads(my_json) - # s = json.dumps(data, indent=4, sort_keys=True) - # print(s) - # elif extension == "parquet": - # # print("To be continued...") - # # # Your byte string containing the Parquet data - # # parquet_bytes = response.read() - - # # # Use BytesIO to create a file-like object from the byte string - # # parquet_file = io.BytesIO(parquet_bytes) - - # # # Read the Parquet data into a PyArrow Table - # # table = pq.read_table(parquet_file) - - # # # Convert the PyArrow Table to a Pandas DataFrame - # # df = table.to_pandas() - - # # # Now you can work with s DataFrame - # # print(df) - # parquet_bytes = ( - # response.read() - # ) # Replace with your actual byte string - - # # Create a temporary file and write the byte string to it - # with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - # tmp_file.write(parquet_bytes) - # tmp_file_path = tmp_file.name - - # # Read the Parquet file using PySpark - # df = spark.read.parquet(tmp_file_path) - - # # Show the DataFrame - # print(df.toPandas()) - # else: - # print(response.read()) - - # finally: - # print("----------------") - # response.close() - # response.release_conn() - - create_iceberg_table(instance, TABLE_NAME) - - # print("Debug Print") - - # print(instance.query(f"SELECT * FROM {TABLE_NAME}")) - - # print(instance.query("SELECT number FROM numbers(100)")) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( "SELECT number, toString(number + 1) FROM numbers(100)" @@ -332,12 +277,11 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_partition_by(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_by_" + format_version + TABLE_NAME = "test_partition_by_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -348,22 +292,23 @@ def test_partition_by(started_cluster, format_version): partition_by="a", ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert len(files) == 14 # 10 partitiions + 4 metadata files - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_multiple_iceberg_files(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_multiple_iceberg_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_iceberg_files_" + format_version + TABLE_NAME = "test_multiple_iceberg_files_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -373,9 +318,13 @@ def test_multiple_iceberg_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) + # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/3127466b-299d-48ca-a367-6b9b1df1e78c-m0.avro', @@ -383,7 +332,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): # '/iceberg_data/default/test_multiple_iceberg_files/metadata/v1.metadata.json'] assert len(files) == 5 - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 write_iceberg_from_df( @@ -393,8 +342,11 @@ def test_multiple_iceberg_files(started_cluster, format_version): mode="append", format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert len(files) == 9 @@ -405,12 +357,11 @@ def test_multiple_iceberg_files(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_types(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_types_" + format_version + TABLE_NAME = "test_types_" + format_version + "_" + storage_type data = [ ( @@ -436,24 +387,29 @@ def test_types(started_cluster, format_version): spark, df, TABLE_NAME, mode="overwrite", format_version=format_version ) - upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1 assert ( instance.query(f"SELECT a, b, c, d, e FROM {TABLE_NAME}").strip() == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - table_function = f"iceberg(s3, filename='iceberg_data/default/{TABLE_NAME}/')" + table_function_expr = create_iceberg_table( + storage_type, instance, TABLE_NAME, started_cluster, table_function=True + ) assert ( - instance.query(f"SELECT a, b, c, d, e FROM {table_function}").strip() + instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - assert instance.query(f"DESCRIBE {table_function} FORMAT TSV") == TSV( + assert instance.query(f"DESCRIBE {table_function_expr} FORMAT TSV") == TSV( [ ["a", "Nullable(Int32)"], ["b", "Nullable(String)"], @@ -465,12 +421,13 @@ def test_types(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_delete_files(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_delete_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_delete_files_" + format_version + TABLE_NAME = "test_delete_files_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -480,17 +437,22 @@ def test_delete_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 @@ -504,27 +466,34 @@ def test_delete_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_evolved_schema(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_evolved_schema(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_evolved_schema_" + format_version + TABLE_NAME = "test_evolved_schema_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -534,19 +503,25 @@ def test_evolved_schema(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 expected_data = instance.query(f"SELECT * FROM {TABLE_NAME} order by a, b") spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -558,12 +533,13 @@ def test_evolved_schema(started_cluster, format_version): assert data == expected_data -def test_row_based_deletes(started_cluster): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_row_based_deletes(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_row_based_deletes" + TABLE_NAME = "test_row_based_deletes_" + storage_type spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -572,17 +548,23 @@ def test_row_based_deletes(started_cluster): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)" ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -590,13 +572,19 @@ def test_row_based_deletes(started_cluster): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_schema_inference(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_schema_inference(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket for format in ["Parquet", "ORC", "Avro"]: - TABLE_NAME = "test_schema_inference_" + format + "_" + format_version + TABLE_NAME = ( + "test_schema_inference_" + + format + + "_" + + format_version + + "_" + + storage_type + ) # Types time, timestamptz, fixed are not supported in Spark. spark.sql( @@ -606,12 +594,16 @@ def test_schema_inference(started_cluster, format_version): spark.sql( f"insert into {TABLE_NAME} select 42, 4242, 42.42, 4242.4242, decimal(42.42), decimal(42.42), decimal(42.42), date('2020-01-01'), timestamp('2020-01-01 20:00:00'), 'hello', binary('hello'), array(1,2,3), map('key', 'value'), struct(42, 'hello'), array(struct(map('key', array(map('key', 42))), struct(42, 'hello')))" ) - - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME, format) + create_iceberg_table( + storage_type, instance, TABLE_NAME, started_cluster, format=format + ) res = instance.query( f"DESC {TABLE_NAME} FORMAT TSVRaw", settings={"print_pretty_type_names": 0} @@ -646,12 +638,11 @@ def test_schema_inference(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_metadata_file_selection(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_" + format_version + TABLE_NAME = "test_metadata_selection_" + format_version + "_" + storage_type spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -662,22 +653,26 @@ def test_metadata_file_selection(started_cluster, format_version): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_metadata_file_format_with_uuid(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_with_uuid_" + format_version + TABLE_NAME = ( + "test_metadata_selection_with_uuid_" + format_version + "_" + storage_type + ) spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -694,63 +689,70 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 -def test_restart_broken(started_cluster): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = "broken2" - TABLE_NAME = "test_restart_broken_table_function" +# @pytest.mark.parametrize("storage_type", ["s3", "azure"]) +# def test_restart_broken(started_cluster): +# instance = started_cluster.instances["node1"] +# spark = started_cluster.spark_session +# minio_client = started_cluster.minio_client +# bucket = "broken2" +# TABLE_NAME = "test_restart_broken_table_function" - if not minio_client.bucket_exists(bucket): - minio_client.make_bucket(bucket) +# if not minio_client.bucket_exists(bucket): +# minio_client.make_bucket(bucket) - parquet_data_path = create_initial_data_file( - started_cluster, - instance, - "SELECT number, toString(number) FROM numbers(100)", - TABLE_NAME, - ) +# parquet_data_path = create_initial_data_file( +# started_cluster, +# instance, +# "SELECT number, toString(number) FROM numbers(100)", +# TABLE_NAME, +# ) - write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" - ) - create_iceberg_table(instance, TABLE_NAME, bucket=bucket) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 +# write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") +# files = default_upload_directory( +# started_cluster, +# storage_type, +# f"/iceberg_data/default/{TABLE_NAME}/", +# "", +# ) +# create_iceberg_table(instance, TABLE_NAME, bucket=bucket) +# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - s3_objects = list_s3_objects(minio_client, bucket, prefix="") - assert ( - len( - list( - minio_client.remove_objects( - bucket, - [DeleteObject(obj) for obj in s3_objects], - ) - ) - ) - == 0 - ) - minio_client.remove_bucket(bucket) +# s3_objects = list_s3_objects(minio_client, bucket, prefix="") +# assert ( +# len( +# list( +# minio_client.remove_objects( +# bucket, +# [DeleteObject(obj) for obj in s3_objects], +# ) +# ) +# ) +# == 0 +# ) +# minio_client.remove_bucket(bucket) - instance.restart_clickhouse() +# instance.restart_clickhouse() - assert "NoSuchBucket" in instance.query_and_get_error( - f"SELECT count() FROM {TABLE_NAME}" - ) +# assert "NoSuchBucket" in instance.query_and_get_error( +# f"SELECT count() FROM {TABLE_NAME}" +# ) - minio_client.make_bucket(bucket) +# minio_client.make_bucket(bucket) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" - ) +# files = default_upload_directory( +# S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" +# ) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 +# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 0dc4d773edd530494c1ab514d104a45665bdd16a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 7 Aug 2024 18:46:34 +0200 Subject: [PATCH 0586/1722] Fxi style --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 918a4cda714..93904c1a838 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -72,7 +72,6 @@ namespace ErrorCodes extern const int BAD_TTL_FILE; extern const int NOT_IMPLEMENTED; extern const int NO_SUCH_COLUMN_IN_TABLE; - extern const int FILE_DOESNT_EXIST; } From 06d154055f9e233180f13585e43e2992ae5ccfdf Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 7 Aug 2024 19:19:33 +0200 Subject: [PATCH 0587/1722] adjust tests --- tests/integration/test_disk_configuration/test.py | 2 +- tests/queries/0_stateless/03008_s3_plain_rewritable.sh | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index afc5303298c..f297c665dc5 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -373,7 +373,7 @@ def test_merge_tree_setting_override(start_cluster): CREATE TABLE {TABLE_NAME} (a Int32) ENGINE = MergeTree() ORDER BY tuple() - SETTINGS disk = 'kek', storage_policy = 's3'; + SETTINGS disk = 's3', storage_policy = 's3'; """ ) ) diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable.sh b/tests/queries/0_stateless/03008_s3_plain_rewritable.sh index 4d5989f6f12..8eea7940774 100755 --- a/tests/queries/0_stateless/03008_s3_plain_rewritable.sh +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable.sh @@ -46,7 +46,12 @@ ${CLICKHOUSE_CLIENT} --query "drop table if exists test_s3_mt_dst" ${CLICKHOUSE_CLIENT} -m --query " create table test_s3_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) -settings disk = '03008_s3_plain_rewritable' +settings disk = disk( + name = 03008_s3_plain_rewritable, + type = s3_plain_rewritable, + endpoint = 'http://localhost:11111/test/03008_test_s3_mt/', + access_key_id = clickhouse, + secret_access_key = clickhouse); " ${CLICKHOUSE_CLIENT} -m --query " From ec3a248e70e16d2cb4db1ec5ff17e95cc11dedae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Aug 2024 17:37:54 +0000 Subject: [PATCH 0588/1722] Fix clang-tidy --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b24d7968b61..5ee0bd328e0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1146,7 +1146,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( auto metadata_snapshot = getInMemoryMetadataPtr(); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); - auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false); + auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_partial_result=*/ false); if (!filter_dag) return {}; From 1082792950ca7b962c1288ab49bb8ff3ca855bbe Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 7 Aug 2024 20:21:50 +0100 Subject: [PATCH 0589/1722] fix test --- .../test_async_metrics_in_cgroup/test.py | 98 +++++++++---------- 1 file changed, 45 insertions(+), 53 deletions(-) diff --git a/tests/integration/test_async_metrics_in_cgroup/test.py b/tests/integration/test_async_metrics_in_cgroup/test.py index 00951c95a0e..d9f2e3aaaed 100644 --- a/tests/integration/test_async_metrics_in_cgroup/test.py +++ b/tests/integration/test_async_metrics_in_cgroup/test.py @@ -1,11 +1,10 @@ import pytest -import subprocess -import time from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") +node1 = cluster.add_instance("node1", stay_alive=True) +node2 = cluster.add_instance("node2", stay_alive=True) @pytest.fixture(scope="module") @@ -17,61 +16,54 @@ def start_cluster(): cluster.shutdown() -def test_user_cpu_accounting(start_cluster): - if node.is_built_with_sanitizer(): - pytest.skip("Disabled for sanitizers") - - # check that our metrics sources actually exist - assert ( - subprocess.Popen("test -f /sys/fs/cgroup/cpu.stat".split(" ")).wait() == 0 - or subprocess.Popen( - "test -f /sys/fs/cgroup/cpuacct/cpuacct.stat".split(" ") - ).wait() - == 0 - ) - - # first let's spawn some cpu-intensive process outside of the container and check that it doesn't accounted by ClickHouse server - proc = subprocess.Popen( - "openssl speed -multi 8".split(" "), - stdout=subprocess.DEVNULL, - stderr=subprocess.DEVNULL, - ) - - time.sleep(5) - - metric = node.query( - """ - SELECT max(value) - FROM ( - SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t - ) - """ - ).strip("\n") - - assert float(metric) < 2 - - proc.kill() - - # then let's test that we will account cpu time spent by the server itself +def run_cpu_intensive_task(node): node.query( - "SELECT cityHash64(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=10", + "SELECT sum(*) FROM system.numbers_mt FORMAT Null SETTINGS max_execution_time=10", ignore_error=True, ) - metric = node.query( + +def get_async_metric(node, metric): + node.query("SYSTEM FLUSH LOGS") + return node.query( + f""" + SELECT max(value) + FROM ( + SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value + FROM system.asynchronous_metric_log + WHERE event_time >= now() - 60 AND metric = '{metric}' + GROUP BY t + ) + SETTINGS max_threads = 1 """ - SELECT max(value) - FROM ( - SELECT toStartOfInterval(event_time, toIntervalSecond(1)) AS t, avg(value) AS value - FROM system.asynchronous_metric_log - WHERE event_time >= now() - 60 AND metric = 'OSUserTime' - GROUP BY t - ) - """ ).strip("\n") + +def test_user_cpu_accounting(start_cluster): + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + + # run query on the other node, its usage shouldn't be accounted by node1 + run_cpu_intensive_task(node2) + + node1_cpu_time = get_async_metric(node1, "OSUserTime") + assert float(node1_cpu_time) < 2 + + # then let's test that we will account cpu time spent by the server itself + node2_cpu_time = get_async_metric(node2, "OSUserTime") # this check is really weak, but CI is tough place and we cannot guarantee that test process will get many cpu time - assert float(metric) > 1 + assert float(node2_cpu_time) > 2 + + +def test_normalized_user_cpu(start_cluster): + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + + # run query on the other node, its usage shouldn't be accounted by node1 + run_cpu_intensive_task(node2) + + node1_cpu_time = get_async_metric(node1, "OSUserTimeNormalized") + assert float(node1_cpu_time) < 1.01 + + node2_cpu_time = get_async_metric(node2, "OSUserTimeNormalized") + assert float(node2_cpu_time) < 1.01 From d81b5239debaf01b74521511db44d6cb4cd419c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Aug 2024 21:37:01 +0200 Subject: [PATCH 0590/1722] Remove unused CLI option --- src/Client/ClientApplicationBase.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index 9f133616d2e..71d13ad4f53 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -200,8 +200,6 @@ void ClientApplicationBase::init(int argc, char ** argv) ("pager", po::value(), "Pipe all output into this command (less or similar)") ("max_memory_usage_in_client", po::value(), "Set memory limit in client/local server") - ("fuzzer-args", po::value(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.") - ("client_logs_file", po::value(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)") ; From 725640613b0d1cf47515697b5856a85953b73483 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Aug 2024 22:35:57 +0200 Subject: [PATCH 0591/1722] Add annotations --- .../0_stateless/00111_shard_external_sort_distributed.sql | 3 ++- .../0_stateless/00376_shard_group_uniq_array_of_int_array.sql | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql index 93efc317bfa..9e06654195d 100644 --- a/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql +++ b/tests/queries/0_stateless/00111_shard_external_sort_distributed.sql @@ -1,4 +1,5 @@ --- Tags: distributed, long +-- Tags: distributed, long, no-flaky-check +-- ^ no-flaky-check - sometimes longer than 600s with ThreadFuzzer. SET max_memory_usage = 150000000; SET max_bytes_before_external_sort = 10000000; diff --git a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql index 24b7f1c30a6..4453c26283c 100644 --- a/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql +++ b/tests/queries/0_stateless/00376_shard_group_uniq_array_of_int_array.sql @@ -1,4 +1,4 @@ --- Tags: shard +-- Tags: long SET max_rows_to_read = '55M'; From 1dece979fe317a04f98d2b8008619c47fb72edb1 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 7 Aug 2024 23:10:40 +0200 Subject: [PATCH 0592/1722] CI: pass job timout into tests --- tests/ci/ci.py | 1 + tests/ci/ci_definitions.py | 2 +- tests/ci/functional_test_check.py | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 8d0414ce7a8..49b597333dc 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -985,6 +985,7 @@ def _run_test(job_name: str, run_command: str) -> int: else: print("Use run command from the workflow") env["CHECK_NAME"] = job_name + env["MAX_RUN_TIME"] = str(timeout or 0) print(f"Going to start run command [{run_command}]") stopwatch = Stopwatch() job_log = Path(TEMP_PATH) / "job_log.txt" diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 48847b0d7a6..592cb2f4879 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -420,7 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=9000, + timeout=1000, # test ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 52970404d2d..3aff97643c3 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -114,6 +114,9 @@ def get_run_command( if flaky_check: envs.append("-e NUM_TRIES=50") envs.append("-e MAX_RUN_TIME=2800") + else: + max_run_time = os.getenv("MAX_RUN_TIME", 0) + envs.append(f"-e MAX_RUN_TIME={max_run_time}") envs += [f"-e {e}" for e in additional_envs] From 086b3d240dd696c483f136d79db4587a83bb0a14 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 7 Aug 2024 23:34:36 +0200 Subject: [PATCH 0593/1722] CI: push CI --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 592cb2f4879..b62d2e0aa8e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -420,7 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=1000, # test + timeout=1001, # test ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", From c6c0a44b93c382b384eb3ef83cf9da5102629de8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 7 Aug 2024 23:57:19 +0200 Subject: [PATCH 0594/1722] fix flaky tests --- .../test_storage_azure_blob_storage/test.py | 2 +- tests/integration/test_storage_hdfs/test.py | 51 +++++++------------ 2 files changed, 19 insertions(+), 34 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 092c124855c..fbdc7f29f98 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1272,7 +1272,7 @@ def test_filtering_by_file_or_path(cluster): node.query("SYSTEM FLUSH LOGS") result = node.query( - f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query ilike '%select%azure%test_filter%' AND type='QueryFinish'" + f"SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query ilike '%select%azure%test_filter%' AND type='QueryFinish' ORDER BY event_time_microseconds DESC LIMIT 1" ) assert int(result) == 1 diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 77921b885b0..c52e99b800e 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -41,7 +41,6 @@ def test_read_write_storage(started_cluster): node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" - node1.query("drop table if exists SimpleHDFSStorage") def test_read_write_storage_with_globs(started_cluster): @@ -95,11 +94,6 @@ def test_read_write_storage_with_globs(started_cluster): print(ex) assert "in readonly mode" in str(ex) - node1.query("DROP TABLE HDFSStorageWithRange") - node1.query("DROP TABLE HDFSStorageWithEnum") - node1.query("DROP TABLE HDFSStorageWithQuestionMark") - node1.query("DROP TABLE HDFSStorageWithAsterisk") - def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -341,7 +335,6 @@ def test_virtual_columns(started_cluster): ) == expected ) - node1.query("DROP TABLE virtual_cols") def test_read_files_with_spaces(started_cluster): @@ -363,7 +356,6 @@ def test_read_files_with_spaces(started_cluster): ) assert node1.query("select * from test order by id") == "1\n2\n3\n" fs.delete(dir, recursive=True) - node1.query("DROP TABLE test") def test_truncate_table(started_cluster): @@ -435,7 +427,7 @@ def test_seekable_formats(started_cluster): f"hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String')" ) node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" ) result = node1.query(f"SELECT count() FROM {table_function}") @@ -443,7 +435,7 @@ def test_seekable_formats(started_cluster): table_function = f"hdfs('hdfs://hdfs1:9000/orc', 'ORC', 'a Int32, b String')" node1.query( - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)" ) result = node1.query(f"SELECT count() FROM {table_function}") assert int(result) == 5000000 @@ -467,7 +459,7 @@ def test_read_table_with_default(started_cluster): def test_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000) SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/native', 'Native')") @@ -520,7 +512,6 @@ def test_hdfs_directory_not_exist(started_cluster): assert "" == node1.query( "select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1" ) - node1.query("DROP TABLE HDFSStorageWithNotExistDir") def test_overwrite(started_cluster): @@ -540,7 +531,6 @@ def test_overwrite(started_cluster): result = node1.query(f"select count() from test_overwrite") assert int(result) == 10 - node1.query(f"DROP TABLE test_overwrite") def test_multiple_inserts(started_cluster): @@ -577,7 +567,6 @@ def test_multiple_inserts(started_cluster): result = node1.query(f"select count() from test_multiple_inserts") assert int(result) == 60 - node1.query(f"DROP TABLE test_multiple_inserts") def test_format_detection(started_cluster): @@ -591,10 +580,10 @@ def test_format_detection(started_cluster): def test_schema_inference_with_globs(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data1.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0 SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data2.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select 0" ) result = node1.query( @@ -608,7 +597,7 @@ def test_schema_inference_with_globs(started_cluster): assert sorted(result.split()) == ["0", "\\N"] node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data3.jsoncompacteachrow', 'JSONCompactEachRow', 'x Nullable(UInt32)') select NULL" ) filename = "data{1,3}.jsoncompacteachrow" @@ -620,7 +609,7 @@ def test_schema_inference_with_globs(started_cluster): assert "All attempts to extract table structure from files failed" in result node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]' SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/data0.jsoncompacteachrow', 'TSV', 'x String') select '[123;]'" ) result = node1.query_and_get_error( @@ -632,7 +621,7 @@ def test_schema_inference_with_globs(started_cluster): def test_insert_select_schema_inference(started_cluster): node1.query( - f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x SETTINGS hdfs_truncate_on_insert=1" + f"insert into table function hdfs('hdfs://hdfs1:9000/test.native.zst') select toUInt64(1) as x" ) result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/test.native.zst')") @@ -675,9 +664,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_2', 'Parquet', 'a Int32, b String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "parquet_2" @@ -685,9 +672,7 @@ def test_virtual_columns_2(started_cluster): table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" ) - node1.query( - f"insert into table function {table_function} SELECT 1, 'kek' SETTINGS hdfs_truncate_on_insert=1" - ) + node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") assert result.strip() == "kek" @@ -984,11 +969,11 @@ def test_read_subcolumns(started_cluster): node = started_cluster.instances["node1"] node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" ) node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3) SETTINGS hdfs_truncate_on_insert=1" + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" ) res = node.query( @@ -1034,11 +1019,11 @@ def test_union_schema_inference_mode(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b" ) node.query("system drop schema cache for hdfs") @@ -1070,7 +1055,7 @@ def test_union_schema_inference_mode(started_cluster): ) assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" node.query( - f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error' SETTINGS hdfs_truncate_on_insert=1" + f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error'" ) error = node.query_and_get_error( @@ -1083,11 +1068,11 @@ def test_format_detection(started_cluster): node = started_cluster.instances["node1"] node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0) SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0)" ) node.query( - "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10) SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10)" ) expected_desc_result = node.query( @@ -1151,7 +1136,7 @@ def test_write_to_globbed_partitioned_path(started_cluster): node = started_cluster.instances["node1"] error = node.query_and_get_error( - "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42 SETTINGS hdfs_truncate_on_insert=1" + "insert into function hdfs('hdfs://hdfs1:9000/test_data_*_{_partition_id}.csv') partition by 42 select 42" ) assert "DATABASE_ACCESS_DENIED" in error From 55ad7d30946d609159fe5ae9156f02f5b160585a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 8 Aug 2024 00:08:12 +0200 Subject: [PATCH 0595/1722] Fix stylelint --- tests/ci/functional_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 3aff97643c3..b7391eff01b 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -115,7 +115,7 @@ def get_run_command( envs.append("-e NUM_TRIES=50") envs.append("-e MAX_RUN_TIME=2800") else: - max_run_time = os.getenv("MAX_RUN_TIME", 0) + max_run_time = os.getenv("MAX_RUN_TIME", "0") envs.append(f"-e MAX_RUN_TIME={max_run_time}") envs += [f"-e {e}" for e in additional_envs] From c19ee360d1a4cf0bc7607923505ba1e2a3848132 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 00:44:42 +0200 Subject: [PATCH 0596/1722] Update StorageObjectStorageSource.cpp --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 810bad4788b..d8e26977e75 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -208,7 +208,6 @@ Chunk StorageObjectStorageSource::generate() .filename = &filename, .last_modified = object_info->metadata->last_modified, .etag = &(object_info->metadata->etag) - .last_modified = object_info->metadata->last_modified, }, getContext(), read_from_format_info.columns_description); const auto & partition_columns = configuration->getPartitionColumns(); From 8426e0d5e5d7f102fd57a45c82ae6acccda65369 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 7 Aug 2024 16:44:10 +0800 Subject: [PATCH 0597/1722] fix crash --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 3 +-- src/Interpreters/HashJoin/JoinFeatures.h | 2 +- .../03006_join_on_inequal_expression_fast.reference | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 2bf5f6aef4a..5fefe53d145 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -361,7 +361,6 @@ size_t HashJoinMethods::joinRightColumns( } bool right_row_found = false; - KnownRowsHolder known_rows; for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) { @@ -693,7 +692,7 @@ size_t HashJoinMethods::joinRightColumnsWithAddti any_matched = true; if constexpr (join_features.is_semi_join || join_features.is_any_join) { - auto used_once = used_flags.template setUsedOnce( + auto used_once = used_flags.template setUsedOnce( selected_right_row_it->block, selected_right_row_it->row_num, 0); if (used_once) { diff --git a/src/Interpreters/HashJoin/JoinFeatures.h b/src/Interpreters/HashJoin/JoinFeatures.h index a530179f0b4..b8de606c51e 100644 --- a/src/Interpreters/HashJoin/JoinFeatures.h +++ b/src/Interpreters/HashJoin/JoinFeatures.h @@ -22,7 +22,7 @@ struct JoinFeatures static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); static constexpr bool add_missing = (left || full) && !is_semi_join; - static constexpr bool need_flags = MapGetter, HashJoin::MapsOne>>::flagged; + static constexpr bool need_flags = MapGetter, HashJoin::MapsAll>>::flagged; static constexpr bool is_maps_all = std::is_same_v, HashJoin::MapsAll>; }; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index 11ac01d24d5..aa8d4103db2 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -1,4 +1,3 @@ -03006_join_on_inequal_expression_fast.sql -- { echoOn } SET join_algorithm='hash'; SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); From 42aa967311a55d3da0e1230595b0e0ca9928e777 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 00:38:05 +0000 Subject: [PATCH 0598/1722] add profile events for merges --- src/Common/ProfileEvents.cpp | 25 +++++- .../Merges/AggregatingSortedTransform.h | 10 +++ .../Algorithms/AggregatingSortedAlgorithm.h | 2 + .../FinishAggregatingInOrderAlgorithm.cpp | 3 + .../FinishAggregatingInOrderAlgorithm.h | 5 ++ .../GraphiteRollupSortedAlgorithm.h | 2 + .../Merges/Algorithms/IMergingAlgorithm.h | 11 ++- .../IMergingAlgorithmWithSharedChunks.h | 2 + src/Processors/Merges/Algorithms/MergedData.h | 2 + .../Algorithms/MergingSortedAlgorithm.h | 2 +- .../Algorithms/SummingSortedAlgorithm.h | 2 + .../Merges/CollapsingSortedTransform.h | 10 +++ src/Processors/Merges/IMergingTransform.h | 35 +++++++- .../Merges/MergingSortedTransform.cpp | 26 ++---- .../Merges/MergingSortedTransform.h | 4 - .../Merges/ReplacingSortedTransform.h | 9 ++ .../Merges/SummingSortedTransform.h | 10 +++ .../Merges/VersionedCollapsingTransform.h | 9 ++ .../Transforms/ColumnGathererTransform.cpp | 57 ++++++------- .../Transforms/ColumnGathererTransform.h | 11 ++- .../Transforms/MergeJoinTransform.cpp | 12 ++- .../Transforms/MergeJoinTransform.h | 2 + .../Transforms/MergeSortingTransform.cpp | 2 - .../Transforms/PasteJoinTransform.cpp | 10 +++ .../Transforms/PasteJoinTransform.h | 3 +- .../gtest_blocks_size_merging_streams.cpp | 4 +- src/Storages/MergeTree/MergeList.h | 1 + src/Storages/MergeTree/MergeProgress.h | 27 +++--- src/Storages/MergeTree/MergeTask.cpp | 84 +++++++++++++++---- src/Storages/MergeTree/MergeTask.h | 20 ++++- src/Storages/MergeTree/MutateTask.cpp | 10 ++- 31 files changed, 308 insertions(+), 104 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ccdce7ff584..857a08d8a5d 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -210,7 +210,29 @@ M(Merge, "Number of launched background merges.") \ M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \ M(MergedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for background merges. This is the number before merge.") \ - M(MergesTimeMilliseconds, "Total time spent for background merges.")\ + M(MergeTotalMilliseconds, "Total time spent for background merges") \ + M(MergeExecuteMilliseconds, "Total busy time spent for execution of background merges") \ + M(MergeHorizontalStageTotalMilliseconds, "Total time spent for horizontal stage of background merges") \ + M(MergeHorizontalStageExecuteMilliseconds, "Total busy time spent for execution of horizontal stage of background merges") \ + M(MergeVerticalStageTotalMilliseconds, "Total time spent for vertical stage of background merges") \ + M(MergeVerticalStageExecuteMilliseconds, "Total busy time spent for execution of vertical stage of background merges") \ + M(MergeProjectionStageTotalMilliseconds, "Total time spent for projection stage of background merges") \ + M(MergeProjectionStageExecuteMilliseconds, "Total busy time spent for execution of projection stage of background merges") \ + \ + M(MergingSortedMilliseconds, "Total time spent while merging sorted columns") \ + M(AggregatingSortedMilliseconds, "Total time spent while aggregating sorted columns") \ + M(CollapsingSortedMilliseconds, "Total time spent while collapsing sorted columns") \ + M(ReplacingSortedMilliseconds, "Total time spent while replacing sorted columns") \ + M(SummingSortedMilliseconds, "Total time spent while summing sorted columns") \ + M(VersionedCollapsingSortedMilliseconds, "Total time spent while version collapsing sorted columns") \ + M(GatheringColumnMilliseconds, "Total time spent while gathering columns for vertical merge") \ + \ + M(MutationTotalParts, "Number of total parts for which mutations tried to be applied") \ + M(MutationUntouchedParts, "Number of total parts for which mutations tried to be applied but which was completely skipped according to predicate") \ + M(MutatedRows, "Rows read for mutations. This is the number of rows before mutation") \ + M(MutatedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for mutations. This is the number before mutation.") \ + M(MutationTimeMilliseconds, "Total time spent for mutations.") \ + M(MutateTaskProjectionsCalculationMicroseconds, "Time spent calculating projections") \ \ M(MergeTreeDataWriterRows, "Number of rows INSERTed to MergeTree tables.") \ M(MergeTreeDataWriterUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) INSERTed to MergeTree tables.") \ @@ -225,7 +247,6 @@ M(MergeTreeDataWriterProjectionsCalculationMicroseconds, "Time spent calculating projections") \ M(MergeTreeDataProjectionWriterSortingBlocksMicroseconds, "Time spent sorting blocks (for projection it might be a key different from table's sorting key)") \ M(MergeTreeDataProjectionWriterMergingBlocksMicroseconds, "Time spent merging blocks") \ - M(MutateTaskProjectionsCalculationMicroseconds, "Time spent calculating projections") \ \ M(InsertedWideParts, "Number of parts inserted in Wide format.") \ M(InsertedCompactParts, "Number of parts inserted in Compact format.") \ diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index c6d7e844c65..c96ad3db525 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -3,6 +3,11 @@ #include #include +namespace ProfileEvents +{ + extern const Event AggregatingSortedMilliseconds; +} + namespace DB { @@ -29,6 +34,11 @@ public: } String getName() const override { return "AggregatingSortedTransform"; } + + void onFinish() override + { + logMergedStats(ProfileEvents::AggregatingSortedMilliseconds, "Aggregated sorted", getLogger("AggregatingSortedTransform")); + } }; } diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h index 53c103e7038..908994e1851 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h @@ -30,6 +30,8 @@ public: void consume(Input & input, size_t source_num) override; Status merge() override; + MergedStats getMergedStats() const override { return merged_data.getMergedStats(); } + /// Stores information for aggregation of SimpleAggregateFunction columns struct SimpleAggregateDescription { diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index 86675bcb237..477566d8a94 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -126,6 +126,9 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() { + total_merged_rows += accumulated_rows; + total_merged_bytes += accumulated_bytes; + accumulated_rows = 0; accumulated_bytes = 0; diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index cc6578e79be..39171c5a978 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -50,6 +50,8 @@ public: void consume(Input & input, size_t source_num) override; Status merge() override; + MergedStats getMergedStats() const override { return {.bytes = accumulated_bytes, .rows = accumulated_rows, .blocks = chunk_num}; } + private: Chunk prepareToMerge(); void addToAggregation(); @@ -92,6 +94,9 @@ private: UInt64 chunk_num = 0; size_t accumulated_rows = 0; size_t accumulated_bytes = 0; + + size_t total_merged_rows = 0; + size_t total_merged_bytes = 0; }; } diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index aaa3859efb6..cb2775c968d 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -33,6 +33,8 @@ public: const char * getName() const override { return "GraphiteRollupSortedAlgorithm"; } Status merge() override; + MergedStats getMergedStats() const override { return merged_data->getMergedStats(); } + struct ColumnsDefinition { size_t path_column_num; diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithm.h b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h index 9a1c7c24270..83f11232b71 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { @@ -65,6 +65,15 @@ public: IMergingAlgorithm() = default; virtual ~IMergingAlgorithm() = default; + + struct MergedStats + { + UInt64 bytes = 0; + UInt64 rows = 0; + UInt64 blocks = 0; + }; + + virtual MergedStats getMergedStats() const = 0; }; // TODO: use when compile with clang which could support it diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h index bc1aafe93f7..1725108ac5d 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h @@ -16,6 +16,8 @@ public: void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; + MergedStats getMergedStats() const override { return merged_data->getMergedStats(); } + private: Block header; SortDescription description; diff --git a/src/Processors/Merges/Algorithms/MergedData.h b/src/Processors/Merges/Algorithms/MergedData.h index c5bb074bb0c..8f47f89d8ee 100644 --- a/src/Processors/Merges/Algorithms/MergedData.h +++ b/src/Processors/Merges/Algorithms/MergedData.h @@ -183,6 +183,8 @@ public: UInt64 totalAllocatedBytes() const { return total_allocated_bytes; } UInt64 maxBlockSize() const { return max_block_size; } + IMergingAlgorithm::MergedStats getMergedStats() const { return {.bytes = total_allocated_bytes, .rows = total_merged_rows, .blocks = total_chunks}; } + virtual ~MergedData() = default; protected: diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index bcb111baadf..c889668a38e 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -31,7 +31,7 @@ public: void consume(Input & input, size_t source_num) override; Status merge() override; - const MergedData & getMergedData() const { return merged_data; } + MergedStats getMergedStats() const override { return merged_data.getMergedStats(); } private: Block header; diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index 664b171c4b9..74b4e397831 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -30,6 +30,8 @@ public: void consume(Input & input, size_t source_num) override; Status merge() override; + MergedStats getMergedStats() const override { return merged_data.getMergedStats(); } + struct AggregateDescription; struct MapDescription; diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 4479ac82f66..99fb700abf1 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -3,6 +3,11 @@ #include #include +namespace ProfileEvents +{ + extern const Event CollapsingSortedMilliseconds; +} + namespace DB { @@ -36,6 +41,11 @@ public: } String getName() const override { return "CollapsingSortedTransform"; } + + void onFinish() override + { + logMergedStats(ProfileEvents::CollapsingSortedMilliseconds, "Collapsed sorted", getLogger("CollapsingSortedTransform")); + } }; } diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index be629271736..fba5b038618 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -2,7 +2,10 @@ #include #include +#include #include +#include +#include namespace DB { @@ -110,6 +113,8 @@ public: void work() override { + Stopwatch watch; + if (!state.init_chunks.empty()) algorithm.initialize(std::move(state.init_chunks)); @@ -147,6 +152,8 @@ public: // std::cerr << "Finished" << std::endl; state.is_finished = true; } + + merging_elapsed_ns += watch.elapsedNanoseconds(); } protected: @@ -156,7 +163,33 @@ protected: Algorithm algorithm; /// Profile info. - Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; + UInt64 merging_elapsed_ns = 0; + + void logMergedStats(ProfileEvents::Event elapsed_ms_event, std::string_view transform_message, LoggerPtr log) const + { + auto stats = algorithm.getMergedStats(); + + UInt64 elapsed_ms = merging_elapsed_ns / 1000000LL; + ProfileEvents::increment(elapsed_ms_event, elapsed_ms); + + /// Don't print info for small parts (< 1M rows) + if (stats.rows < 1000000) + return; + + double seconds = static_cast(merging_elapsed_ns) / 1000000000ULL; + + if (seconds == 0.0) + { + LOG_DEBUG(log, "{}: {} blocks, {} rows, {} bytes in 0 sec.", + transform_message, stats.blocks, stats.rows, stats.bytes); + } + else + { + LOG_DEBUG(log, "{}: {} blocks, {} rows, {} bytes in {} sec., {} rows/sec., {}/sec.", + transform_message, stats.blocks, stats.rows, stats.bytes, + seconds, stats.rows / seconds, ReadableSize(stats.bytes / seconds)); + } + } private: using IMergingTransformBase::state; diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 338b1ff7935..d2895a2a2e9 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -1,9 +1,12 @@ #include #include #include - #include -#include + +namespace ProfileEvents +{ + extern const Event MergingSortedMilliseconds; +} namespace DB { @@ -18,7 +21,6 @@ MergingSortedTransform::MergingSortedTransform( UInt64 limit_, bool always_read_till_end_, WriteBuffer * out_row_sources_buf_, - bool quiet_, bool use_average_block_sizes, bool have_all_inputs_) : IMergingTransform( @@ -37,7 +39,6 @@ MergingSortedTransform::MergingSortedTransform( limit_, out_row_sources_buf_, use_average_block_sizes) - , quiet(quiet_) { } @@ -48,22 +49,7 @@ void MergingSortedTransform::onNewInput() void MergingSortedTransform::onFinish() { - if (quiet) - return; - - const auto & merged_data = algorithm.getMergedData(); - - auto log = getLogger("MergingSortedTransform"); - - double seconds = total_stopwatch.elapsedSeconds(); - - if (seconds == 0.0) - LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in 0 sec.", merged_data.totalChunks(), merged_data.totalMergedRows()); - else - LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in {} sec., {} rows/sec., {}/sec", - merged_data.totalChunks(), merged_data.totalMergedRows(), seconds, - merged_data.totalMergedRows() / seconds, - ReadableSize(merged_data.totalAllocatedBytes() / seconds)); + logMergedStats(ProfileEvents::MergingSortedMilliseconds, "Merged sorted", getLogger("MergingSortedTransform")); } } diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 2b53939f309..6e52450efa7 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -21,7 +21,6 @@ public: UInt64 limit_ = 0, bool always_read_till_end_ = false, WriteBuffer * out_row_sources_buf_ = nullptr, - bool quiet_ = false, bool use_average_block_sizes = false, bool have_all_inputs_ = true); @@ -30,9 +29,6 @@ public: protected: void onNewInput() override; void onFinish() override; - -private: - bool quiet = false; }; } diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index 2657987f161..dc262aab9ee 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -3,6 +3,10 @@ #include #include +namespace ProfileEvents +{ + extern const Event ReplacingSortedMilliseconds; +} namespace DB { @@ -38,6 +42,11 @@ public: } String getName() const override { return "ReplacingSorted"; } + + void onFinish() override + { + logMergedStats(ProfileEvents::ReplacingSortedMilliseconds, "Replaced sorted", getLogger("ReplacingSortedTransform")); + } }; } diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 70ddebfea95..d7c20223d7e 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -3,6 +3,11 @@ #include #include +namespace ProfileEvents +{ + extern const Event SummingSortedMilliseconds; +} + namespace DB { @@ -33,6 +38,11 @@ public: } String getName() const override { return "SummingSortedTransform"; } + + void onFinish() override + { + logMergedStats(ProfileEvents::SummingSortedMilliseconds, "Summed sorted", getLogger("SummingSortedTransform")); + } }; } diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index 18244469bd7..32b5d7bf343 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -3,6 +3,10 @@ #include #include +namespace ProfileEvents +{ + extern const Event VersionedCollapsingSortedMilliseconds; +} namespace DB { @@ -33,6 +37,11 @@ public: } String getName() const override { return "VersionedCollapsingTransform"; } + + void onFinish() override + { + logMergedStats(ProfileEvents::VersionedCollapsingSortedMilliseconds, "Versioned collapsed sorted", getLogger("VersionedCollapsingTransform")); + } }; } diff --git a/src/Processors/Transforms/ColumnGathererTransform.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp index 15f8355bdc7..52fa42fdb51 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -1,11 +1,15 @@ #include +#include #include #include #include #include #include -#include +namespace ProfileEvents +{ + extern const Event GatheringColumnMilliseconds; +} namespace DB { @@ -33,6 +37,13 @@ ColumnGathererStream::ColumnGathererStream( throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "There are no streams to gather"); } +void ColumnGathererStream::updateStats(const IColumn & column) +{ + merged_rows += column.size(); + merged_bytes += column.allocatedBytes(); + ++merged_blocks; +} + void ColumnGathererStream::initialize(Inputs inputs) { Columns source_columns; @@ -82,7 +93,9 @@ IMergingAlgorithm::Status ColumnGathererStream::merge() { res.addColumn(source_to_fully_copy->column); } - merged_rows += source_to_fully_copy->size; + + updateStats(*source_to_fully_copy->column); + source_to_fully_copy->pos = source_to_fully_copy->size; source_to_fully_copy = nullptr; return Status(std::move(res)); @@ -96,8 +109,7 @@ IMergingAlgorithm::Status ColumnGathererStream::merge() { next_required_source = 0; Chunk res; - merged_rows += sources.front().column->size(); - merged_bytes += sources.front().column->allocatedBytes(); + updateStats(*sources.front().column); res.addColumn(std::move(sources.front().column)); sources.front().pos = sources.front().size = 0; return Status(std::move(res)); @@ -123,8 +135,8 @@ IMergingAlgorithm::Status ColumnGathererStream::merge() if (source_to_fully_copy && result_column->empty()) { Chunk res; - merged_rows += source_to_fully_copy->column->size(); - merged_bytes += source_to_fully_copy->column->allocatedBytes(); + updateStats(*source_to_fully_copy->column); + if (result_column->hasDynamicStructure()) { auto col = result_column->cloneEmpty(); @@ -140,13 +152,13 @@ IMergingAlgorithm::Status ColumnGathererStream::merge() return Status(std::move(res)); } - auto col = result_column->cloneEmpty(); - result_column.swap(col); + auto return_column = result_column->cloneEmpty(); + result_column.swap(return_column); Chunk res; - merged_rows += col->size(); - merged_bytes += col->allocatedBytes(); - res.addColumn(std::move(col)); + updateStats(*return_column); + + res.addColumn(std::move(return_column)); return Status(std::move(res), row_sources_buf.eof() && !source_to_fully_copy); } @@ -185,31 +197,10 @@ ColumnGathererTransform::ColumnGathererTransform( toString(header.columns())); } -void ColumnGathererTransform::work() -{ - Stopwatch stopwatch; - IMergingTransform::work(); - elapsed_ns += stopwatch.elapsedNanoseconds(); -} - void ColumnGathererTransform::onFinish() { - auto merged_rows = algorithm.getMergedRows(); - auto merged_bytes = algorithm.getMergedRows(); - /// Don't print info for small parts (< 10M rows) - if (merged_rows < 10000000) - return; - - double seconds = static_cast(elapsed_ns) / 1000000000ULL; const auto & column_name = getOutputPort().getHeader().getByPosition(0).name; - - if (seconds == 0.0) - LOG_DEBUG(log, "Gathered column {} ({} bytes/elem.) in 0 sec.", - column_name, static_cast(merged_bytes) / merged_rows); - else - LOG_DEBUG(log, "Gathered column {} ({} bytes/elem.) in {} sec., {} rows/sec., {}/sec.", - column_name, static_cast(merged_bytes) / merged_rows, seconds, - merged_rows / seconds, ReadableSize(merged_bytes / seconds)); + logMergedStats(ProfileEvents::GatheringColumnMilliseconds, fmt::format("Gathered column {}", column_name), log); } } diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index ec5691316ce..a535b2669d0 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -2,6 +2,7 @@ #include #include +#include "base/types.h" #include #include @@ -72,10 +73,11 @@ public: template void gather(Column & column_res); - UInt64 getMergedRows() const { return merged_rows; } - UInt64 getMergedBytes() const { return merged_bytes; } + MergedStats getMergedStats() const override { return {.bytes = merged_bytes, .rows = merged_rows, .blocks = merged_blocks}; } private: + void updateStats(const IColumn & column); + /// Cache required fields struct Source { @@ -105,6 +107,7 @@ private: ssize_t next_required_source = -1; UInt64 merged_rows = 0; UInt64 merged_bytes = 0; + UInt64 merged_blocks = 0; }; class ColumnGathererTransform final : public IMergingTransform @@ -120,12 +123,8 @@ public: String getName() const override { return "ColumnGathererTransform"; } - void work() override; - protected: void onFinish() override; - UInt64 elapsed_ns = 0; - LoggerPtr log; }; diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index e96a75d277b..26601207da8 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -511,6 +511,16 @@ void MergeJoinAlgorithm::logElapsed(double seconds) stat.max_blocks_loaded); } +IMergingAlgorithm::MergedStats MergeJoinAlgorithm::getMergedStats() const +{ + return + { + .bytes = 0, + .rows = stat.num_rows[0] + stat.num_rows[1], + .blocks = stat.num_blocks[0] + stat.num_blocks[1], + }; +} + static void prepareChunk(Chunk & chunk) { if (!chunk) @@ -1271,7 +1281,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - algorithm.logElapsed(total_stopwatch.elapsedSeconds()); + algorithm.logElapsed(merging_elapsed_ns / 1000000000ULL); } } diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index d37a0b9f3ae..841a3f15a92 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -245,6 +245,8 @@ public: void setAsofInequality(ASOFJoinInequality asof_inequality_); void logElapsed(double seconds); + MergedStats getMergedStats() const override; + private: std::optional handleAnyJoinState(); Status anyJoin(); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index ede13b29219..c45192e7118 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -185,7 +185,6 @@ void MergeSortingTransform::consume(Chunk chunk) if (!external_merging_sorted) { - bool quiet = false; bool have_all_inputs = false; bool use_average_block_sizes = false; @@ -199,7 +198,6 @@ void MergeSortingTransform::consume(Chunk chunk) limit, /*always_read_till_end_=*/ false, nullptr, - quiet, use_average_block_sizes, have_all_inputs); diff --git a/src/Processors/Transforms/PasteJoinTransform.cpp b/src/Processors/Transforms/PasteJoinTransform.cpp index d2fa7eed256..ad01b721726 100644 --- a/src/Processors/Transforms/PasteJoinTransform.cpp +++ b/src/Processors/Transforms/PasteJoinTransform.cpp @@ -58,6 +58,16 @@ static void prepareChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } +IMergingAlgorithm::MergedStats PasteJoinAlgorithm::getMergedStats() const +{ + return + { + .bytes = 0, + .rows = stat.num_rows[0] + stat.num_rows[1], + .blocks = stat.num_blocks[0] + stat.num_blocks[1], + }; +} + void PasteJoinAlgorithm::initialize(Inputs inputs) { if (inputs.size() != 2) diff --git a/src/Processors/Transforms/PasteJoinTransform.h b/src/Processors/Transforms/PasteJoinTransform.h index 6a7e65ee27c..fbe85f6993b 100644 --- a/src/Processors/Transforms/PasteJoinTransform.h +++ b/src/Processors/Transforms/PasteJoinTransform.h @@ -35,8 +35,7 @@ public: void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; Status merge() override; - - void logElapsed(double seconds); + MergedStats getMergedStats() const override; private: Chunk createBlockWithDefaults(size_t source_num); diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index bc22f249f97..f41a447049c 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -83,7 +83,7 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - 8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, false, true); + 8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, true); pipe.addTransform(std::move(transform)); @@ -125,7 +125,7 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - 8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, false, true); + 8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, true); pipe.addTransform(std::move(transform)); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index d40af6abf43..3a96ba0abae 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -6,6 +6,7 @@ #include #include #include +#include "base/types.h" #include #include #include diff --git a/src/Storages/MergeTree/MergeProgress.h b/src/Storages/MergeTree/MergeProgress.h index dd4922051b5..8562e81e761 100644 --- a/src/Storages/MergeTree/MergeProgress.h +++ b/src/Storages/MergeTree/MergeProgress.h @@ -8,10 +8,10 @@ namespace ProfileEvents { - extern const Event MergesTimeMilliseconds; extern const Event MergedUncompressedBytes; extern const Event MergedRows; - extern const Event Merge; + extern const Event MutatedRows; + extern const Event MutatedUncompressedBytes; } namespace DB @@ -63,18 +63,17 @@ public: void updateWatch() { UInt64 watch_curr_elapsed = merge_list_element_ptr->watch.elapsed(); - ProfileEvents::increment(ProfileEvents::MergesTimeMilliseconds, (watch_curr_elapsed - watch_prev_elapsed) / 1000000); watch_prev_elapsed = watch_curr_elapsed; } - void operator() (const Progress & value) + void operator()(const Progress & value) { - ProfileEvents::increment(ProfileEvents::MergedUncompressedBytes, value.read_bytes); - if (stage.is_first) - { - ProfileEvents::increment(ProfileEvents::MergedRows, value.read_rows); - ProfileEvents::increment(ProfileEvents::Merge); - } + if (merge_list_element_ptr->is_mutation) + updateProfileEvents(value, ProfileEvents::MutatedRows, ProfileEvents::MutatedUncompressedBytes); + else + updateProfileEvents(value, ProfileEvents::MergedRows, ProfileEvents::MergedUncompressedBytes); + + updateWatch(); merge_list_element_ptr->bytes_read_uncompressed += value.read_bytes; @@ -90,6 +89,14 @@ public: std::memory_order_relaxed); } } + +private: + void updateProfileEvents(const Progress & value, ProfileEvents::Event rows_event, ProfileEvents::Event bytes_event) const + { + ProfileEvents::increment(bytes_event, value.read_bytes); + if (stage.is_first) + ProfileEvents::increment(rows_event, value.read_rows); + } }; } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ce06adf110c..5f178f08ec3 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -5,9 +5,13 @@ #include #include +#include "Common/ElapsedTimeProfileEventIncrement.h" +#include "Common/Logger.h" +#include "Common/Stopwatch.h" #include #include #include +#include #include #include #include @@ -39,6 +43,16 @@ #include #include +namespace ProfileEvents +{ + extern const Event Merge; + extern const Event MergeTotalMilliseconds; + extern const Event MergeExecuteMilliseconds; + extern const Event MergeHorizontalStageExecuteMilliseconds; + extern const Event MergeVerticalStageExecuteMilliseconds; + extern const Event MergeProjectionStageExecuteMilliseconds; +} + namespace DB { @@ -186,6 +200,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (isTTLMergeType(global_ctx->future_part->merge_type) && global_ctx->ttl_merges_blocker->isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with TTL"); + ProfileEvents::increment(ProfileEvents::Merge); + LOG_DEBUG(ctx->log, "Merging {} parts: from {} to {} into {} with storage {}", global_ctx->future_part->parts.size(), global_ctx->future_part->parts.front()->name, @@ -446,6 +462,9 @@ void MergeTask::addGatheringColumn(GlobalRuntimeContextPtr global_ctx, const Str MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::getContextForNextStage() { + ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + ProfileEvents::increment(ProfileEvents::MergeHorizontalStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + auto new_ctx = std::make_shared(); new_ctx->rows_sources_write_buf = std::move(ctx->rows_sources_write_buf); @@ -463,8 +482,10 @@ MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::g MergeTask::StageRuntimeContextPtr MergeTask::VerticalMergeStage::getContextForNextStage() { - auto new_ctx = std::make_shared(); + ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + ProfileEvents::increment(ProfileEvents::MergeVerticalStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + auto new_ctx = std::make_shared(); new_ctx->need_sync = std::move(ctx->need_sync); ctx.reset(); @@ -474,9 +495,14 @@ MergeTask::StageRuntimeContextPtr MergeTask::VerticalMergeStage::getContextForNe bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() { - assert(subtasks_iterator != subtasks.end()); - if ((this->**subtasks_iterator)()) - return true; + chassert(subtasks_iterator != subtasks.end()); + + Stopwatch watch; + bool res = (this->**subtasks_iterator)(); + ctx->elapsed_execute_ns += watch.elapsedNanoseconds(); + + if (res) + return res; /// Move to the next subtask in an array of subtasks ++subtasks_iterator; @@ -534,7 +560,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const { - /// No need to execute this part if it is horizontal merge. + /// No need to execute this part if it is horizontal merge. if (global_ctx->chosen_merge_algorithm != MergeAlgorithm::Vertical) return false; @@ -906,12 +932,24 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const return false; } +MergeTask::StageRuntimeContextPtr MergeTask::MergeProjectionsStage::getContextForNextStage() +{ + ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + ProfileEvents::increment(ProfileEvents::MergeProjectionStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + + return nullptr; +} bool MergeTask::VerticalMergeStage::execute() { - assert(subtasks_iterator != subtasks.end()); - if ((this->**subtasks_iterator)()) - return true; + chassert(subtasks_iterator != subtasks.end()); + + Stopwatch watch; + bool res = (this->**subtasks_iterator)(); + ctx->elapsed_execute_ns += watch.elapsedNanoseconds(); + + if (res) + return res; /// Move to the next subtask in an array of subtasks ++subtasks_iterator; @@ -920,9 +958,14 @@ bool MergeTask::VerticalMergeStage::execute() bool MergeTask::MergeProjectionsStage::execute() { - assert(subtasks_iterator != subtasks.end()); - if ((this->**subtasks_iterator)()) - return true; + chassert(subtasks_iterator != subtasks.end()); + + Stopwatch watch; + bool res = (this->**subtasks_iterator)(); + ctx->elapsed_execute_ns += watch.elapsedNanoseconds(); + + if (res) + return res; /// Move to the next subtask in an array of subtasks ++subtasks_iterator; @@ -969,12 +1012,22 @@ bool MergeTask::VerticalMergeStage::executeVerticalMergeForAllColumns() const bool MergeTask::execute() { - assert(stages_iterator != stages.end()); - if ((*stages_iterator)->execute()) + chassert(stages_iterator != stages.end()); + const auto & current_stage = *stages_iterator; + + if (current_stage->execute()) return true; - /// Stage is finished, need initialize context for the next stage - auto next_stage_context = (*stages_iterator)->getContextForNextStage(); + /// Stage is finished, need to initialize context for the next stage and update profile events. + + UInt64 current_elapsed_ms = global_ctx->merge_list_element_ptr->watch.elapsedMilliseconds(); + UInt64 stage_elapsed_ms = current_elapsed_ms - global_ctx->prev_elapesed_ms; + global_ctx->prev_elapesed_ms = current_elapsed_ms; + + ProfileEvents::increment(current_stage->getTotalTimeProfileEvent(), stage_elapsed_ms); + ProfileEvents::increment(ProfileEvents::MergeTotalMilliseconds, stage_elapsed_ms); + + auto next_stage_context = current_stage->getContextForNextStage(); /// Move to the next stage in an array of stages ++stages_iterator; @@ -1099,7 +1152,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() /* limit_= */0, /* always_read_till_end_= */false, ctx->rows_sources_write_buf.get(), - true, ctx->blocks_are_granules_size); break; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 8b0f2130e8e..979c85482e5 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -26,6 +27,12 @@ #include #include +namespace ProfileEvents +{ + extern const Event MergeHorizontalStageTotalMilliseconds; + extern const Event MergeVerticalStageTotalMilliseconds; + extern const Event MergeProjectionStageTotalMilliseconds; +} namespace DB { @@ -134,6 +141,7 @@ private: { virtual void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) = 0; virtual StageRuntimeContextPtr getContextForNextStage() = 0; + virtual ProfileEvents::Event getTotalTimeProfileEvent() const = 0; virtual bool execute() = 0; virtual ~IStage() = default; }; @@ -195,6 +203,7 @@ private: bool need_prefix; scope_guard temporary_directory_lock; + UInt64 prev_elapesed_ms{0}; }; using GlobalRuntimeContextPtr = std::shared_ptr; @@ -233,6 +242,7 @@ private: /// Dependencies for next stages std::list::const_iterator it_name_and_type; bool need_sync{false}; + UInt64 elapsed_execute_ns{0}; }; using ExecuteAndFinalizeHorizontalPartRuntimeContextPtr = std::shared_ptr; @@ -256,7 +266,6 @@ private: ExecuteAndFinalizeHorizontalPartSubtasks::const_iterator subtasks_iterator = subtasks.begin(); - MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream(); void extractMergingAndGatheringColumns() const; @@ -268,6 +277,7 @@ private: } StageRuntimeContextPtr getContextForNextStage() override; + ProfileEvents::Event getTotalTimeProfileEvent() const override { return ProfileEvents::MergeHorizontalStageTotalMilliseconds; } ExecuteAndFinalizeHorizontalPartRuntimeContextPtr ctx; GlobalRuntimeContextPtr global_ctx; @@ -307,6 +317,7 @@ private: QueryPipeline column_parts_pipeline; std::unique_ptr executor; std::unique_ptr rows_sources_read_buf{nullptr}; + UInt64 elapsed_execute_ns{0}; }; using VerticalMergeRuntimeContextPtr = std::shared_ptr; @@ -321,6 +332,7 @@ private: global_ctx = static_pointer_cast(global); } StageRuntimeContextPtr getContextForNextStage() override; + ProfileEvents::Event getTotalTimeProfileEvent() const override { return ProfileEvents::MergeVerticalStageTotalMilliseconds; } bool prepareVerticalMergeForAllColumns() const; bool executeVerticalMergeForAllColumns() const; @@ -361,6 +373,7 @@ private: MergeTasks::iterator projections_iterator; LoggerPtr log{getLogger("MergeTask::MergeProjectionsStage")}; + UInt64 elapsed_execute_ns{0}; }; using MergeProjectionsRuntimeContextPtr = std::shared_ptr; @@ -368,12 +381,15 @@ private: struct MergeProjectionsStage : public IStage { bool execute() override; + void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override { ctx = static_pointer_cast(local); global_ctx = static_pointer_cast(global); } - StageRuntimeContextPtr getContextForNextStage() override { return nullptr; } + + StageRuntimeContextPtr getContextForNextStage() override; + ProfileEvents::Event getTotalTimeProfileEvent() const override { return ProfileEvents::MergeProjectionStageTotalMilliseconds; } bool mergeMinMaxIndexAndPrepareProjections() const; bool executeProjections() const; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9a775db73e2..fe78964a241 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -38,7 +38,10 @@ namespace ProfileEvents { -extern const Event MutateTaskProjectionsCalculationMicroseconds; + extern const Event MutationTotalParts; + extern const Event MutationUntouchedParts; + extern const Event MutationTimeMilliseconds; + extern const Event MutateTaskProjectionsCalculationMicroseconds; } namespace CurrentMetrics @@ -2034,6 +2037,9 @@ bool MutateTask::execute() if (task->executeStep()) return true; + auto total_elapsed_ms = (*ctx->mutate_entry)->watch.elapsedMilliseconds(); + ProfileEvents::increment(ProfileEvents::MutationTimeMilliseconds, total_elapsed_ms); + // The `new_data_part` is a shared pointer and must be moved to allow // part deletion in case it is needed in `MutateFromLogEntryTask::finalize`. // @@ -2118,6 +2124,7 @@ bool MutateTask::prepare() throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to mutate {} parts, not one. " "This is a bug.", ctx->future_part->parts.size()); + ProfileEvents::increment(ProfileEvents::MutationTotalParts); ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); auto context_for_reading = Context::createCopy(ctx->context); @@ -2174,6 +2181,7 @@ bool MutateTask::prepare() ctx->temporary_directory_lock = std::move(lock); } + ProfileEvents::increment(ProfileEvents::MutationUntouchedParts); promise.set_value(std::move(part)); return false; } From 0882b810a753db4ed7643a2fed3f94fea785ae4d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 01:04:37 +0000 Subject: [PATCH 0599/1722] try to make bigger steps for execution of merges --- src/Storages/MergeTree/MergeTask.cpp | 54 ++++++++++++++-------- src/Storages/MergeTree/MergeTask.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + 3 files changed, 37 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5f178f08ec3..3203120c6eb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "base/types.h" #include #include #include @@ -512,11 +513,20 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { - Block block; - if (!ctx->is_cancelled() && (global_ctx->merging_executor->pull(block))) - { - global_ctx->rows_written += block.rows(); + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + do + { + Block block; + + if (ctx->is_cancelled() || !global_ctx->merging_executor->pull(block)) + { + finalize(); + return false; + } + + global_ctx->rows_written += block.rows(); const_cast(*global_ctx->to).write(block); UInt64 result_rows = 0; @@ -536,11 +546,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() global_ctx->space_reservation->update(static_cast((1. - progress) * ctx->initial_reservation)); } + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } + /// Need execute again + return true; +} +void MergeTask::ExecuteAndFinalizeHorizontalPart::finalize() const +{ global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -550,14 +563,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with expired TTL"); - const auto data_settings = global_ctx->data->getSettings(); const size_t sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; - ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); - - return false; + ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *global_ctx->data->getSettings()); } - bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const { /// No need to execute this part if it is horizontal merge. @@ -734,17 +743,24 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const bool MergeTask::VerticalMergeStage::executeVerticalMergeForOneColumn() const { - Block block; - if (!global_ctx->merges_blocker->isCancelled() && !global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed) - && ctx->executor->pull(block)) + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + + do { + Block block; + + if (global_ctx->merges_blocker->isCancelled() + || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed) + || !ctx->executor->pull(block)) + return false; + ctx->column_elems_written += block.rows(); ctx->column_to->write(block); + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } - return false; + /// Need execute again + return true; } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 979c85482e5..24917a4cb0e 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -254,6 +254,7 @@ private: bool prepare(); bool executeImpl(); + void finalize() const; /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5ba1988cc5d..e56bb15f7b0 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,6 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ + M(Milliseconds, merge_preferred_step_execution_time_ms, 100, "Target time to execetion of one step of merge. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From 142e9528f0f1f7daa68f296181f1c0e5f7b6108b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 03:10:45 +0200 Subject: [PATCH 0600/1722] Add a test for #57420 --- .../03218_materialize_msan.reference | 1 + .../0_stateless/03218_materialize_msan.sql | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03218_materialize_msan.reference create mode 100644 tests/queries/0_stateless/03218_materialize_msan.sql diff --git a/tests/queries/0_stateless/03218_materialize_msan.reference b/tests/queries/0_stateless/03218_materialize_msan.reference new file mode 100644 index 00000000000..eb84f35f9f4 --- /dev/null +++ b/tests/queries/0_stateless/03218_materialize_msan.reference @@ -0,0 +1 @@ +[(NULL,'11\01111111\011111','1111')] -2147483648 \N diff --git a/tests/queries/0_stateless/03218_materialize_msan.sql b/tests/queries/0_stateless/03218_materialize_msan.sql new file mode 100644 index 00000000000..b41300ea1e3 --- /dev/null +++ b/tests/queries/0_stateless/03218_materialize_msan.sql @@ -0,0 +1,21 @@ +SELECT + materialize([(NULL, '11\01111111\011111', '1111')]) AS t, + (t[1048576]).2, + materialize(-2147483648), + (t[-2147483648]).1 +GROUP BY + materialize([(NULL, '1')]), + '', + (materialize((t[1023]).2), (materialize(''), (t[2147483647]).1, materialize(9223372036854775807)), (materialize(''), materialize(NULL, 2147483647, t[65535], 256)), materialize(NULL)) +; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT + materialize([(NULL, '11\01111111\011111', '1111')]) AS t, + (t[1048576]).2, + materialize(-2147483648), + (t[-2147483648]).1 +GROUP BY + materialize([(NULL, '1')]), + '', + (materialize((t[1023]).2), (materialize(''), (t[2147483647]).1, materialize(9223372036854775807)), (materialize(''), materialize(NULL), materialize(2147483647), materialize(t[65535]), materialize(256)), materialize(NULL)) +; From 22dad244e75137580e08aa2cc83538e5ffd950c8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 8 Aug 2024 01:50:49 +0000 Subject: [PATCH 0601/1722] fix whitespace --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 49e9121b65a..4133e47d4bc 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -887,7 +887,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c ctx->projections_iterator = ctx->tasks_for_projections.begin(); return false; } - + for (const auto & projection : global_ctx->projections_to_merge) { MergeTreeData::DataPartsVector projection_parts = global_ctx->projections_to_merge_parts[projection->name]; From da6378752fc562f7b8df487f86fe2257215eb96a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 04:07:43 +0200 Subject: [PATCH 0602/1722] Add annotations --- tests/queries/0_stateless/01304_direct_io_long.sh | 3 ++- tests/queries/0_stateless/02177_issue_31009_pt2.sql.j2 | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01304_direct_io_long.sh b/tests/queries/0_stateless/01304_direct_io_long.sh index 6ab25eebaf7..867c37667fe 100755 --- a/tests/queries/0_stateless/01304_direct_io_long.sh +++ b/tests/queries/0_stateless/01304_direct_io_long.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, no-object-storage-with-slow-build +# Tags: long, no-object-storage-with-slow-build, no-flaky-check +# It can be too long with ThreadFuzzer CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02177_issue_31009_pt2.sql.j2 b/tests/queries/0_stateless/02177_issue_31009_pt2.sql.j2 index 47940356302..7df77595347 100644 --- a/tests/queries/0_stateless/02177_issue_31009_pt2.sql.j2 +++ b/tests/queries/0_stateless/02177_issue_31009_pt2.sql.j2 @@ -1,4 +1,5 @@ --- Tags: long +-- Tags: long, no-flaky-check +-- It can be too long with ThreadFuzzer DROP TABLE IF EXISTS left; DROP TABLE IF EXISTS right; From 4fac40a3cb4823f0014a2c5324593b6ef8a6b6ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 04:41:22 +0200 Subject: [PATCH 0603/1722] Step back --- programs/client/Client.cpp | 10 ++++++++++ programs/local/LocalServer.cpp | 3 +++ src/Client/ClientBase.h | 3 +++ src/Client/LocalConnection.cpp | 1 + src/Client/LocalConnection.h | 2 ++ src/Client/Suggest.cpp | 4 ---- 6 files changed, 19 insertions(+), 4 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 0f136664de8..1d99d223ee9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -327,6 +328,7 @@ int Client::main(const std::vector & /*args*/) try { UseSSL use_ssl; + auto & thread_status = MainThreadStatus::getInstance(); setupSignalHandler(); std::cout << std::fixed << std::setprecision(3); @@ -341,6 +343,14 @@ try initTTYBuffer(toProgressOption(config().getString("progress", "default"))); ASTAlterCommand::setFormatAlterCommandsWithParentheses(true); + { + // All that just to set DB::CurrentThread::get().getGlobalContext() + // which is required for client timezone (pushed from server) to work. + auto thread_group = std::make_shared(); + const_cast(thread_group->global_context) = global_context; + thread_status.attachToGroup(thread_group, false); + } + /// Includes delayed_interactive. if (is_interactive) { diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index cb89c6c5510..0d731ed0e14 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -29,8 +29,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -461,6 +463,7 @@ int LocalServer::main(const std::vector & /*args*/) try { UseSSL use_ssl; + thread_status.emplace(); StackTrace::setShowAddresses(server_settings.show_addresses_in_stack_traces); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 26deb1eda26..1a23b6b1363 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -296,6 +296,9 @@ protected: Settings cmd_settings; MergeTreeSettings cmd_merge_tree_settings; + /// thread status should be destructed before shared context because it relies on process list. + std::optional thread_status; + ServerConnectionPtr connection; ConnectionParameters connection_parameters; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index b0a5ef99253..072184e0a66 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -128,6 +128,7 @@ void LocalConnection::sendQuery( state->query_id = query_id; state->query = query; + state->query_scope_holder = std::make_unique(query_context); state->stage = QueryProcessingStage::Enum(stage); state->profile_queue = std::make_shared(std::numeric_limits::max()); CurrentThread::attachInternalProfileEventsQueue(state->profile_queue); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 5cc3d0b30ec..b424c5b5aa3 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -61,6 +61,8 @@ struct LocalQueryState /// Time after the last check to stop the request and send the progress. Stopwatch after_send_progress; Stopwatch after_send_profile_events; + + std::unique_ptr query_scope_holder; }; diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index c1f163939e8..0188ebc8173 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -96,10 +96,6 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p loading_thread = std::thread([my_context = Context::createCopy(context), connection_parameters, suggestion_limit, this] { ThreadStatus thread_status; - my_context->makeQueryContext(); - auto group = ThreadGroup::createForQuery(my_context); - CurrentThread::attachToGroup(group); - for (size_t retry = 0; retry < 10; ++retry) { try From ca9bd647fbd97fd36de5e30778c824ae522e03c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 04:52:18 +0200 Subject: [PATCH 0604/1722] Simplification --- src/Client/ClientBase.cpp | 43 +++++++-------------------------------- 1 file changed, 7 insertions(+), 36 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 62a008bc88c..a305278fb4d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1877,48 +1877,19 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin profile_events.watch.restart(); { - /// Temporarily apply query settings to context. - std::optional old_settings; - SCOPE_EXIT_SAFE({ - if (old_settings) - client_context->setSettings(*old_settings); + /// Temporarily apply query settings to the context. + Settings old_settings = client_context->getSettingsCopy(); + SCOPE_EXIT_SAFE( + { + client_context->setSettings(old_settings); }); - - auto apply_query_settings = [&](const IAST & settings_ast) - { - if (!old_settings) - old_settings.emplace(client_context->getSettingsRef()); - client_context->applySettingsChanges(settings_ast.as()->changes); - client_context->resetSettingsToDefaultValue(settings_ast.as()->default_settings); - }; - - const auto * insert = parsed_query->as(); - if (const auto * select = parsed_query->as(); select && select->settings()) - apply_query_settings(*select->settings()); - else if (const auto * select_with_union = parsed_query->as()) - { - const ASTs & children = select_with_union->list_of_selects->children; - if (!children.empty()) - { - // On the client it is enough to apply settings only for the - // last SELECT, since the only thing that is important to apply - // on the client is format settings. - const auto * last_select = children.back()->as(); - if (last_select && last_select->settings()) - { - apply_query_settings(*last_select->settings()); - } - } - } - else if (const auto * query_with_output = parsed_query->as(); query_with_output && query_with_output->settings_ast) - apply_query_settings(*query_with_output->settings_ast); - else if (insert && insert->settings_ast) - apply_query_settings(*insert->settings_ast); + InterpreterSetQuery::applySettingsFromQuery(parsed_query, client_context); if (!connection->checkConnected(connection_parameters.timeouts)) connect(); ASTPtr input_function; + const auto * insert = parsed_query->as(); if (insert && insert->select) insert->tryFindInputFunction(input_function); From 5f747bc41174cb03c06288393a026d2385d57e8b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 8 Aug 2024 11:09:59 +0800 Subject: [PATCH 0605/1722] Add more test for implicitly type conversion --- .../02864_statistics_estimation.reference | 26 +++++++-- .../02864_statistics_estimation.sql | 55 ++++++++++++++++--- 2 files changed, 67 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index d475926df8b..077476d28c1 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -23,9 +23,23 @@ Test statistics multi-types: Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\')) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), greater(b, 50.1_Float64)) (removed) +1 +0 +0 +0 +0 +50 +50 +0 +0 +50 +0 +0 +1 +1 +1 +0 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 5eb51fe7111..1189de46267 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -1,6 +1,7 @@ -- Tags: no-fasttest DROP TABLE IF EXISTS tab SYNC; +DROP TABLE IF EXISTS tab2 SYNC; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; @@ -78,17 +79,55 @@ SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTi FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +DROP TABLE IF EXISTS tab SYNC; SELECT 'Test statistics implicitly type conversion:'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +CREATE TABLE tab2 +( + a String, + b UInt64, + c UInt8, + d DateTime, + e Boolean, + f Float64, + g Decimal32(1), + pk String, +) Engine = MergeTree() ORDER BY pk; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and b > 50.1/*5000*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS a, b, c, d; +ALTER TABLE tab2 ADD STATISTICS a TYPE count_min, uniq; +ALTER TABLE tab2 ADD STATISTICS b, c, d, e, f, g TYPE count_min, minmax, uniq, tdigest; +INSERT INTO tab2 select toString(number), number, number, cast(number, 'DateTime'), number % 2, number, toDecimal32(number, 1), toString(number) FROM system.numbers LIMIT 100; -DROP TABLE IF EXISTS tab SYNC; +SELECT count(*) FROM tab2 WHERE a = '0'; +SELECT count(*) FROM tab2 WHERE a = 0; -- { serverError NO_COMMON_TYPE } + +SELECT count(*) FROM tab2 WHERE b = 1.1; + +SELECT count(*) FROM tab2 WHERE c = 1.1; +SELECT count(*) FROM tab2 WHERE c = 1000; -- out of range of UInt16 + +SELECT count(*) FROM tab2 WHERE d = '2024-08-06 09:58:09'; +SELECT count(*) FROM tab2 WHERE d = '2024-08-06 09:58:0'; -- { serverError CANNOT_PARSE_DATETIME } + +SELECT count(*) FROM tab2 WHERE e = true; +SELECT count(*) FROM tab2 WHERE e = 'true'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab2 WHERE e = 1; +SELECT count(*) FROM tab2 WHERE e = 2; +SELECT count(*) FROM tab2 WHERE e = 1.1; +SELECT count(*) FROM tab2 WHERE e = '1'; + +SELECT count(*) FROM tab2 WHERE f = 1.1; +SELECT count(*) FROM tab2 WHERE f = '1.1'; +SELECT count(*) FROM tab2 WHERE f = 1; +SELECT count(*) FROM tab2 WHERE f = '1'; + +SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.0, 1); +SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.10, 1); +SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.0, 2); +SELECT count(*) FROM tab2 WHERE g = 1.0; +SELECT count(*) FROM tab2 WHERE g = 1.0; +SELECT count(*) FROM tab2 WHERE g = '1.0'; + +DROP TABLE IF EXISTS tab2 SYNC; From 21cdbe03f94bd3bdaef8b02e61b09e5e79ab380c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 8 Aug 2024 11:21:19 +0800 Subject: [PATCH 0606/1722] Add more test cases --- .../queries/0_stateless/02864_statistics_estimation.reference | 2 ++ tests/queries/0_stateless/02864_statistics_estimation.sql | 3 +++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index 077476d28c1..ad923d91dda 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -24,6 +24,8 @@ Test statistics multi-types: Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: 1 +1 +1 0 0 0 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 1189de46267..63cc2e47c97 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -103,7 +103,10 @@ INSERT INTO tab2 select toString(number), number, number, cast(number, 'DateTime SELECT count(*) FROM tab2 WHERE a = '0'; SELECT count(*) FROM tab2 WHERE a = 0; -- { serverError NO_COMMON_TYPE } +SELECT count(*) FROM tab2 WHERE b = 1; +SELECT count(*) FROM tab2 WHERE b = '1'; SELECT count(*) FROM tab2 WHERE b = 1.1; +SELECT count(*) FROM tab2 WHERE b = '1.1'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab2 WHERE c = 1.1; SELECT count(*) FROM tab2 WHERE c = 1000; -- out of range of UInt16 From d6ecabb41dff64a1fb8ac5a77ffc1a8bed15162b Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 5 Aug 2024 00:40:23 +0000 Subject: [PATCH 0607/1722] Fix flaky test_storage_s3_queue/test.py::test_multiple_tables_streaming_sync_distributed Disable parallel processing for the Ordered mode for the test_storage_s3_queue/test.py::test_multiple_tables_streaming_sync_distributed test. The reason for this is that the load between the processing nodes is too uneven when s3queue_processing_threads_num != 1, e.g.: ``` $ grep res1 pytest.log 2024-08-07 07:15:58 [ 575 ] DEBUG : res1 size: 13300, res2 size: 1700, total_rows: 15000 (test.py:813, test_multiple_tables_streaming_sync_distributed) ``` In CIs environment, there are rare cases when one of the processors handles all the workload, while the other is busy-waiting, and the test fails on assert: When s3queue_processing_threads_num == 1, the workload is evenly distributed: ``` $ grep res1 pytest.log 2024-08-07 07:26:52 [ 586 ] DEBUG : res1 size: 7200, res2 size: 7800, total_rows: 15000 (test.py:813, test_multiple_tables_streaming_sync_distributed) ``` This change only fixes test flakiness. Further investigation of the Order mode parallelism is required. --- tests/integration/test_storage_s3_queue/test.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 92d6f181464..8f197e09e61 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -771,7 +771,11 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): table_name, mode, files_path, - additional_settings={"keeper_path": keeper_path, "s3queue_buckets": 2}, + additional_settings={ + "keeper_path": keeper_path, + "s3queue_buckets": 2, + **({"s3queue_processing_threads_num": 1} if mode == "ordered" else {}), + }, ) for instance in [node, node_2]: @@ -806,6 +810,10 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): list(map(int, l.split())) for l in run_query(node_2, get_query).splitlines() ] + logging.debug( + f"res1 size: {len(res1)}, res2 size: {len(res2)}, total_rows: {total_rows}" + ) + assert len(res1) + len(res2) == total_rows # Checking that all engines have made progress From 6e7587e9b3dbc791f6970fe23703c1d65aa4b7f0 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 8 Aug 2024 14:44:08 +0800 Subject: [PATCH 0608/1722] Fix test for old analyzer --- .../02864_statistics_estimation.reference | 10 ++++---- .../02864_statistics_estimation.sql | 25 ++++++++++--------- 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index ad923d91dda..0e6cad6528b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -6,22 +6,22 @@ Test statistics count_min: Test statistics minmax: Prewhere info Prewhere filter - Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) + Prewhere filter column: and(greater(d, CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) Test statistics tdigest: Prewhere info Prewhere filter - Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) + Prewhere filter column: and(greater(d, CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) Test statistics uniq: Prewhere info Prewhere filter - Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), equals(c, 0)) (removed) + Prewhere filter column: and(equals(d, CAST(1, \'DateTime\')), equals(c, 0)) (removed) Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) + Prewhere filter column: and(equals(d, CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: 1 1 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 63cc2e47c97..94fd7d12e0a 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS tab2 SYNC; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; SET allow_suspicious_low_cardinality_types=1; +SET allow_experimental_analyzer=1; SET mutations_sync = 2; CREATE TABLE tab @@ -27,9 +28,9 @@ SELECT 'Test statistics count_min:'; ALTER TABLE tab ADD STATISTICS a, b, c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS a, b, c; @@ -38,9 +39,9 @@ SELECT 'Test statistics minmax:'; ALTER TABLE tab ADD STATISTICS b, c, d TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS b, c, d; @@ -49,9 +50,9 @@ SELECT 'Test statistics tdigest:'; ALTER TABLE tab ADD STATISTICS b, c, d TYPE tdigest; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS b, c, d; @@ -59,9 +60,9 @@ SELECT 'Test statistics uniq:'; ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS a, b, c, d; @@ -71,13 +72,13 @@ ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; ALTER TABLE tab ADD STATISTICS b, c, d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); DROP TABLE IF EXISTS tab SYNC; From bd3674e6e982b66f571248a717628649b1482fe5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Aug 2024 20:13:50 +0200 Subject: [PATCH 0609/1722] Add restart --- docker/test/stateless/run.sh | 34 ++++++++++++++++++++++++++++++---- 1 file changed, 30 insertions(+), 4 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b33c261dacc..19f2cfca43f 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -54,8 +54,6 @@ source /utils.lib /usr/share/clickhouse-test/config/install.sh ./setup_minio.sh stateless -./mc admin trace clickminio > /test_output/minio.log & -MC_ADMIN_PID=$! ./setup_hdfs_minicluster.sh @@ -176,7 +174,7 @@ done setup_logs_replication attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 -# create minio log webhooks for both audit and server logs +# create tables for minio log webhooks clickhouse-client --query "CREATE TABLE minio_audit_logs ( log String, @@ -184,7 +182,6 @@ clickhouse-client --query "CREATE TABLE minio_audit_logs ) ENGINE = MergeTree ORDER BY tuple()" -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" clickhouse-client --query "CREATE TABLE minio_server_logs ( @@ -193,7 +190,36 @@ clickhouse-client --query "CREATE TABLE minio_server_logs ) ENGINE = MergeTree ORDER BY tuple()" + +# create minio log webhooks for both audit and server logs ./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" +max_retries=100 +retry=1 + +while [ $retry -le $max_retries ]; do + echo "clickminio restart attempt $retry:" + + output=$(mc admin service restart clickminio 2>&1) + echo "$output" + + if echo "$output" | grep -q "Restarted \`clickminio\` successfully in 1 seconds"; then + echo "Restarted clickminio successfully." + break + fi + + sleep 1 + + retry=$((retry + 1)) +done + +if [ $retry -gt $max_retries ]; then + echo "Failed to restart clickminio after $max_retries attempts." +fi + +./mc admin service restart clickminio +./mc admin trace clickminio > /test_output/minio.log & +MC_ADMIN_PID=$! function fn_exists() { declare -F "$1" > /dev/null; From 9d30b45dbeddec977e2b5ef1ca8286c35c012129 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 08:43:49 +0100 Subject: [PATCH 0610/1722] Fix --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 19f2cfca43f..4d86afa4bac 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -200,7 +200,7 @@ retry=1 while [ $retry -le $max_retries ]; do echo "clickminio restart attempt $retry:" - output=$(mc admin service restart clickminio 2>&1) + output=$(./mc admin service restart clickminio 2>&1) echo "$output" if echo "$output" | grep -q "Restarted \`clickminio\` successfully in 1 seconds"; then From f4aac7bbd9431e4d95eadfea31239b331ea18d77 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 09:19:45 +0100 Subject: [PATCH 0611/1722] Another fix --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 4d86afa4bac..830a02a64a3 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -203,7 +203,7 @@ while [ $retry -le $max_retries ]; do output=$(./mc admin service restart clickminio 2>&1) echo "$output" - if echo "$output" | grep -q "Restarted \`clickminio\` successfully in 1 seconds"; then + if echo "$output" | grep -q "Restarted \`clickminio\` successfully"; then echo "Restarted clickminio successfully." break fi From d0f35ce6a60e13b8aff9687a45e293ce89693241 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 10:29:01 +0200 Subject: [PATCH 0612/1722] Fix setting prefix --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 433a0e96d2e..7205b5b3294 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -305,7 +305,8 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet S3::ListObjectsV2Request request; request.SetBucket(uri.bucket); - request.SetPrefix(path); + if (path != "/") + request.SetPrefix(path); if (max_keys) request.SetMaxKeys(static_cast(max_keys)); else From 59b737c9ac045ca0ec48eb2b8893c5e54646003a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 8 Aug 2024 10:50:14 +0200 Subject: [PATCH 0613/1722] CI: set correct timeout for stateless tests --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index b62d2e0aa8e..48847b0d7a6 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -420,7 +420,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=1001, # test + timeout=9000, ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", From dbe3035b6d96e40ae78204ba53cc91296b2af765 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:32:42 +0200 Subject: [PATCH 0614/1722] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 35cb124acfd..9f83b09c8a2 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -26,7 +26,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string { Poco::Util::AbstractConfiguration::Keys disk_settings_keys; config->keys(disk_settings_keys); - // Check that no settings are defined when disk from the config is referred. + /// Check that no settings are defined when disk from the config is referred. if (disk_settings_keys.empty()) throw Exception( ErrorCodes::BAD_ARGUMENTS, From 376d643e39b82011d8135a23c926ae22fee4d68a Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:32:55 +0200 Subject: [PATCH 0615/1722] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 9f83b09c8a2..6af7ad21366 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -30,7 +30,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string if (disk_settings_keys.empty()) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Disk function has no arguments. Invalid disk description."); + "Disk function must have arguments. Invalid disk description."); if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) throw Exception( From 8a93b1c7cff3c42f03642b4f7722c80c95937062 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:33:10 +0200 Subject: [PATCH 0616/1722] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 6af7ad21366..5c9ece699c4 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -35,7 +35,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Disk function `{}` has to have the other arguments which describe the disk. Invalid disk description.", + "Disk function `{}` must have other arguments apart from `name`, which describe disk configuration. Invalid disk description.", serialization); auto disk_settings_hash = sipHash128(serialization.data(), serialization.size()); From b79e701d8f7fda40bfb647c3c22bd637aef93fce Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 11:33:36 +0200 Subject: [PATCH 0617/1722] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 5c9ece699c4..6d0a4ec2ea3 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -137,7 +137,7 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte return disk_name; } -std::string DiskFomAST::getConfigDefinedDisk(const std::string &disk_name, ContextPtr context) +std::string DiskFomAST::getConfigDefinedDisk(const std::string & disk_name, ContextPtr context) { if (auto result = context->tryGetDisk(disk_name)) { From 35eb4fa1766c6f5b5d81cbfc68e63cadd4b01838 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 12:03:14 +0200 Subject: [PATCH 0618/1722] Schedule WV cleanup after any fire trigger --- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 94eed575ca8..4a20a07ae89 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1096,7 +1096,7 @@ void StorageWindowView::threadFuncFireProc() slide_interval); } - if (max_watermark >= timestamp_now) + if (max_watermark >= timestamp_start) clean_cache_task->schedule(); UInt64 next_fire_ms = static_cast(next_fire_signal) * 1000; From df23a3456f66f50397091f3d84c6f07a6a1d8d81 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 12:28:26 +0200 Subject: [PATCH 0619/1722] work with review --- src/Disks/DiskFomAST.cpp | 24 ++++++++------------ src/Disks/DiskFomAST.h | 2 +- src/Disks/StoragePolicy.h | 2 -- src/Interpreters/Context.cpp | 9 -------- src/Interpreters/Context.h | 1 - src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- 6 files changed, 11 insertions(+), 29 deletions(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 6d0a4ec2ea3..638161c5c16 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -133,25 +133,19 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte FlattenDiskConfigurationVisitor::Data data{context, attach}; FlattenDiskConfigurationVisitor{data}.visit(ast); - auto disk_name = assert_cast(*ast).value.get(); - return disk_name; + return assert_cast(*ast).value.get(); } -std::string DiskFomAST::getConfigDefinedDisk(const std::string & disk_name, ContextPtr context) +void DiskFomAST::ensureDiskIsNotCustom(const std::string & disk_name, ContextPtr context) { - if (auto result = context->tryGetDisk(disk_name)) - { - if (result->isCustomDisk()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Disk name `{}` is a custom disk that is used in other table. " - "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", - disk_name); + auto disk = context->getDisk(disk_name); - return disk_name; - } - - throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", disk_name); + if (disk->isCustomDisk()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Disk name `{}` is a custom disk that is used in other table. " + "That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.", + disk_name); } } diff --git a/src/Disks/DiskFomAST.h b/src/Disks/DiskFomAST.h index 3a70484eda0..0a30834533e 100644 --- a/src/Disks/DiskFomAST.h +++ b/src/Disks/DiskFomAST.h @@ -8,7 +8,7 @@ namespace DB namespace DiskFomAST { - std::string getConfigDefinedDisk(const std::string & name, ContextPtr context); + void ensureDiskIsNotCustom(const std::string & name, ContextPtr context); std::string createCustomDisk(const ASTPtr & disk_function, ContextPtr context, bool attach); } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index ccf2e2071b2..8e49ed910e3 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -12,8 +12,6 @@ #include #include -#include -#include #include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 30f77f799e9..5413b568068 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4395,15 +4395,6 @@ DiskPtr Context::getDisk(const String & name) const return disk_selector->get(name); } -DiskPtr Context::tryGetDisk(const String & name) const -{ - std::lock_guard lock(shared->storage_policies_mutex); - - auto disk_selector = getDiskSelector(lock); - - return disk_selector->tryGet(name); -} - DiskPtr Context::getOrCreateDisk(const String & name, DiskCreator creator) const { std::lock_guard lock(shared->storage_policies_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 586eff768df..d5e35c3e4b3 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1186,7 +1186,6 @@ public: /// Provides storage disks DiskPtr getDisk(const String & name) const; - DiskPtr tryGetDisk(const String & name) const; using DiskCreator = std::function; DiskPtr getOrCreateDisk(const String & name, DiskCreator creator) const; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index e11af43ed23..dabb6991b0b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -71,7 +71,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte } else { - value = DiskFomAST::getConfigDefinedDisk(value.safeGet(), context); + DiskFomAST::ensureDiskIsNotCustom(value.safeGet(), context); } if (has("storage_policy")) From ec145c86f5b4c4c716d4b27334381eeae83f99a8 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 8 Aug 2024 12:28:52 +0200 Subject: [PATCH 0620/1722] Update src/Disks/DiskFomAST.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Disks/DiskFomAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index 638161c5c16..bb2fcda68cb 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -64,7 +64,7 @@ std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string if (!disk->isCustomDisk()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The disk `{}` is already exist and described by the config." + "Disk `{}` already exists and is described by the config." " It is impossible to redefine it.", disk_name); From 69ac203c9fce9972b89082ca653894fc8709f2fd Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 12:56:33 +0200 Subject: [PATCH 0621/1722] fix tests --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/SystemLog.h | 2 +- tests/integration/test_system_flush_logs/test.py | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3051ed3e567..4a08fd5fe5b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4256,7 +4256,7 @@ std::shared_ptr Context::getS3QueueLog() const if (!shared->system_logs) return {}; - return shared->system_logs->s3_queue_log; + return shared->system_logs->s3queue_log; } std::shared_ptr Context::getAzureQueueLog() const diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 6682829c0c6..24ef6a18eb8 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -18,7 +18,7 @@ M(ErrorLog, error_log, "Contains history of error values from table system.errors, periodically flushed to disk.") \ M(FilesystemCacheLog, filesystem_cache_log, "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem.") \ M(FilesystemReadPrefetchesLog, filesystem_read_prefetches_log, "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem.") \ - M(ObjectStorageQueueLog, s3_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ + M(ObjectStorageQueueLog, s3queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ M(ObjectStorageQueueLog, azure_queue_log, "Contains logging entries with the information files processes by S3Queue engine.") \ M(AsynchronousMetricLog, asynchronous_metric_log, "Contains the historical values for system.asynchronous_metrics, once per time interval (one second by default).") \ M(OpenTelemetrySpanLog, opentelemetry_span_log, "Contains information about trace spans for executed queries.") \ diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index dd48ef055f5..cfecea5b3d6 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -24,8 +24,7 @@ def start_cluster(): def test_system_logs_exists(): system_logs = [ - # disabled by default - ("system.text_log", 0), + ("system.text_log", 1), ("system.query_log", 1), ("system.query_thread_log", 1), ("system.part_log", 1), From 117fedd3bbad038ba449e3ff69c85cd937388e80 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 12:59:32 +0200 Subject: [PATCH 0622/1722] fix style --- src/Disks/DiskFomAST.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index bb2fcda68cb..b2f1280c507 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_DISK; } std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string & serialization, ContextPtr context, bool attach) From b0ba53788ac758ac1405ceefacd91bb0418b5834 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 16:29:07 +0000 Subject: [PATCH 0623/1722] Refactor tests for (experimental) statistics --- docs/en/development/tests.md | 4 +- .../statements/alter/statistics.md | 16 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- ...2864_statistics_count_min_sketch.reference | 14 -- .../02864_statistics_count_min_sketch.sql | 70 ------ .../02864_statistics_ddl.reference | 37 +-- .../0_stateless/02864_statistics_ddl.sql | 234 ++++++++++++++---- ...delayed_materialization_in_merge.reference | 12 + ...stics_delayed_materialization_in_merge.sql | 36 +++ .../02864_statistics_exception.reference | 0 .../02864_statistics_exception.sql | 55 ---- ..._statistics_materialize_in_merge.reference | 10 - .../02864_statistics_materialize_in_merge.sql | 52 ---- .../02864_statistics_predicates.reference | 92 +++++++ .../02864_statistics_predicates.sql | 214 ++++++++++++++++ .../02864_statistics_uniq.reference | 35 --- .../0_stateless/02864_statistics_uniq.sql | 73 ------ .../02864_statistics_usage.reference | 20 ++ .../0_stateless/02864_statistics_usage.sql | 42 ++++ 19 files changed, 619 insertions(+), 399 deletions(-) delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.sql create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.reference create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql create mode 100644 tests/queries/0_stateless/02864_statistics_usage.reference create mode 100644 tests/queries/0_stateless/02864_statistics_usage.sql diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 269995a1a96..6cb36e2049b 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. :::note A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs @@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes ### Adding a New Test -To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 6880cef0e5c..7a1774a01b5 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -8,26 +8,28 @@ sidebar_label: STATISTICS The following operations are available: -- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. +- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata. -- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. +- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata. -- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. +- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. -- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. +- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. -- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. -There is an example adding two statistics types to two columns: +## Example: + +Adding two statistics types to two columns: ``` ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` :::note -Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 051d52a71cd..fe4857e9449 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3513,7 +3513,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistics is not not safe " + "ALTER types of column {} with statistics is not safe " "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference deleted file mode 100644 index 02c41656a36..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql deleted file mode 100644 index c730aa7b4a7..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql +++ /dev/null @@ -1,70 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS tab SYNC; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET allow_suspicious_low_cardinality_types=1; -SET mutations_sync = 2; - -CREATE TABLE tab -( - a String, - b UInt64, - c Int64, - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE tab; - -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'Test statistics count_min:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min; -ALTER TABLE tab ADD STATISTICS c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - - -SELECT 'Test statistics multi-types:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - -DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_ddl.reference b/tests/queries/0_stateless/02864_statistics_ddl.reference index a7ff5caa0b0..0e453b0ee8a 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.reference +++ b/tests/queries/0_stateless/02864_statistics_ddl.reference @@ -1,31 +1,6 @@ -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -10 -0 -After drop statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(a, 10)) (removed) -10 -CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After add statistic -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After materialize statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After rename - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, 10)) (removed) -20 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index fe612efe2ac..32b56a842b7 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -1,59 +1,195 @@ --- Tests that various DDL statements create/drop/materialize statistics +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests that DDL statements which create / drop / materialize statistics + +SET mutations_sync = 1; DROP TABLE IF EXISTS tab; +-- Error case: Can't create statistics when allow_experimental_statistics = 0 +CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; + +-- Error case: Unknown statistics types are rejected +CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- Error case: The same statistics type can't exist more than once on a column +CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +SET allow_suspicious_low_cardinality_types = 1; + +-- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*) + +-- tdigest requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- uniq requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- CREATE TABLE was easy, ALTER is more fun CREATE TABLE tab ( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f32 Float32, + s String, + a Array(Float64) +) +Engine = MergeTree() +ORDER BY tuple(); +-- Error case: Unknown statistics types are rejected +-- (relevant for ADD and MODIFY) +ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +-- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported + +-- Error case: The same statistics type can't exist more than once on a column +-- (relevant for ADD and MODIFY) +-- Create the same statistics object twice +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +-- Create an statistics which exists already +ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op +ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op + +-- Error case: Column does not exist +-- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS } + +-- Error case: Column exists but has no statistics +-- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS } + +-- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the +-- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that +-- works and one that doesn't work. +-- tdigest +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +-- uniq +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +-- count_min +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } + +-- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing +-- statistics objects (e.g. tdigest can be created on Float64 and UInt64) +ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +-- Finally, do a full-circle test of a good case. Print table definition after each step. +-- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested. +SHOW CREATE TABLE tab; +ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab CLEAR STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab MATERIALIZE STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab DROP STATISTICS f64, f32; SHOW CREATE TABLE tab; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; -SELECT count(*) FROM tab WHERE b < NULL and a < '10'; - -ALTER TABLE tab DROP STATISTICS a, b; - -SELECT 'After drop statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; - -SELECT 'After add statistic'; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After materialize statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -OPTIMIZE TABLE tab FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -ALTER TABLE tab RENAME COLUMN b TO c; -SHOW CREATE TABLE tab; - -SELECT 'After rename'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE c < 10 and a < 10; - -DROP TABLE IF EXISTS tab; +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference new file mode 100644 index 00000000000..eb5e685597c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference @@ -0,0 +1,12 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After truncate, insert, and materialize + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql new file mode 100644 index 00000000000..33a5f9052ba --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql @@ -0,0 +1,36 @@ +-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +DROP TABLE IF EXISTS tab; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_analyzer = 1; + +SET materialize_statistics_on_insert = 0; + +CREATE TABLE tab +( + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), +) ENGINE = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +TRUNCATE TABLE tab; +SET mutations_sync = 2; +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +SELECT 'After truncate, insert, and materialize'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql deleted file mode 100644 index 289ffee6600..00000000000 --- a/tests/queries/0_stateless/02864_statistics_exception.sql +++ /dev/null @@ -1,55 +0,0 @@ --- Tests creating/dropping/materializing statistics produces the right exceptions. - -DROP TABLE IF EXISTS tab; - --- Can't create statistics when allow_experimental_statistics = 0 -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - -SET allow_experimental_statistics = 1; - --- The same type of statistics can't exist more than once on a column -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest, tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- Unknown statistics types are rejected -CREATE TABLE tab -( - a Float64 STATISTICS(no_statistics_type) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- tDigest statistics can only be created on numeric columns -CREATE TABLE tab -( - a String STATISTICS(tdigest), -) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - -CREATE TABLE tab -( - a Float64, - b String -) Engine = MergeTree() ORDER BY tuple(); - -ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; --- Statistics can be created only on integer columns -ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS a; -ALTER TABLE tab DROP STATISTICS IF EXISTS a; -ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS a; -ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } - -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; -ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference deleted file mode 100644 index 5e969cf41cb..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference +++ /dev/null @@ -1,10 +0,0 @@ -10 -10 -10 -statistics not used Condition less(b, 10_UInt8) moved to PREWHERE -statistics not used Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE -2 0 diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql deleted file mode 100644 index 6606cff263f..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql +++ /dev/null @@ -1,52 +0,0 @@ --- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). - -DROP TABLE IF EXISTS tab; - -SET enable_analyzer = 1; -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -SET materialize_statistics_on_insert = 0; - -CREATE TABLE tab -( - a Int64 STATISTICS(tdigest), - b Int16 STATISTICS(tdigest), -) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; - -OPTIMIZE TABLE tab FINAL; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; - -TRUNCATE TABLE tab; -SET mutations_sync = 2; - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; - -DROP TABLE tab; - -SYSTEM FLUSH LOGS; - -SELECT log_comment, message FROM system.text_log JOIN -( - SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log - WHERE current_database = currentDatabase() - AND query LIKE 'SELECT count(*) FROM tab%' - AND type = 'QueryFinish' -) AS query_log USING (query_id) -WHERE message LIKE '%moved to PREWHERE%' -ORDER BY event_time_microseconds; - -SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) -FROM system.query_log -WHERE current_database = currentDatabase() - AND query LIKE 'INSERT INTO tab SELECT%' - AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference new file mode 100644 index 00000000000..1c2abd47aaf --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -0,0 +1,92 @@ +u64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +10 +u64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +f64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +0 +0 +0 +f64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +80 +80 +80 +dt and = +0 +0 +0 +0 +10 +10 +10 +10 +dt and < +10000 +10000 +10000 +70 +70 +70 +70 +b and = +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +0 +0 +0 +0 +s and = +10 +10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql new file mode 100644 index 00000000000..3e754dfb1de --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -0,0 +1,214 @@ +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types. + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + u64 UInt64, + u64_tdigest UInt64 STATISTICS(tdigest), + u64_count_min UInt64 STATISTICS(count_min), + u64_uniq UInt64 STATISTICS(uniq), + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f64_count_min Float64 STATISTICS(count_min), + f64_uniq Float64 STATISTICS(uniq), + dt DateTime, + dt_tdigest DateTime STATISTICS(tdigest), + dt_count_min DateTime STATISTICS(count_min), + dt_uniq DateTime STATISTICS(uniq), + b Bool, + b_tdigest Bool STATISTICS(tdigest), + b_count_min Bool STATISTICS(count_min), + b_uniq Bool STATISTICS(uniq), + s String, + -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest + s_count_min String STATISTICS(count_min) + -- s_uniq String STATISTICS(uniq), -- not supported by uniq +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab +-- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; +SELECT number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 2, + number % 2, + number % 2, + number % 2, + toString(number % 1000), + toString(number % 1000) +FROM system.numbers LIMIT 10000; + +-- u64 ---------------------------------------------------- + +SELECT 'u64 and ='; + +SELECT count(*) FROM tab WHERE u64 = 7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7; +SELECT count(*) FROM tab WHERE u64_count_min = 7; +SELECT count(*) FROM tab WHERE u64_uniq = 7; + +SELECT count(*) FROM tab WHERE u64 = 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE u64_count_min = 7.7; +SELECT count(*) FROM tab WHERE u64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE u64 = '7'; +SELECT count(*) FROM tab WHERE u64_tdigest = '7'; +SELECT count(*) FROM tab WHERE u64_count_min = '7'; +SELECT count(*) FROM tab WHERE u64_uniq = '7'; + +SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } + +SELECT 'u64 and <'; + +SELECT count(*) FROM tab WHERE u64 < 7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7; +SELECT count(*) FROM tab WHERE u64_count_min < 7; +SELECT count(*) FROM tab WHERE u64_uniq < 7; + +SELECT count(*) FROM tab WHERE u64 < 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE u64_count_min < 7.7; +SELECT count(*) FROM tab WHERE u64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE u64 < '7'; +-- SELECT count(*) FROM tab WHERE u64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE u64_count_min < '7'; +SELECT count(*) FROM tab WHERE u64_uniq < '7'; + +SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } + +-- f64 ---------------------------------------------------- + +SELECT 'f64 and ='; + +SELECT count(*) FROM tab WHERE f64 = 7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7; +SELECT count(*) FROM tab WHERE f64_count_min = 7; +SELECT count(*) FROM tab WHERE f64_uniq = 7; + +SELECT count(*) FROM tab WHERE f64 = 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE f64_count_min = 7.7; +SELECT count(*) FROM tab WHERE f64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE f64 = '7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest = '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min = '7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7'; + +SELECT count(*) FROM tab WHERE f64 = '7.7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; + +SELECT 'f64 and <'; + +SELECT count(*) FROM tab WHERE f64 < 7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7; +SELECT count(*) FROM tab WHERE f64_count_min < 7; +SELECT count(*) FROM tab WHERE f64_uniq < 7; + +SELECT count(*) FROM tab WHERE f64 < 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE f64_count_min < 7.7; +SELECT count(*) FROM tab WHERE f64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE f64 < '7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min < '7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7'; + +SELECT count(*) FROM tab WHERE f64 < '7.7'; +-- SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; + +-- dt ---------------------------------------------------- + +SELECT 'dt and ='; + +SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt = 7; +SELECT count(*) FROM tab WHERE dt_tdigest = 7; +SELECT count(*) FROM tab WHERE dt_count_min = 7; +SELECT count(*) FROM tab WHERE dt_uniq = 7; + +SELECT 'dt and <'; + +SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; +-- SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt < 7; +SELECT count(*) FROM tab WHERE dt_tdigest < 7; +SELECT count(*) FROM tab WHERE dt_count_min < 7; +SELECT count(*) FROM tab WHERE dt_uniq < 7; + +-- b ---------------------------------------------------- + +SELECT 'b and ='; + +SELECT count(*) FROM tab WHERE b = true; +SELECT count(*) FROM tab WHERE b_tdigest = true; +SELECT count(*) FROM tab WHERE b_count_min = true; +SELECT count(*) FROM tab WHERE b_uniq = true; + +SELECT count(*) FROM tab WHERE b = 'true'; +SELECT count(*) FROM tab WHERE b_tdigest = 'true'; +SELECT count(*) FROM tab WHERE b_count_min = 'true'; +SELECT count(*) FROM tab WHERE b_uniq = 'true'; + +SELECT count(*) FROM tab WHERE b = 1; +SELECT count(*) FROM tab WHERE b_tdigest = 1; +SELECT count(*) FROM tab WHERE b_count_min = 1; +SELECT count(*) FROM tab WHERE b_uniq = 1; + +SELECT count(*) FROM tab WHERE b = 1.1; +SELECT count(*) FROM tab WHERE b_tdigest = 1.1; +SELECT count(*) FROM tab WHERE b_count_min = 1.1; +SELECT count(*) FROM tab WHERE b_uniq = 1.1; + +-- s ---------------------------------------------------- + +SELECT 's and ='; + +SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported + +SELECT count(*) FROM tab WHERE s = '7'; +-- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = '7'; +-- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference deleted file mode 100644 index 77786dbdd8c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.reference +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After modify TDigest - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) -After drop - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql deleted file mode 100644 index 0f5f353c045..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.sql +++ /dev/null @@ -1,73 +0,0 @@ -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; - -CREATE TABLE t1 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Int64 STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; -INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -OPTIMIZE TABLE t1 FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT 'After modify TDigest'; -ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; -ALTER TABLE t1 MATERIALIZE STATISTICS c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -ALTER TABLE t1 DROP STATISTICS c; - -SELECT 'After drop'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c LowCardinality(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Nullable(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t3; - diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference new file mode 100644 index 00000000000..a9f669b88c1 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.reference @@ -0,0 +1,20 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After drop statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After add and materialize statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After rename + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql new file mode 100644 index 00000000000..f936854df44 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.sql @@ -0,0 +1,42 @@ +-- Test that the optimizer picks up column statistics +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; +SET allow_analyzer = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest) +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab DROP STATISTICS a, b; +SELECT 'After drop statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After add and materialize statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab RENAME COLUMN b TO c; +SELECT 'After rename'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used) + +DROP TABLE IF EXISTS tab; From 4e97549f7336d33e4c3fc08fec21ce7af9f433a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 14:22:11 +0200 Subject: [PATCH 0624/1722] Make 02884_authentication_quota.sh more parallel --- .../0_stateless/02884_authentication_quota.sh | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02884_authentication_quota.sh b/tests/queries/0_stateless/02884_authentication_quota.sh index f013bb4d639..ef48ac741f0 100755 --- a/tests/queries/0_stateless/02884_authentication_quota.sh +++ b/tests/queries/0_stateless/02884_authentication_quota.sh @@ -1,13 +1,12 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -QUOTA="2884_quota_$$" -USER="2884_user_$$" -ROLE="2884_role_$$" +QUOTA="2884_quota_${CLICKHOUSE_DATABASE}" +USER="2884_user_${CLICKHOUSE_DATABASE}" +ROLE="2884_role_${CLICKHOUSE_DATABASE}" function login_test() @@ -26,10 +25,10 @@ function login_test() echo "> Alter the quota with MAX FAILED SEQUENTIAL AUTHENTICATIONS = 4" ${CLICKHOUSE_CLIENT} -q "ALTER QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX FAILED SEQUENTIAL AUTHENTICATIONS = 4 TO ${USER}" - + echo "> Try to login to the user account with correct password" ${CLICKHOUSE_CLIENT} --user ${USER} --password "pass" --query "select 1 format Null" - + echo "> Successfull login should reset failed authentications counter. Check the failed_sequential_authentications, max_failed_sequential_authentications fields." ${CLICKHOUSE_CLIENT} -q "SELECT failed_sequential_authentications, max_failed_sequential_authentications FROM system.quotas_usage WHERE quota_name = '${QUOTA}'" @@ -39,7 +38,7 @@ function login_test() ${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'password is incorrect' ${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'password is incorrect' ${CLICKHOUSE_CLIENT} --user ${USER} --password "wrong_pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'QUOTA_EXCEEDED' - + echo "> Also try to login with correct password. Quota should stay exceeded." ${CLICKHOUSE_CLIENT} --user ${USER} --password "pass" --query "select 1 format Null" 2>&1 | grep -m1 -o 'QUOTA_EXCEEDED' From dbf743d58d7b6100183b2409cf3259da1a1a457f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 14:49:35 +0200 Subject: [PATCH 0625/1722] Make 02722_database_filesystem parallelizable --- .../02722_database_filesystem.reference | 2 +- .../0_stateless/02722_database_filesystem.sh | 60 +++++++++---------- 2 files changed, 30 insertions(+), 32 deletions(-) diff --git a/tests/queries/0_stateless/02722_database_filesystem.reference b/tests/queries/0_stateless/02722_database_filesystem.reference index 2aa7e74befb..290bb9d4fbc 100644 --- a/tests/queries/0_stateless/02722_database_filesystem.reference +++ b/tests/queries/0_stateless/02722_database_filesystem.reference @@ -1,6 +1,6 @@ Test 1: create filesystem database and check implicit calls 0 -test1 +default_test1 4 4 30 diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index fa23d847d90..e28c9358f1e 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -15,14 +14,11 @@ echo '2,"def",456,"bacabaa"' >> ${user_files_tmp_dir}/tmp.csv echo '3,"story",78912,"acabaab"' >> ${user_files_tmp_dir}/tmp.csv echo '4,"history",21321321,"cabaaba"' >> ${user_files_tmp_dir}/tmp.csv -tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME} - $CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_1.csv') select * from numbers(1, 10)" $CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_2.csv') select * from numbers(11, 10)" $CLICKHOUSE_LOCAL -q "insert into function file('$user_files_tmp_dir/tmp_numbers_30.csv') select * from numbers(21, 10)" -[[ -d $tmp_dir ]] && rm -rd $tmp_dir -mkdir $tmp_dir +tmp_dir=$(mktemp -d ${CLICKHOUSE_TEST_UNIQUE_NAME}_XXXX) cp ${user_files_tmp_dir}/tmp.csv ${tmp_dir}/tmp.csv cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp/tmp.csv cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp.myext @@ -30,55 +26,57 @@ cp ${user_files_tmp_dir}/tmp_numbers_1.csv ${user_files_tmp_dir}/tmp/tmp_numbers ################# echo "Test 1: create filesystem database and check implicit calls" +DATABASE_TEST1="${CLICKHOUSE_DATABASE}_test1" ${CLICKHOUSE_CLIENT} --multiline -q """ -DROP DATABASE IF EXISTS test1; -CREATE DATABASE test1 ENGINE = Filesystem; +DROP DATABASE IF EXISTS ${DATABASE_TEST1}; +CREATE DATABASE ${DATABASE_TEST1} ENGINE = Filesystem; """ echo $? -${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "test1" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/tmp.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp_numbers_*.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/*tmp_numbers_*.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/*/*tmp_numbers_*.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "${DATABASE_TEST1}" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp/tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp_numbers_*.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/tmp/*tmp_numbers_*.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`${unique_name}/*/*tmp_numbers_*.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; # **/* does not search in the current directory but searches recursively in nested directories. -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/**/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; -${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM test1.\`${unique_name}/**/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/**/*.csv\` WHERE startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SELECT count(DISTINCT _path) FROM ${DATABASE_TEST1}.\`${unique_name}/**/*.csv\` WHERE not startsWith(_path, '${user_files_tmp_dir}')"; +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\"" ################# echo "Test 2: check DatabaseFilesystem access rights and errors handling on server" # DATABASE_ACCESS_DENIED: Allows list files only inside user_files -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../*/tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../tmp_numbers_*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../*.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --multiline --query """ -USE test1; +USE ${DATABASE_TEST1}; SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; """ 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: path should be inside user_files +DATABASE_TEST2="${CLICKHOUSE_DATABASE}_test2" ${CLICKHOUSE_CLIENT} --multiline -q """ -DROP DATABASE IF EXISTS test2; -CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); +DROP DATABASE IF EXISTS ${DATABASE_TEST2}; +CREATE DATABASE ${DATABASE_TEST2} ENGINE = Filesystem('/tmp'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exist ${CLICKHOUSE_CLIENT} --multiline -q """ -DROP DATABASE IF EXISTS test2; -CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); +DROP DATABASE IF EXISTS ${DATABASE_TEST2}; +CREATE DATABASE ${DATABASE_TEST2} ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # FILE_DOESNT_EXIST: unknown file -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM ${DATABASE_TEST1}.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||: # Clean -${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" +${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS ${DATABASE_TEST1};" rm -rd $tmp_dir rm -rd $user_files_tmp_dir From de0d6f036853e8c74f2d0bde3a754a924c294388 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 15:00:13 +0200 Subject: [PATCH 0626/1722] Make 03002_part_log_rmt_fetch_* tests parallel --- .../0_stateless/03002_part_log_rmt_fetch_merge_error.sh | 8 ++++---- .../0_stateless/03002_part_log_rmt_fetch_mutate_error.sh | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh index e58c542b8ac..dbaf7bf20c4 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-shared-merge-tree +# Tags: no-replicated-database, no-shared-merge-tree, no-fasttest # SMT: The merge process is completely different from RMT +# no-fasttest: Avoid long waits CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -12,8 +13,7 @@ function wait_until() { local q=$1 && shift while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do - # too frequent FLUSH LOGS is too costly - sleep 2 + sleep 0.5 done } @@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT -nm -q " system start replicated sends rmt_master; " # wait until rmt_slave will fetch the part and reflect this error in system.part_log -wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" +wait_until "select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" $CLICKHOUSE_CLIENT -nm -q " system sync replica rmt_slave; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh index cc8f53aafb9..a6a02946602 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-shared-merge-tree +# Tags: no-replicated-database, no-shared-merge-tree, no-fasttest # SMT: The merge process is completely different from RMT +# no-fasttest: Avoid long waits CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -15,8 +16,7 @@ function wait_until() { local q=$1 && shift while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do - # too frequent FLUSH LOGS is too costly - sleep 2 + sleep 0.5 done } @@ -43,9 +43,8 @@ $CLICKHOUSE_CLIENT -nm -q " wait_for_mutation rmt_master 0000000000 $CLICKHOUSE_CLIENT -nm -q "system start pulling replication log rmt_slave" # and wait until rmt_slave to fetch the part and reflect this error in system.part_log -wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" +wait_until "select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" $CLICKHOUSE_CLIENT -nm -q " - system flush logs; select 'before'; select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; From d3dffce4409a6afb1968c09a4f79ce21fb861515 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 15:03:46 +0200 Subject: [PATCH 0627/1722] Make 02247_written_bytes_quota parallel --- .../0_stateless/02247_written_bytes_quota.sh | 51 ++++++++++--------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/02247_written_bytes_quota.sh b/tests/queries/0_stateless/02247_written_bytes_quota.sh index 2430184c4df..ce32c76ab18 100755 --- a/tests/queries/0_stateless/02247_written_bytes_quota.sh +++ b/tests/queries/0_stateless/02247_written_bytes_quota.sh @@ -1,48 +1,51 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +ROLE="r_${CLICKHOUSE_TEST_UNIQUE_NAME}" +USER="u_${CLICKHOUSE_TEST_UNIQUE_NAME}" +QUOTA="q_${CLICKHOUSE_TEST_UNIQUE_NAME}" + ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247" -${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247" -${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247" -${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247" +${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS ${ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS ${USER}" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS ${QUOTA}" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE written_bytes_02247(s String) ENGINE = Memory" -${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02247" -${CLICKHOUSE_CLIENT} -q "CREATE USER u02247" -${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO r02247" -${CLICKHOUSE_CLIENT} -q "GRANT r02247 to u02247" -${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO r02247" +${CLICKHOUSE_CLIENT} -q "CREATE ROLE ${ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${USER}" +${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO ${ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT ${ROLE} to ${USER}" +${CLICKHOUSE_CLIENT} -q "CREATE QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO ${ROLE}" # The value 'qwqw' means about 13 bytes are to be written, so the current quota (30 bytes) gives the ability to write 'qwqw' 2 times. -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" -#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA" -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" -#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA" -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED -${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" +#${CLICKHOUSE_CLIENT} --user ${USER} -q "SHOW CURRENT QUOTA" +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" +#${CLICKHOUSE_CLIENT} --user ${USER} -q "SHOW CURRENT QUOTA" +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED +${CLICKHOUSE_CLIENT} --user ${USER} --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED -${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = 'q02247'" +${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = '${QUOTA}'" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247" -${CLICKHOUSE_CLIENT} -q "DROP QUOTA q02247" -${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO r02247" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA ${QUOTA}" +${CLICKHOUSE_CLIENT} -q "CREATE QUOTA ${QUOTA} FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO ${ROLE}" ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE written_bytes_02247" # Numbers from 0 to 50 means about 540 bytes are to be written, so the current quota (1000 bytes) is enough to do so. -${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)" +${CLICKHOUSE_CLIENT} --user ${USER} -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)" # Numbers from 0 to 100 means about 1090 bytes are to be written, so the current quota (1000 bytes total - 540 bytes already used) is NOT enough to do so. -${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED +${CLICKHOUSE_CLIENT} --user ${USER} -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED -${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = 'q02247'" +${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = '${QUOTA}'" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247" -${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247" -${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247" -${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247" +${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS ${ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS ${USER}" +${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS ${QUOTA}" From cab274e1b696e8e355066cce3b05d4337c486157 Mon Sep 17 00:00:00 2001 From: kruglov Date: Fri, 2 Aug 2024 10:46:56 +0300 Subject: [PATCH 0628/1722] Fixed error on generated columns in MaterializedPostgreSQL --- .../fetchPostgreSQLTableStructure.cpp | 34 +++++++++----- .../fetchPostgreSQLTableStructure.h | 1 + .../test.py | 44 ++++++++++++++++++- 3 files changed, 67 insertions(+), 12 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 943f3ae502e..e2f2358c892 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -196,7 +196,7 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( } else { - std::tuple row; + std::tuple row; while (stream >> row) { const auto column_name = std::get<0>(row); @@ -206,13 +206,14 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( std::get<3>(row)); columns.push_back(NameAndTypePair(column_name, data_type)); - auto attgenerated = std::get<6>(row); + auto attgenerated = std::get<7>(row); attributes.emplace( column_name, PostgreSQLTableStructure::PGAttribute{ .atttypid = parse(std::get<4>(row)), .atttypmod = parse(std::get<5>(row)), + .attnum = parse(std::get<6>(row)), .atthasdef = false, .attgenerated = attgenerated.empty() ? char{} : char(attgenerated[0]), .attr_def = {} @@ -308,6 +309,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "attndims AS dims, " /// array dimensions "atttypid as type_id, " "atttypmod as type_modifier, " + "attnum as att_num, " "attgenerated as generated " /// if column has GENERATED "FROM pg_attribute " "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) " @@ -338,17 +340,29 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "WHERE adrelid = (SELECT oid FROM pg_class WHERE {});", where); pqxx::result result{tx.exec(attrdef_query)}; - for (const auto row : result) + if (static_cast(result.size()) > table.physical_columns->names.size()) { - size_t adnum = row[0].as(); - if (!adnum || adnum > table.physical_columns->names.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Received {} attrdef, but currently fetched columns list has {} columns", + result.size(), table.physical_columns->attributes.size()); + } + + for (const auto & column_attrs : table.physical_columns->attributes) + { + if (column_attrs.second.attgenerated != 's') { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Received adnum {}, but currently fetched columns list has {} columns", - adnum, table.physical_columns->attributes.size()); + continue; + } + + for (const auto row : result) + { + int adnum = row[0].as(); + if (column_attrs.second.attnum == adnum) + { + table.physical_columns->attributes.at(column_attrs.first).attr_def = row[1].as(); + break; + } } - const auto column_name = table.physical_columns->names[adnum - 1]; - table.physical_columns->attributes.at(column_name).attr_def = row[1].as(); } } diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 81bf7b278fc..25ece6909fd 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -16,6 +16,7 @@ struct PostgreSQLTableStructure { Int32 atttypid; Int32 atttypmod; + Int32 attnum; bool atthasdef; char attgenerated; std::string attr_def; diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 406b50bc486..75edb22aab1 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -953,12 +953,14 @@ def test_generated_columns(started_cluster): "", f"""CREATE TABLE {table} ( key integer PRIMARY KEY, - x integer, + x integer DEFAULT 0, + temp integer DEFAULT 0, y integer GENERATED ALWAYS AS (x*2) STORED, - z text); + z text DEFAULT 'z'); """, ) + pg_manager.execute(f"alter table {table} drop column temp;") pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'1');") pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'2');") @@ -991,6 +993,44 @@ def test_generated_columns(started_cluster): ) +def test_generated_columns_with_sequence(started_cluster): + table = "test_generated_columns_with_sequence" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer, + y integer GENERATED ALWAYS AS (x*2) STORED, + z text); + """, + ) + + pg_manager.execute( + f"create sequence {table}_id_seq increment by 1 minvalue 1 start 1;" + ) + pg_manager.execute( + f"alter table {table} alter key set default nextval('{table}_id_seq');" + ) + pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'1');") + pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'2');") + + pg_manager.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + ], + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + def test_default_columns(started_cluster): table = "test_default_columns" From 72bc5cd2e99cee09d0e003fb75192c0bb3114bad Mon Sep 17 00:00:00 2001 From: Kruglov Kirill Date: Mon, 5 Aug 2024 16:10:27 +0300 Subject: [PATCH 0629/1722] Update src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index e2f2358c892..b9fd9c325f8 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -349,7 +349,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( for (const auto & column_attrs : table.physical_columns->attributes) { - if (column_attrs.second.attgenerated != 's') + if (column_attrs.second.attgenerated != 's') /// e.g. not a generated column { continue; } From 55b2000d38e0bc6282714fdb1204d450437433ec Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 13:58:43 +0000 Subject: [PATCH 0630/1722] Fix fasttest --- .../02864_statistics_delayed_materialization_in_merge.sql | 2 +- tests/queries/0_stateless/02864_statistics_usage.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql index 33a5f9052ba..d469a4c2036 100644 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS tab; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; -SET allow_analyzer = 1; +SET enable_analyzer = 1; SET materialize_statistics_on_insert = 0; diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql index f936854df44..4956bd27e87 100644 --- a/tests/queries/0_stateless/02864_statistics_usage.sql +++ b/tests/queries/0_stateless/02864_statistics_usage.sql @@ -6,7 +6,7 @@ SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; SET mutations_sync = 1; -SET allow_analyzer = 1; +SET enable_analyzer = 1; DROP TABLE IF EXISTS tab; From 05febdfb2bdfa78f2d017758ce2261fb554e9546 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 13:47:44 +0000 Subject: [PATCH 0631/1722] add more events and add tests --- src/Common/ProfileEvents.cpp | 9 +- .../Transforms/ColumnGathererTransform.h | 1 - .../Transforms/MergeJoinTransform.cpp | 3 +- .../Transforms/MergeJoinTransform.h | 1 + .../Transforms/PasteJoinTransform.cpp | 2 +- .../Transforms/PasteJoinTransform.h | 1 + src/Storages/MergeTree/MergeList.h | 1 - src/Storages/MergeTree/MergeTask.cpp | 16 ++-- src/Storages/MergeTree/MergeTask.h | 2 +- .../MergeTree/MutateFromLogEntryTask.cpp | 2 + .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 + src/Storages/MergeTree/MutateTask.cpp | 25 ++++-- src/Storages/MergeTree/MutateTask.h | 1 + .../02378_part_log_profile_events.sql | 2 +- .../03221_merge_profile_events.reference | 3 + .../03221_merge_profile_events.sql | 88 +++++++++++++++++++ .../03221_mutate_profile_events.reference | 2 + .../03221_mutate_profile_events.sql | 33 +++++++ 18 files changed, 174 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/03221_merge_profile_events.reference create mode 100644 tests/queries/0_stateless/03221_merge_profile_events.sql create mode 100644 tests/queries/0_stateless/03221_mutate_profile_events.reference create mode 100644 tests/queries/0_stateless/03221_mutate_profile_events.sql diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 857a08d8a5d..d43d9fdcea8 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -209,6 +209,8 @@ \ M(Merge, "Number of launched background merges.") \ M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \ + M(MergedColumns, "Number of columns merged during the horizontal stage of merges.") \ + M(GatheredColumns, "Number of columns gathered during the vertical stage of merges.") \ M(MergedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for background merges. This is the number before merge.") \ M(MergeTotalMilliseconds, "Total time spent for background merges") \ M(MergeExecuteMilliseconds, "Total busy time spent for execution of background merges") \ @@ -231,8 +233,11 @@ M(MutationUntouchedParts, "Number of total parts for which mutations tried to be applied but which was completely skipped according to predicate") \ M(MutatedRows, "Rows read for mutations. This is the number of rows before mutation") \ M(MutatedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for mutations. This is the number before mutation.") \ - M(MutationTimeMilliseconds, "Total time spent for mutations.") \ - M(MutateTaskProjectionsCalculationMicroseconds, "Time spent calculating projections") \ + M(MutationTotalMilliseconds, "Total time spent for mutations.") \ + M(MutationExecuteMilliseconds, "Total busy time spent for execution of mutations.") \ + M(MutationAllPartColumns, "Number of times when task to mutate all columns in part was created") \ + M(MutationSomePartColumns, "Number of times when task to mutate some columns in part was created") \ + M(MutateTaskProjectionsCalculationMicroseconds, "Time spent calculating projections in mutations.") \ \ M(MergeTreeDataWriterRows, "Number of rows INSERTed to MergeTree tables.") \ M(MergeTreeDataWriterUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) INSERTed to MergeTree tables.") \ diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index a535b2669d0..fbc9a6bfcc6 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -2,7 +2,6 @@ #include #include -#include "base/types.h" #include #include diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 26601207da8..ec7f567ea57 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -515,7 +515,7 @@ IMergingAlgorithm::MergedStats MergeJoinAlgorithm::getMergedStats() const { return { - .bytes = 0, + .bytes = stat.num_bytes[0] + stat.num_bytes[1], .rows = stat.num_rows[0] + stat.num_rows[1], .blocks = stat.num_blocks[0] + stat.num_blocks[1], }; @@ -557,6 +557,7 @@ void MergeJoinAlgorithm::consume(Input & input, size_t source_num) { stat.num_blocks[source_num] += 1; stat.num_rows[source_num] += input.chunk.getNumRows(); + stat.num_bytes[source_num] += input.chunk.allocatedBytes(); } prepareChunk(input.chunk); diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 841a3f15a92..8f74974af0f 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -282,6 +282,7 @@ private: { size_t num_blocks[2] = {0, 0}; size_t num_rows[2] = {0, 0}; + size_t num_bytes[2] = {0, 0}; size_t max_blocks_loaded = 0; }; diff --git a/src/Processors/Transforms/PasteJoinTransform.cpp b/src/Processors/Transforms/PasteJoinTransform.cpp index ad01b721726..982a347a70f 100644 --- a/src/Processors/Transforms/PasteJoinTransform.cpp +++ b/src/Processors/Transforms/PasteJoinTransform.cpp @@ -62,7 +62,7 @@ IMergingAlgorithm::MergedStats PasteJoinAlgorithm::getMergedStats() const { return { - .bytes = 0, + .bytes = stat.num_bytes[0] + stat.num_bytes[1], .rows = stat.num_rows[0] + stat.num_rows[1], .blocks = stat.num_blocks[0] + stat.num_blocks[1], }; diff --git a/src/Processors/Transforms/PasteJoinTransform.h b/src/Processors/Transforms/PasteJoinTransform.h index fbe85f6993b..c184f20362d 100644 --- a/src/Processors/Transforms/PasteJoinTransform.h +++ b/src/Processors/Transforms/PasteJoinTransform.h @@ -54,6 +54,7 @@ private: { size_t num_blocks[2] = {0, 0}; size_t num_rows[2] = {0, 0}; + size_t num_bytes[2] = {0, 0}; size_t max_blocks_loaded = 0; }; diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 3a96ba0abae..d40af6abf43 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -6,7 +6,6 @@ #include #include #include -#include "base/types.h" #include #include #include diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5f178f08ec3..39bac8f7c24 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -5,9 +5,6 @@ #include #include -#include "Common/ElapsedTimeProfileEventIncrement.h" -#include "Common/Logger.h" -#include "Common/Stopwatch.h" #include #include #include @@ -46,6 +43,8 @@ namespace ProfileEvents { extern const Event Merge; + extern const Event MergedColumns; + extern const Event GatheredColumns; extern const Event MergeTotalMilliseconds; extern const Event MergeExecuteMilliseconds; extern const Event MergeHorizontalStageExecuteMilliseconds; @@ -183,6 +182,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() { + ProfileEvents::increment(ProfileEvents::Merge); + String local_tmp_prefix; if (global_ctx->need_prefix) { @@ -200,8 +201,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (isTTLMergeType(global_ctx->future_part->merge_type) && global_ctx->ttl_merges_blocker->isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with TTL"); - ProfileEvents::increment(ProfileEvents::Merge); - LOG_DEBUG(ctx->log, "Merging {} parts: from {} to {} into {} with storage {}", global_ctx->future_part->parts.size(), global_ctx->future_part->parts.front()->name, @@ -810,6 +809,9 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c /// Print overall profiling info. NOTE: it may duplicates previous messages { + ProfileEvents::increment(ProfileEvents::MergedColumns, global_ctx->merging_columns.size()); + ProfileEvents::increment(ProfileEvents::GatheredColumns, global_ctx->gathering_columns.size()); + double elapsed_seconds = global_ctx->merge_list_element_ptr->watch.elapsedSeconds(); LOG_DEBUG(ctx->log, "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", @@ -1021,8 +1023,8 @@ bool MergeTask::execute() /// Stage is finished, need to initialize context for the next stage and update profile events. UInt64 current_elapsed_ms = global_ctx->merge_list_element_ptr->watch.elapsedMilliseconds(); - UInt64 stage_elapsed_ms = current_elapsed_ms - global_ctx->prev_elapesed_ms; - global_ctx->prev_elapesed_ms = current_elapsed_ms; + UInt64 stage_elapsed_ms = current_elapsed_ms - global_ctx->prev_elapsed_ms; + global_ctx->prev_elapsed_ms = current_elapsed_ms; ProfileEvents::increment(current_stage->getTotalTimeProfileEvent(), stage_elapsed_ms); ProfileEvents::increment(ProfileEvents::MergeTotalMilliseconds, stage_elapsed_ms); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 979c85482e5..38ccc287187 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -203,7 +203,7 @@ private: bool need_prefix; scope_guard temporary_directory_lock; - UInt64 prev_elapesed_ms{0}; + UInt64 prev_elapsed_ms{0}; }; using GlobalRuntimeContextPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 73084f487b9..56f68fd265a 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -254,6 +254,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); + mutate_task->updateProfileEvents(); write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) @@ -281,6 +282,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit */ finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMutations); + mutate_task->updateProfileEvents(); write_part_log({}); return true; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 9aec074deae..10461eb5942 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -102,6 +102,7 @@ bool MutatePlainMergeTreeTask::executeStep() transaction.commit(); storage.updateMutationEntriesErrors(future_part, true, ""); + mutate_task->updateProfileEvents(); write_part_log({}); state = State::NEED_FINISH; @@ -114,6 +115,7 @@ bool MutatePlainMergeTreeTask::executeStep() PreformattedMessage exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); LOG_ERROR(getLogger("MutatePlainMergeTreeTask"), exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message.text); + mutate_task->updateProfileEvents(); write_part_log(ExecutionStatus::fromCurrentException("", true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index fe78964a241..f4af38d3b67 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -40,7 +40,10 @@ namespace ProfileEvents { extern const Event MutationTotalParts; extern const Event MutationUntouchedParts; - extern const Event MutationTimeMilliseconds; + extern const Event MutationTotalMilliseconds; + extern const Event MutationExecuteMilliseconds; + extern const Event MutationAllPartColumns; + extern const Event MutationSomePartColumns; extern const Event MutateTaskProjectionsCalculationMicroseconds; } @@ -1049,6 +1052,7 @@ struct MutationContext /// Whether we need to count lightweight delete rows in this mutation bool count_lightweight_deleted_rows; + UInt64 execute_elapsed_ns = 0; }; using MutationContextPtr = std::shared_ptr; @@ -2020,6 +2024,9 @@ MutateTask::MutateTask( bool MutateTask::execute() { + Stopwatch watch; + SCOPE_EXIT({ ctx->execute_elapsed_ns += watch.elapsedNanoseconds(); }); + switch (state) { case State::NEED_PREPARE: @@ -2037,9 +2044,6 @@ bool MutateTask::execute() if (task->executeStep()) return true; - auto total_elapsed_ms = (*ctx->mutate_entry)->watch.elapsedMilliseconds(); - ProfileEvents::increment(ProfileEvents::MutationTimeMilliseconds, total_elapsed_ms); - // The `new_data_part` is a shared pointer and must be moved to allow // part deletion in case it is needed in `MutateFromLogEntryTask::finalize`. // @@ -2056,6 +2060,15 @@ bool MutateTask::execute() return false; } +void MutateTask::updateProfileEvents() const +{ + UInt64 total_elapsed_ms = (*ctx->mutate_entry)->watch.elapsedMilliseconds(); + UInt64 execute_elapsed_ms = ctx->execute_elapsed_ns / 1000000UL; + + ProfileEvents::increment(ProfileEvents::MutationTotalMilliseconds, total_elapsed_ms); + ProfileEvents::increment(ProfileEvents::MutationExecuteMilliseconds, execute_elapsed_ms); +} + static bool canSkipConversionToNullable(const MergeTreeDataPartPtr & part, const MutationCommand & command) { if (command.type != MutationCommand::READ_COLUMN) @@ -2118,13 +2131,13 @@ static bool canSkipMutationCommandForPart(const MergeTreeDataPartPtr & part, con bool MutateTask::prepare() { + ProfileEvents::increment(ProfileEvents::MutationTotalParts); MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry); if (ctx->future_part->parts.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to mutate {} parts, not one. " "This is a bug.", ctx->future_part->parts.size()); - ProfileEvents::increment(ProfileEvents::MutationTotalParts); ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); auto context_for_reading = Context::createCopy(ctx->context); @@ -2291,6 +2304,7 @@ bool MutateTask::prepare() ctx->new_data_part->remove_tmp_policy = IMergeTreeDataPart::BlobsRemovalPolicyForTemporaryParts::REMOVE_BLOBS; task = std::make_unique(ctx); + ProfileEvents::increment(ProfileEvents::MutationAllPartColumns); } else /// TODO: check that we modify only non-key columns in this case. { @@ -2330,6 +2344,7 @@ bool MutateTask::prepare() ctx->new_data_part->remove_tmp_policy = IMergeTreeDataPart::BlobsRemovalPolicyForTemporaryParts::ASK_KEEPER; task = std::make_unique(ctx); + ProfileEvents::increment(ProfileEvents::MutationSomePartColumns); } return true; diff --git a/src/Storages/MergeTree/MutateTask.h b/src/Storages/MergeTree/MutateTask.h index dc22b90f0e9..08427bff6d8 100644 --- a/src/Storages/MergeTree/MutateTask.h +++ b/src/Storages/MergeTree/MutateTask.h @@ -39,6 +39,7 @@ public: bool need_prefix_); bool execute(); + void updateProfileEvents() const; std::future getFuture() { diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.sql b/tests/queries/0_stateless/02378_part_log_profile_events.sql index 38d3f8b4c05..eec76d6f50e 100644 --- a/tests/queries/0_stateless/02378_part_log_profile_events.sql +++ b/tests/queries/0_stateless/02378_part_log_profile_events.sql @@ -39,7 +39,7 @@ SYSTEM FLUSH LOGS; SELECT if(count() == 2, 'Ok', 'Error: ' || toString(count())), - if(SUM(ProfileEvents['MergedRows']) == 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergedRows']))), + if(SUM(ProfileEvents['MutatedRows']) == 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MutatedRows']))), if(SUM(ProfileEvents['FileOpen']) > 1, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['FileOpen']))) FROM system.part_log WHERE event_time > now() - INTERVAL 10 MINUTE diff --git a/tests/queries/0_stateless/03221_merge_profile_events.reference b/tests/queries/0_stateless/03221_merge_profile_events.reference new file mode 100644 index 00000000000..729e53eae79 --- /dev/null +++ b/tests/queries/0_stateless/03221_merge_profile_events.reference @@ -0,0 +1,3 @@ +Horizontal 1 20000 3 0 480000 1 1 1 1 +Vertical 1 20000 1 2 480000 1 1 1 1 1 1 +Vertical 2 20020 4 2 480660 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03221_merge_profile_events.sql b/tests/queries/0_stateless/03221_merge_profile_events.sql new file mode 100644 index 00000000000..787aff93ffc --- /dev/null +++ b/tests/queries/0_stateless/03221_merge_profile_events.sql @@ -0,0 +1,88 @@ +-- Tags: no-random-settings, no-random-merge-tree-settings + +DROP TABLE IF EXISTS t_merge_profile_events_1; + +CREATE TABLE t_merge_profile_events_1 (id UInt64, v1 UInt64, v2 UInt64) +ENGINE = MergeTree ORDER BY id +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO t_merge_profile_events_1 SELECT number, number, number FROM numbers(10000); +INSERT INTO t_merge_profile_events_1 SELECT number, number, number FROM numbers(10000); + +OPTIMIZE TABLE t_merge_profile_events_1 FINAL; +SYSTEM FLUSH LOGS; + +SELECT + merge_algorithm, + ProfileEvents['Merge'], + ProfileEvents['MergedRows'], + ProfileEvents['MergedColumns'], + ProfileEvents['GatheredColumns'], + ProfileEvents['MergedUncompressedBytes'], + ProfileEvents['MergeTotalMilliseconds'] > 0, + ProfileEvents['MergeExecuteMilliseconds'] > 0, + ProfileEvents['MergeHorizontalStageTotalMilliseconds'] > 0, + ProfileEvents['MergeHorizontalStageExecuteMilliseconds'] > 0 +FROM system.part_log WHERE database = currentDatabase() AND table = 't_merge_profile_events_1' AND event_type = 'MergeParts' AND part_name = 'all_1_2_1'; + +DROP TABLE IF EXISTS t_merge_profile_events_1; + +DROP TABLE IF EXISTS t_merge_profile_events_2; + +CREATE TABLE t_merge_profile_events_2 (id UInt64, v1 UInt64, v2 UInt64) +ENGINE = MergeTree ORDER BY id +SETTINGS min_bytes_for_wide_part = 0, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; + +INSERT INTO t_merge_profile_events_2 SELECT number, number, number FROM numbers(10000); +INSERT INTO t_merge_profile_events_2 SELECT number, number, number FROM numbers(10000); + +OPTIMIZE TABLE t_merge_profile_events_2 FINAL; +SYSTEM FLUSH LOGS; + +SELECT + merge_algorithm, + ProfileEvents['Merge'], + ProfileEvents['MergedRows'], + ProfileEvents['MergedColumns'], + ProfileEvents['GatheredColumns'], + ProfileEvents['MergedUncompressedBytes'], + ProfileEvents['MergeTotalMilliseconds'] > 0, + ProfileEvents['MergeExecuteMilliseconds'] > 0, + ProfileEvents['MergeHorizontalStageTotalMilliseconds'] > 0, + ProfileEvents['MergeHorizontalStageExecuteMilliseconds'] > 0, + ProfileEvents['MergeVerticalStageTotalMilliseconds'] > 0, + ProfileEvents['MergeVerticalStageExecuteMilliseconds'] > 0, +FROM system.part_log WHERE database = currentDatabase() AND table = 't_merge_profile_events_2' AND event_type = 'MergeParts' AND part_name = 'all_1_2_1'; + +DROP TABLE IF EXISTS t_merge_profile_events_2; + +DROP TABLE IF EXISTS t_merge_profile_events_3; + +CREATE TABLE t_merge_profile_events_3 (id UInt64, v1 UInt64, v2 UInt64, PROJECTION p (SELECT sum(v1), sum(v2) GROUP BY id % 10)) +ENGINE = MergeTree ORDER BY id +SETTINGS min_bytes_for_wide_part = 0, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; + +INSERT INTO t_merge_profile_events_3 SELECT number, number, number FROM numbers(10000); +INSERT INTO t_merge_profile_events_3 SELECT number, number, number FROM numbers(10000); + +OPTIMIZE TABLE t_merge_profile_events_3 FINAL; +SYSTEM FLUSH LOGS; + +SELECT + merge_algorithm, + ProfileEvents['Merge'], + ProfileEvents['MergedRows'], + ProfileEvents['MergedColumns'], + ProfileEvents['GatheredColumns'], + ProfileEvents['MergedUncompressedBytes'], + ProfileEvents['MergeTotalMilliseconds'] > 0, + ProfileEvents['MergeExecuteMilliseconds'] > 0, + ProfileEvents['MergeHorizontalStageTotalMilliseconds'] > 0, + ProfileEvents['MergeHorizontalStageExecuteMilliseconds'] > 0, + ProfileEvents['MergeVerticalStageTotalMilliseconds'] > 0, + ProfileEvents['MergeVerticalStageExecuteMilliseconds'] > 0, + ProfileEvents['MergeProjectionStageTotalMilliseconds'] > 0, + ProfileEvents['MergeProjectionStageExecuteMilliseconds'] > 0, +FROM system.part_log WHERE database = currentDatabase() AND table = 't_merge_profile_events_3' AND event_type = 'MergeParts' AND part_name = 'all_1_2_1'; + +DROP TABLE IF EXISTS t_merge_profile_events_3; diff --git a/tests/queries/0_stateless/03221_mutate_profile_events.reference b/tests/queries/0_stateless/03221_mutate_profile_events.reference new file mode 100644 index 00000000000..d094c37ff88 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutate_profile_events.reference @@ -0,0 +1,2 @@ +3 2 1 10000 160000 0 1 1 1 +4 2 1 10000 320000 1 0 1 1 diff --git a/tests/queries/0_stateless/03221_mutate_profile_events.sql b/tests/queries/0_stateless/03221_mutate_profile_events.sql new file mode 100644 index 00000000000..e9f7f9670bd --- /dev/null +++ b/tests/queries/0_stateless/03221_mutate_profile_events.sql @@ -0,0 +1,33 @@ +-- Tags: no-random-settings, no-random-merge-tree-settings + +DROP TABLE IF EXISTS t_mutate_profile_events; + +CREATE TABLE t_mutate_profile_events (key UInt64, id UInt64, v1 UInt64, v2 UInt64) +ENGINE = MergeTree ORDER BY id PARTITION BY key +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO t_mutate_profile_events SELECT 1, number, number, number FROM numbers(10000); +INSERT INTO t_mutate_profile_events SELECT 2, number, number, number FROM numbers(10000); + +SET mutations_sync = 2; + +ALTER TABLE t_mutate_profile_events UPDATE v1 = 1000 WHERE key = 1; +ALTER TABLE t_mutate_profile_events DELETE WHERE key = 2 AND v2 % 10 = 0; + +SYSTEM FLUSH LOGS; + +SELECT + splitByChar('_', part_name)[-1] AS version, + sum(ProfileEvents['MutationTotalParts']), + sum(ProfileEvents['MutationUntouchedParts']), + sum(ProfileEvents['MutatedRows']), + sum(ProfileEvents['MutatedUncompressedBytes']), + sum(ProfileEvents['MutationAllPartColumns']), + sum(ProfileEvents['MutationSomePartColumns']), + sum(ProfileEvents['MutationTotalMilliseconds']) > 0, + sum(ProfileEvents['MutationExecuteMilliseconds']) > 0, +FROM system.part_log +WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart' +GROUP BY version ORDER BY version; + +DROP TABLE IF EXISTS t_mutate_profile_events From 4b7080f2cd4881fa2c369e4a05a3904744d23867 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 16:49:33 +0200 Subject: [PATCH 0632/1722] Parallelize 01600_detach_permanently --- tests/queries/0_stateless/01600_detach_permanently.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index be405f8e7d9..4810d2e9732 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-fasttest +# no-fasttest: It's a bit demanding # Creation of a database with Ordinary engine emits a warning. CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal @@ -9,9 +10,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ## tests with real clickhouse restart would be a bit to heavy, -## to ensure the table will not reappear back clickhose-local is enough. +## to ensure the table will not reappear back clickhouse-local is enough. -WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/01600_detach_permanently" +WORKING_FOLDER_01600="${CLICKHOUSE_TMP}/${CLICKHOUSE_TEST_UNIQUE_NAME}" rm -rf "${WORKING_FOLDER_01600}" mkdir -p "${WORKING_FOLDER_01600}" From 453b62b692a9aa2a623e737a43002f1534182a43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 16:53:07 +0200 Subject: [PATCH 0633/1722] 02455_one_row_from_csv_memory_usage is parallelizable --- .../queries/0_stateless/02455_one_row_from_csv_memory_usage.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 7906f2917c4..1a0bc69f2e8 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz +ln -s "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz ${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=1000000000" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=100000000" From 09627d9a0910279eece78f4471142dbbf6048f7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 17:01:38 +0200 Subject: [PATCH 0634/1722] Make 01038_dictionary_lifetime_min_zero_sec parallelizable --- .../0_stateless/01038_dictionary_lifetime_min_zero_sec.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh index 66732205f95..8e15453cbe5 100755 --- a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh +++ b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -42,6 +41,7 @@ function check() while [ "$query_result" != "2.2" ] do + sleep 0.2 query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))") done } From a53050d45455bb5159f752527dc40ec0ef53d6bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Aug 2024 17:01:46 +0200 Subject: [PATCH 0635/1722] Add some comments --- .../01360_materialized_view_with_join_on_query_log.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql index 1b22082bd83..b9ad132bcd2 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel +-- no-parallel: Slows down query_log DROP TABLE IF EXISTS slow_log; DROP TABLE IF EXISTS expected_times; From 34bbf3d9c49f0f563eee8973ac3c3bae0e64c6de Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Aug 2024 17:12:27 +0200 Subject: [PATCH 0636/1722] Use async inserts --- docker/test/stateless/run.sh | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 830a02a64a3..c81f33ace01 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -192,8 +192,9 @@ ENGINE = MergeTree ORDER BY tuple()" # create minio log webhooks for both audit and server logs -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" +# use async inserts to avoid creating too many parts +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" max_retries=100 retry=1 @@ -375,6 +376,12 @@ done # collect minio audit and server logs +has_async_inserts=$(clickhouse-client -q "SELECT count() FROM system.asynchronous_inserts WHERE table = 'minio_audit_logs' OR table = 'minio_server_logs'") +if [[ has_async_inserts -eq 1 ]]; then + echo "Waiting for async inserts to flush" + sleep 5 +fi + clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" From 33ba78ee42bc85690dce69c82fd51d723a6d2eab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 8 Aug 2024 17:47:12 +0200 Subject: [PATCH 0637/1722] Update test.py --- tests/integration/test_drop_is_lock_free/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_drop_is_lock_free/test.py b/tests/integration/test_drop_is_lock_free/test.py index 1bb8767a9a0..3855bc21f90 100644 --- a/tests/integration/test_drop_is_lock_free/test.py +++ b/tests/integration/test_drop_is_lock_free/test.py @@ -176,7 +176,7 @@ def test_query_is_permanent(transaction, permanent, exclusive_table): select_handler = node.get_query_request( f""" - SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0; + SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0, max_threads=1; """, query_id=query_id, ) From e264ecd2011c7860c4898208a53555d676222bbb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 16:10:46 +0000 Subject: [PATCH 0638/1722] fix skip of parts in mutation with analyzer --- src/Interpreters/MutationsInterpreter.cpp | 11 +++++----- src/Interpreters/MutationsInterpreter.h | 1 - src/Storages/MergeTree/MutateTask.cpp | 2 +- ...3221_mutation_analyzer_skip_part.reference | 4 ++++ .../03221_mutation_analyzer_skip_part.sql | 21 +++++++++++++++++++ 5 files changed, 31 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference create mode 100644 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 57ad5caa4c7..5de33b3ed86 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -146,7 +146,6 @@ ColumnDependencies getAllColumnDependencies( bool isStorageTouchedByMutations( - MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, @@ -155,7 +154,9 @@ bool isStorageTouchedByMutations( if (commands.empty()) return false; + auto storage_from_part = std::make_shared(source_part); bool all_commands_can_be_skipped = true; + for (const auto & command : commands) { if (command.type == MutationCommand::APPLY_DELETED_MASK) @@ -170,7 +171,7 @@ bool isStorageTouchedByMutations( if (command.partition) { - const String partition_id = storage.getPartitionIDFromQuery(command.partition, context); + const String partition_id = storage_from_part->getPartitionIDFromQuery(command.partition, context); if (partition_id == source_part->info.partition_id) all_commands_can_be_skipped = false; } @@ -184,20 +185,18 @@ bool isStorageTouchedByMutations( if (all_commands_can_be_skipped) return false; - auto storage_from_part = std::make_shared(source_part); - std::optional interpreter_select_query; BlockIO io; if (context->getSettingsRef().allow_experimental_analyzer) { - auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context); + auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage_from_part, context); InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits()); io = interpreter.execute(); } else { - ASTPtr select_query = prepareQueryAffectedAST(commands, storage.shared_from_this(), context); + ASTPtr select_query = prepareQueryAffectedAST(commands, storage_from_part, context); /// Interpreter must be alive, when we use result of execute() method. /// For some reason it may copy context and give it into ExpressionTransform /// after that we will use context from destroyed stack frame in our stream. diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6aaa233cda3..57863e9ae73 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -19,7 +19,6 @@ using QueryPipelineBuilderPtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( - MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9a775db73e2..5fcf699de59 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2134,7 +2134,7 @@ bool MutateTask::prepare() ctx->commands_for_part.emplace_back(command); if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) + ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) { NameSet files_to_copy_instead_of_hardlinks; auto settings_ptr = ctx->data->getSettings(); diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference new file mode 100644 index 00000000000..68f8708dbaf --- /dev/null +++ b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.reference @@ -0,0 +1,4 @@ +1_1_1_0_3 10000 +1_1_1_0_4 0 +2_2_2_0_3 0 +2_2_2_0_4 10000 diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql new file mode 100644 index 00000000000..bf9a10e2af4 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS t_mutate_skip_part; + +CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) ENGINE = MergeTree ORDER BY id PARTITION BY key; + +INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000); +INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000); + +SET mutations_sync = 2; + +ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1; +ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0; + +SYSTEM FLUSH LOGS; + +-- If part is skipped in mutation and hardlinked then read_rows must be 0. +SELECT part_name, read_rows +FROM system.part_log +WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart' +ORDER BY part_name; + +DROP TABLE IF EXISTS t_mutate_skip_part; From f9f13a8e415e9f2130281d069c28dd6e9a68be75 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 16:27:25 +0000 Subject: [PATCH 0639/1722] enable setting optimize_functions_to_subcolumns by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f04c696645a..f0a8d0c2647 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -605,7 +605,7 @@ class IColumn; M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_multiif_to_if, true, "Replace 'multiIf' with only one condition to 'if'.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ - M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ + M(Bool, optimize_functions_to_subcolumns, true, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 41e4ac2e154..b00d0964e01 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -83,6 +83,7 @@ static std::initializer_list Date: Thu, 8 Aug 2024 16:32:40 +0000 Subject: [PATCH 0640/1722] fix build --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 18cd6f2c1dd..37a59576edd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1561,12 +1561,8 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) mutations_snapshot, context, query_info, -<<<<<<< HEAD - metadata_for_reading, + storage_snapshot->metadata, log); -======= - storage_snapshot->metadata); ->>>>>>> upstream/master } } From d52d599af4db1c779551b6788a5505e521fe3c31 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 18:34:02 +0200 Subject: [PATCH 0641/1722] Annotations --- ...ter_skip_virtual_columns_with_non_deterministic_functions.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql index 6ef8c5a8656..6714a069246 100644 --- a/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql +++ b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql @@ -1,3 +1,4 @@ +-- Tags: long SET max_rows_to_read = 0; create table test (number UInt64) engine=MergeTree order by number; insert into test select * from numbers(50000000); From cd69fa5a4c6f7128c473c780c797d60dc314b2ee Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 8 Aug 2024 11:20:55 -0600 Subject: [PATCH 0642/1722] fixed typos --- docs/en/sql-reference/functions/geo/polygon.md | 2 +- src/Functions/geometryConverters.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index c054e05d39c..be9e9810626 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -197,7 +197,7 @@ The function returns a ClickHouse internal representation of the multilinestring ### Example ```sql -SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')); +SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))'); ``` ```response diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index bf975017a6d..f1156d81f01 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -471,7 +471,7 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) return f(ConverterType>()); /// We should take the name into consideration to avoid ambiguity. - /// Because for example both MultiLineString and Polygon are resolved to Array(Tuple(Point)). + /// Because for example both MultiLineString and Polygon are resolved to Array(Array(Point)). else if (factory.get("MultiLineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "MultiLineString") return f(ConverterType>()); From e55d81fd427af0a62e3534d838d64eb847419248 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 8 Aug 2024 19:38:48 +0200 Subject: [PATCH 0643/1722] fix test --- .../0_stateless/02808_custom_disk_with_user_defined_name.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index b62adea5683..63fa60bd548 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS disk = disk(name = 's3_disk', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); -""" 2>&1 | grep -q "The disk \`s3_disk\` is already exist and described by the config" && echo 'OK' || echo 'FAIL' +""" 2>&1 | grep -q "Disk \`s3_disk\` already exists and is described by the config" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" From 2bd7606280b0714febc1d034233ee0fb9a3532a6 Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 8 Aug 2024 11:39:06 -0600 Subject: [PATCH 0644/1722] added additional tests --- .../03215_multilinestring_geometry.reference | 10 ++++++++++ .../0_stateless/03215_multilinestring_geometry.sql | 12 ++++++++++++ 2 files changed, 22 insertions(+) diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.reference b/tests/queries/0_stateless/03215_multilinestring_geometry.reference index f4c5774018e..f20e21d86f6 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.reference +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.reference @@ -15,3 +15,13 @@ MULTILINESTRING((1 1,2 2,3 3,1 1),(1 0,2 0,3 0)) WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); POLYGON((1 1,2 2,3 3,1 1)) String [[(1,1),(2,2),(3,3),(1,1)]] Polygon +-- Non constant tests + +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +-- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; +POLYGON((1 1,2 2,3 3,1 1)) [[(1,1),(2,2),(3,3),(1,1)]] 1 +POLYGON((1 1,2 2,3 3,1 1),(1 0,2 0,3 0,1 0)) [[(1,1),(2,2),(3,3),(1,1)],[(1,0),(2,0),(3,0)]] 1 diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.sql b/tests/queries/0_stateless/03215_multilinestring_geometry.sql index 71344920c52..6081e081fb3 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.sql +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.sql @@ -10,3 +10,15 @@ SELECT wkt(readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, -- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. WITH wkt(CAST([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'Array(Array(Tuple(Float64, Float64)))')) as x SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); + +-- Non constant tests + +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); + +-- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. +-- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; + + From f2731841de804c30ece1c75e84c8ca8d3eb62ef8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 20:20:21 +0200 Subject: [PATCH 0645/1722] init --- src/Core/callOnTypeIndex.h | 3 + src/DataTypes/getLeastSupertype.cpp | 41 +++++ src/DataTypes/getLeastSupertype.h | 22 +++ src/Functions/FunctionsConversion.cpp | 61 +++++++- ...23_interval_data_type_comparison.reference | 99 ++++++++++++ .../03223_interval_data_type_comparison.sql | 142 ++++++++++++++++++ 6 files changed, 365 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03223_interval_data_type_comparison.reference create mode 100644 tests/queries/0_stateless/03223_interval_data_type_comparison.sql diff --git a/src/Core/callOnTypeIndex.h b/src/Core/callOnTypeIndex.h index f5f67df563b..ae5afce36be 100644 --- a/src/Core/callOnTypeIndex.h +++ b/src/Core/callOnTypeIndex.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB @@ -212,6 +213,8 @@ static bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... ar case TypeIndex::IPv4: return f(TypePair(), std::forward(args)...); case TypeIndex::IPv6: return f(TypePair(), std::forward(args)...); + case TypeIndex::Interval: return f(TypePair(), std::forward(args)...); + default: break; } diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index a71b19d6c92..0b9c744c091 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -228,6 +228,40 @@ void convertUInt64toInt64IfPossible(const DataTypes & types, TypeIndexSet & type } } +DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet &types_set) +{ + const auto& granularity_map = getGranularityMap(); + int min_granularity = std::get<0>(granularity_map.at(IntervalKind::Kind::Year)); + DataTypePtr smallest_type; + + bool is_higher_interval = false; // For Years, Quarters and Months + + for (const auto &type : types) + { + if (const auto * interval_type = typeid_cast(type.get())) + { + int current_granularity = std::get<0>(granularity_map.at(interval_type->getKind())); + if (current_granularity > 8) + is_higher_interval = true; + if (current_granularity < min_granularity) + { + min_granularity = current_granularity; + smallest_type = type; + } + } + } + + if (is_higher_interval && min_granularity <= 8) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); + + if (smallest_type) + { + types_set.clear(); + types_set.insert(smallest_type->getTypeId()); + } + + return smallest_type; +} } template @@ -652,6 +686,13 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return numeric_type; } + /// For interval data types. + { + auto res = findSmallestIntervalSuperType(types, type_ids); + if (res) + return res; + } + /// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases). return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); } diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 2ae1e52ca96..c584eb83011 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -1,5 +1,7 @@ #pragma once #include +#include +#include namespace DB { @@ -48,4 +50,24 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types); DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); +/// A map that enumerated all interval kinds in ascending order with a conversion value to a next interval +inline const std::unordered_map> & getGranularityMap() +{ + static std::unordered_map> granularity_map = + { + {IntervalKind::Kind::Nanosecond, {1, 1000}}, + {IntervalKind::Kind::Microsecond, {2, 1000}}, + {IntervalKind::Kind::Millisecond, {3, 1000}}, + {IntervalKind::Kind::Second, {4, 60}}, + {IntervalKind::Kind::Minute, {5, 60}}, + {IntervalKind::Kind::Hour, {6, 24}}, + {IntervalKind::Kind::Day, {7, 7}}, + {IntervalKind::Kind::Week, {8, 4}}, + {IntervalKind::Kind::Month, {9, 3}}, + {IntervalKind::Kind::Quarter, {10, 4}}, + {IntervalKind::Kind::Year, {11, 1}} + }; + return granularity_map; +} + } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..0ab1858dc97 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -1573,6 +1574,55 @@ struct ConvertImpl arguments, result_type, input_rows_count, additions); } } + else if constexpr (std::is_same_v && std::is_same_v) + { + IntervalKind to = typeid_cast(result_type.get())->getKind(); + IntervalKind from = typeid_cast(arguments[0].type.get())->getKind(); + + if (from == to) + return arguments[0].column; + + const auto &map = getGranularityMap(); + Int64 conversion_factor = 1; + Int64 result_value; + + int from_position = map.at(from).first; + int to_position = map.at(to).first; // Positions of each interval according to granurality map + + if (from_position < to_position) + { + for (int i = from_position - 1; i <= to_position; ++i) + { + // Find the kind that matches this position + for (const auto &entry : map) + { + if (entry.second.first == i) + { + conversion_factor *= entry.second.second; + break; + } + } + } + result_value = arguments[0].column->getInt(0) / conversion_factor; + } + else + { + for (int i = from_position - 1; i >= to_position; --i) + { + for (const auto &entry : map) + { + if (entry.second.first == i) + { + conversion_factor *= entry.second.second; + break; + } + } + } + result_value = arguments[0].column->getInt(0) * conversion_factor; + } + + return ColumnConst::create(ColumnInt64::create(1, result_value), input_rows_count); + } else { using FromFieldType = typename FromDataType::FieldType; @@ -2181,7 +2231,7 @@ private: const DataTypePtr from_type = removeNullable(arguments[0].type); ColumnPtr result_column; - [[maybe_unused]] FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; if (context) date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior.value; @@ -2277,7 +2327,7 @@ private: } } else - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); return true; }; @@ -2334,6 +2384,11 @@ private: else done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); } + + if constexpr (std::is_same_v) + { + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); + } } if (!done) @@ -5224,7 +5279,7 @@ REGISTER_FUNCTION(Conversion) /// MySQL compatibility alias. Cannot be registered as alias, /// because we don't want it to be normalized to toDate in queries, /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); + factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.reference b/tests/queries/0_stateless/03223_interval_data_type_comparison.reference new file mode 100644 index 00000000000..e98f792e4b2 --- /dev/null +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.reference @@ -0,0 +1,99 @@ +Comparing nanoseconds +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +Comparing microseconds +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +0 +Comparing milliseconds +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +Comparing seconds +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +Comparing minutes +1 +1 +1 +1 +0 +0 +0 +0 +Comparing hours +1 +1 +1 +0 +0 +0 +Comparing days +1 +1 +0 +0 +Comparing weeks +1 +0 +Comparing months +1 +1 +1 +0 +0 +0 +Comparing quarters +1 +1 +0 +0 +Comparing years +1 +0 diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql new file mode 100644 index 00000000000..6e4862bf2d2 --- /dev/null +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql @@ -0,0 +1,142 @@ +SELECT('Comparing nanoseconds'); +SELECT toIntervalNanosecond(500) > toIntervalNanosecond(300); +SELECT toIntervalNanosecond(1000) < toIntervalNanosecond(1500); +SELECT toIntervalNanosecond(2000) = toIntervalNanosecond(2000); +SELECT toIntervalNanosecond(1000) >= toIntervalMicrosecond(1); +SELECT toIntervalNanosecond(1000001) > toIntervalMillisecond(1); +SELECT toIntervalNanosecond(2000000001) > toIntervalSecond(2); +SELECT toIntervalNanosecond(60000000000) = toIntervalMinute(1); +SELECT toIntervalNanosecond(7199999999999) < toIntervalHour(2); +SELECT toIntervalNanosecond(1) < toIntervalDay(2); +SELECT toIntervalNanosecond(5) < toIntervalWeek(1); + +SELECT toIntervalNanosecond(500) < toIntervalNanosecond(300); +SELECT toIntervalNanosecond(1000) > toIntervalNanosecond(1500); +SELECT toIntervalNanosecond(2000) != toIntervalNanosecond(2000); +SELECT toIntervalNanosecond(1000) < toIntervalMicrosecond(1); +SELECT toIntervalNanosecond(1000001) < toIntervalMillisecond(1); +SELECT toIntervalNanosecond(2000000001) < toIntervalSecond(2); +SELECT toIntervalNanosecond(60000000000) != toIntervalMinute(1); +SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2); +SELECT toIntervalNanosecond(1) > toIntervalDay(2); +SELECT toIntervalNanosecond(5) > toIntervalWeek(1); + +SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing microseconds'); +SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999); +SELECT toIntervalMicrosecond(1001) > toIntervalMillisecond(1); +SELECT toIntervalMicrosecond(2000000) = toIntervalSecond(2); +SELECT toIntervalMicrosecond(179999999) < toIntervalMinute(3); +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); +SELECT toIntervalMicrosecond(36000000000000) > toIntervalDay(2); +SELECT toIntervalMicrosecond(1209600000000) = toIntervalWeek(2); + +SELECT toIntervalMicrosecond(1) > toIntervalMicrosecond(999); +SELECT toIntervalMicrosecond(1001) < toIntervalMillisecond(1); +SELECT toIntervalMicrosecond(2000000) != toIntervalSecond(2); +SELECT toIntervalMicrosecond(179999999) > toIntervalMinute(3); +SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1); +SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2); +SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2); + +SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing milliseconds'); +SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2); +SELECT toIntervalMillisecond(2000) = toIntervalSecond(2); +SELECT toIntervalMillisecond(170000) < toIntervalMinute(3); +SELECT toIntervalMillisecond(144000001) > toIntervalHour(40); +SELECT toIntervalMillisecond(1728000000) = toIntervalDay(20); +SELECT toIntervalMillisecond(1198599999) < toIntervalWeek(2); + +SELECT toIntervalMillisecond(2000) < toIntervalMillisecond(2); +SELECT toIntervalMillisecond(2000) != toIntervalSecond(2); +SELECT toIntervalMillisecond(170000) > toIntervalMinute(3); +SELECT toIntervalMillisecond(144000001) < toIntervalHour(40); +SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20); +SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2); + +SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing seconds'); +SELECT toIntervalSecond(120) > toIntervalSecond(2); +SELECT toIntervalSecond(120) = toIntervalMinute(2); +SELECT toIntervalSecond(1) < toIntervalHour(2); +SELECT toIntervalSecond(86401) >= toIntervalDay(1); +SELECT toIntervalSecond(1209600) = toIntervalWeek(2); + +SELECT toIntervalSecond(120) < toIntervalSecond(2); +SELECT toIntervalSecond(120) != toIntervalMinute(2); +SELECT toIntervalSecond(1) > toIntervalHour(2); +SELECT toIntervalSecond(86401) < toIntervalDay(1); +SELECT toIntervalSecond(1209600) != toIntervalWeek(2); + +SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing minutes'); +SELECT toIntervalMinute(1) < toIntervalMinute(59); +SELECT toIntervalMinute(1) < toIntervalHour(59); +SELECT toIntervalMinute(1440) = toIntervalDay(1); +SELECT toIntervalMinute(30241) > toIntervalWeek(3); + +SELECT toIntervalMinute(1) > toIntervalMinute(59); +SELECT toIntervalMinute(1) > toIntervalHour(59); +SELECT toIntervalMinute(1440) != toIntervalDay(1); +SELECT toIntervalMinute(30241) < toIntervalWeek(3); + +SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing hours'); +SELECT toIntervalHour(48) > toIntervalHour(2); +SELECT toIntervalHour(48) >= toIntervalDay(2); +SELECT toIntervalHour(672) = toIntervalWeek(4); + +SELECT toIntervalHour(48) < toIntervalHour(2); +SELECT toIntervalHour(48) < toIntervalDay(2); +SELECT toIntervalHour(672) != toIntervalWeek(4); + +SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing days'); +SELECT toIntervalDay(1) < toIntervalDay(23); +SELECT toIntervalDay(25) > toIntervalWeek(3); + +SELECT toIntervalDay(1) > toIntervalDay(23); +SELECT toIntervalDay(25) < toIntervalWeek(3); + +SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing weeks'); +SELECT toIntervalWeek(1) < toIntervalWeek(6); + +SELECT toIntervalWeek(1) > toIntervalWeek(6); + +SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing months'); +SELECT toIntervalMonth(1) < toIntervalMonth(3); +SELECT toIntervalMonth(124) > toIntervalQuarter(5); +SELECT toIntervalMonth(36) = toIntervalYear(3); + +SELECT toIntervalMonth(1) > toIntervalMonth(3); +SELECT toIntervalMonth(124) < toIntervalQuarter(5); +SELECT toIntervalMonth(36) != toIntervalYear(3); + +SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing quarters'); +SELECT toIntervalQuarter(5) > toIntervalQuarter(4); +SELECT toIntervalQuarter(20) = toIntervalYear(5); + +SELECT toIntervalQuarter(5) < toIntervalQuarter(4); +SELECT toIntervalQuarter(20) != toIntervalYear(5); + +SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT('Comparing years'); +SELECT toIntervalYear(1) < toIntervalYear(3); + +SELECT toIntervalYear(1) > toIntervalYear(3); + +SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From b0b150c599052d596b8c746a31ba50019601b6a0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 18:29:20 +0000 Subject: [PATCH 0646/1722] Remove workarounds for solved bugs --- .../02864_statistics_predicates.reference | 6 ++++++ .../0_stateless/02864_statistics_predicates.sql | 12 ++++++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference index 1c2abd47aaf..ffbd7269e05 100644 --- a/tests/queries/0_stateless/02864_statistics_predicates.reference +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -23,6 +23,7 @@ u64 and < 70 70 70 +70 f64 and = 10 10 @@ -35,6 +36,8 @@ f64 and = 10 10 10 +10 +0 0 0 0 @@ -50,6 +53,8 @@ f64 and < 70 70 70 +70 +80 80 80 80 @@ -66,6 +71,7 @@ dt and < 10000 10000 10000 +10000 70 70 70 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql index 3e754dfb1de..779116cf19a 100644 --- a/tests/queries/0_stateless/02864_statistics_predicates.sql +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -92,7 +92,7 @@ SELECT count(*) FROM tab WHERE u64_count_min < 7.7; SELECT count(*) FROM tab WHERE u64_uniq < 7.7; SELECT count(*) FROM tab WHERE u64 < '7'; --- SELECT count(*) FROM tab WHERE u64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE u64_tdigest < '7'; SELECT count(*) FROM tab WHERE u64_count_min < '7'; SELECT count(*) FROM tab WHERE u64_uniq < '7'; @@ -116,12 +116,12 @@ SELECT count(*) FROM tab WHERE f64_count_min = 7.7; SELECT count(*) FROM tab WHERE f64_uniq = 7.7; SELECT count(*) FROM tab WHERE f64 = '7'; --- SELECT count(*) FROM tab WHERE f64_tdigest = '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest = '7'; SELECT count(*) FROM tab WHERE f64_count_min = '7'; SELECT count(*) FROM tab WHERE f64_uniq = '7'; SELECT count(*) FROM tab WHERE f64 = '7.7'; --- SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; @@ -138,12 +138,12 @@ SELECT count(*) FROM tab WHERE f64_count_min < 7.7; SELECT count(*) FROM tab WHERE f64_uniq < 7.7; SELECT count(*) FROM tab WHERE f64 < '7'; --- SELECT count(*) FROM tab WHERE f64_tdigest < '7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest < '7'; SELECT count(*) FROM tab WHERE f64_count_min < '7'; SELECT count(*) FROM tab WHERE f64_uniq < '7'; SELECT count(*) FROM tab WHERE f64 < '7.7'; --- SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; @@ -164,7 +164,7 @@ SELECT count(*) FROM tab WHERE dt_uniq = 7; SELECT 'dt and <'; SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; --- SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; -- BOOM (#67742) +SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; From e9659626adc29d237d23e0f3ced9c8712d472a73 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 20:41:15 +0200 Subject: [PATCH 0647/1722] fix style + add docs --- .../data-types/special-data-types/interval.md | 21 +++++++++---------- .../data-types/special-data-types/interval.md | 21 +++++++++---------- .../data-types/special-data-types/interval.md | 21 +++++++++---------- src/DataTypes/getLeastSupertype.cpp | 2 +- 4 files changed, 31 insertions(+), 34 deletions(-) diff --git a/docs/en/sql-reference/data-types/special-data-types/interval.md b/docs/en/sql-reference/data-types/special-data-types/interval.md index bedbcf0bd28..be26053580b 100644 --- a/docs/en/sql-reference/data-types/special-data-types/interval.md +++ b/docs/en/sql-reference/data-types/special-data-types/interval.md @@ -53,29 +53,28 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -Intervals with different types can’t be combined. You can’t use intervals like `4 DAY 1 HOUR`. Specify intervals in units that are smaller or equal to the smallest unit of the interval, for example, the interval `1 day and an hour` interval can be expressed as `25 HOUR` or `90000 SECOND`. - -You can’t perform arithmetical operations with `Interval`-type values, but you can add intervals of different types consequently to values in `Date` or `DateTime` data types. For example: +Also it is possible to use multiple intervals simultaneously: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┠-│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┠+│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ +└─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -The following query causes an exception: +And to compare values with different intevals: ``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); ``` ``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. +┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┠+│ 1 │ +└─────────────────────────────────────────────────────────────┘ ``` ## See Also diff --git a/docs/ru/sql-reference/data-types/special-data-types/interval.md b/docs/ru/sql-reference/data-types/special-data-types/interval.md index 867a6665f4b..5064391f582 100644 --- a/docs/ru/sql-reference/data-types/special-data-types/interval.md +++ b/docs/ru/sql-reference/data-types/special-data-types/interval.md @@ -54,29 +54,28 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -ÐÐµÐ»ÑŒÐ·Ñ Ð¾Ð±ÑŠÐµÐ´Ð¸Ð½ÑÑ‚ÑŒ интервалы различных типов. ÐÐµÐ»ÑŒÐ·Ñ Ð¸Ñпользовать интервалы вида `4 DAY 1 HOUR`. ВмеÑто Ñтого выражайте интервал в единицах меньших или равных минимальной единице интервала, например, интервал «1 день и 1 чаÑ» можно выразить как `25 HOUR` или `90000 SECOND`. - -ÐрифметичеÑкие операции Ñо значениÑми типов `Interval` не доÑтупны, однако можно поÑледовательно добавлÑÑ‚ÑŒ различные интервалы к значениÑм типов `Date` и `DateTime`. Ðапример: +Также можно иÑпользовать различные типы интервалов одновременно: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┠-│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┠+│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ +└─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -Следующий Ð·Ð°Ð¿Ñ€Ð¾Ñ Ð¿Ñ€Ð¸Ð²ÐµÐ´Ñ‘Ñ‚ к генерированию иÑключениÑ: +И Ñравнивать Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¸Ð· разными интервалами: ``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); ``` ``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. +┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┠+│ 1 │ +└─────────────────────────────────────────────────────────────┘ ``` ## Смотрите также {#smotrite-takzhe} diff --git a/docs/zh/sql-reference/data-types/special-data-types/interval.md b/docs/zh/sql-reference/data-types/special-data-types/interval.md index e05869b2df8..e16f6d5f84f 100644 --- a/docs/zh/sql-reference/data-types/special-data-types/interval.md +++ b/docs/zh/sql-reference/data-types/special-data-types/interval.md @@ -55,29 +55,28 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -ä¸åŒç±»åž‹çš„é—´éš”ä¸èƒ½åˆå¹¶ã€‚ ä½ ä¸èƒ½ä½¿ç”¨è¯¸å¦‚ `4 DAY 1 HOUR` 的时间间隔. 以å°äºŽæˆ–等于时间间隔最å°å•ä½çš„å•ä½æ¥æŒ‡å®šé—´éš”,例如,时间间隔 `1 day and an hour` å¯ä»¥è¡¨ç¤ºä¸º `25 HOUR` 或 `90000 SECOND`. - -ä½ ä¸èƒ½å¯¹ `Interval` 类型的值执行算术è¿ç®—,但你å¯ä»¥å‘ `Date` 或 `DateTime` æ•°æ®ç±»åž‹çš„值添加ä¸åŒç±»åž‹çš„时间间隔,例如: +也å¯ä»¥åŒæ™‚使用多個間隔: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┠-│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┠+│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ +└─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -以下查询将导致异常: +並比較ä¸åŒç›´æ•¸çš„值: ``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); ``` ``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. +┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┠+│ 1 │ +└─────────────────────────────────────────────────────────────┘ ``` ## å¦è¯·å‚阅 {#see-also} diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 0b9c744c091..674284460dc 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -252,7 +252,7 @@ DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet & } if (is_higher_interval && min_granularity <= 8) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); + throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); if (smallest_type) { From fe234bd88f3eb1dca8cdb8b217606abfbcea1d54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 20:56:51 +0200 Subject: [PATCH 0648/1722] Fix test --- tests/queries/0_stateless/03218_materialize_msan.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03218_materialize_msan.sql b/tests/queries/0_stateless/03218_materialize_msan.sql index b41300ea1e3..7e7043e687b 100644 --- a/tests/queries/0_stateless/03218_materialize_msan.sql +++ b/tests/queries/0_stateless/03218_materialize_msan.sql @@ -1,3 +1,5 @@ +SET enable_analyzer = 1; + SELECT materialize([(NULL, '11\01111111\011111', '1111')]) AS t, (t[1048576]).2, From 91ff9f40a2cea46d2ddf14628b16a0ddf923a3cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Aug 2024 21:04:10 +0200 Subject: [PATCH 0649/1722] Misc --- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index 278817b1d48..8a6fa9b7845 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -4,6 +4,7 @@ set enable_filesystem_cache=0; set enable_filesystem_cache_on_write_operations=0; set max_rows_to_read = '30M'; + drop table if exists t; create table t (x UInt64, s String) engine = MergeTree order by x SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; From bc20b637eae8bba72ecfab5256c7eace40586976 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Aug 2024 21:47:23 +0200 Subject: [PATCH 0650/1722] Add missing file --- .../test_broken_projections/config.d/dont_start_broken.xml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 tests/integration/test_broken_projections/config.d/dont_start_broken.xml diff --git a/tests/integration/test_broken_projections/config.d/dont_start_broken.xml b/tests/integration/test_broken_projections/config.d/dont_start_broken.xml new file mode 100644 index 00000000000..9603cdc7e3e --- /dev/null +++ b/tests/integration/test_broken_projections/config.d/dont_start_broken.xml @@ -0,0 +1,6 @@ + + + + 0 + + From 0ad6aa09acb72a67fc88e0cd8186afd32fefd6bf Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 22:51:27 +0200 Subject: [PATCH 0651/1722] fix style --- docs/en/sql-reference/data-types/special-data-types/interval.md | 2 +- src/Functions/FunctionsConversion.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/special-data-types/interval.md b/docs/en/sql-reference/data-types/special-data-types/interval.md index be26053580b..4ef1a7e6238 100644 --- a/docs/en/sql-reference/data-types/special-data-types/interval.md +++ b/docs/en/sql-reference/data-types/special-data-types/interval.md @@ -65,7 +65,7 @@ SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVA └─────────────────────┴────────────────────────────────────────────────────────────────────┘ ``` -And to compare values with different intevals: +And to compare values with different intervals: ``` sql SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 0ab1858dc97..1708991af74 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1606,7 +1606,7 @@ struct ConvertImpl result_value = arguments[0].column->getInt(0) / conversion_factor; } else - { + { for (int i = from_position - 1; i >= to_position; --i) { for (const auto &entry : map) From 6f346c161463330c31d0bce2566df6875cf1d614 Mon Sep 17 00:00:00 2001 From: Jacob Reckhard Date: Thu, 8 Aug 2024 15:20:07 -0600 Subject: [PATCH 0652/1722] added order for repeatablility of test --- .../03215_multilinestring_geometry.reference | 11 +++++++---- .../0_stateless/03215_multilinestring_geometry.sql | 10 ++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.reference b/tests/queries/0_stateless/03215_multilinestring_geometry.reference index f20e21d86f6..9702dd6d6f8 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.reference +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.reference @@ -17,11 +17,14 @@ SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); POLYGON((1 1,2 2,3 3,1 1)) String [[(1,1),(2,2),(3,3),(1,1)]] Polygon -- Non constant tests -CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +DROP TABLE IF EXISTS t; +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String, ord Float64) Engine = Memory; +INSERT INTO t (ord, shape, wkt_string) VALUES (1, [[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (2, [[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (3, [[(1, 0), (2, 1), (3, 0), (4, 1), (5, 0), (6, 1), (7, 0), (8, 1), (9, 0), (10, 1)]], 'MULTILINESTRING ((1 0, 2 1, 3 0, 4 1, 5 0, 6 1, 7 0, 8 1, 9 0, 10 1))'); -- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. -- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) -select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t order by ord; POLYGON((1 1,2 2,3 3,1 1)) [[(1,1),(2,2),(3,3),(1,1)]] 1 POLYGON((1 1,2 2,3 3,1 1),(1 0,2 0,3 0,1 0)) [[(1,1),(2,2),(3,3),(1,1)],[(1,0),(2,0),(3,0)]] 1 +POLYGON((1 0,2 1,3 0,4 1,5 0,6 1,7 0,8 1,9 0,10 1,1 0)) [[(1,0),(2,1),(3,0),(4,1),(5,0),(6,1),(7,0),(8,1),(9,0),(10,1)]] 1 diff --git a/tests/queries/0_stateless/03215_multilinestring_geometry.sql b/tests/queries/0_stateless/03215_multilinestring_geometry.sql index 6081e081fb3..cf4ef15f63d 100644 --- a/tests/queries/0_stateless/03215_multilinestring_geometry.sql +++ b/tests/queries/0_stateless/03215_multilinestring_geometry.sql @@ -13,12 +13,14 @@ SELECT x, toTypeName(x), readWKTPolygon(x) as y, toTypeName(y); -- Non constant tests -CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String) Engine = Memory; -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); -INSERT INTO t (shape, wkt_string) VALUES ([[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +DROP TABLE IF EXISTS t; +CREATE TABLE IF NOT EXISTS t (shape Array(Array(Tuple(Float64, Float64))), wkt_string String, ord Float64) Engine = Memory; +INSERT INTO t (ord, shape, wkt_string) VALUES (1, [[(1, 1), (2, 2), (3, 3), (1, 1)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (2, [[(1, 1), (2, 2), (3, 3), (1, 1)], [(1, 0), (2, 0), (3, 0)]], 'MULTILINESTRING ((1 1, 2 2, 3 3, 1 1), (1 0, 2 0, 3 0))'); +INSERT INTO t (ord, shape, wkt_string) VALUES (3, [[(1, 0), (2, 1), (3, 0), (4, 1), (5, 0), (6, 1), (7, 0), (8, 1), (9, 0), (10, 1)]], 'MULTILINESTRING ((1 0, 2 1, 3 0, 4 1, 5 0, 6 1, 7 0, 8 1, 9 0, 10 1))'); -- Native Array(Array(Tuple(Float64, Float64))) is treated as Polygon, not as MultiLineString. -- but reading MultiLineString should still return an Array(Array(Tuple(Float64, Float64))) -select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t; +select wkt(shape), readWKTMultiLineString(wkt_string), readWKTMultiLineString(wkt_string) = shape from t order by ord; From 94efbb0bf9ab62a5399d4918e7bcfd358421a879 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 23:26:24 +0200 Subject: [PATCH 0653/1722] fix build --- src/Functions/FunctionsConversion.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 1708991af74..43ebe573582 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -5279,7 +5279,7 @@ REGISTER_FUNCTION(Conversion) /// MySQL compatibility alias. Cannot be registered as alias, /// because we don't want it to be normalized to toDate in queries, /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive); + factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); From b4c553718353eb2302f85ea4d096a92036ce832c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 8 Aug 2024 23:49:56 +0200 Subject: [PATCH 0654/1722] fix errorcodes in test --- .../03223_interval_data_type_comparison.sql | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql index 6e4862bf2d2..5d01addae45 100644 --- a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql @@ -21,7 +21,7 @@ SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2); SELECT toIntervalNanosecond(1) > toIntervalDay(2); SELECT toIntervalNanosecond(5) > toIntervalWeek(1); -SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError NO_COMMON_TYPE } SELECT('Comparing microseconds'); SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999); @@ -40,7 +40,7 @@ SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1); SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2); SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2); -SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing milliseconds'); SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2); @@ -57,7 +57,7 @@ SELECT toIntervalMillisecond(144000001) < toIntervalHour(40); SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20); SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2); -SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing seconds'); SELECT toIntervalSecond(120) > toIntervalSecond(2); @@ -72,7 +72,7 @@ SELECT toIntervalSecond(1) > toIntervalHour(2); SELECT toIntervalSecond(86401) < toIntervalDay(1); SELECT toIntervalSecond(1209600) != toIntervalWeek(2); -SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing minutes'); SELECT toIntervalMinute(1) < toIntervalMinute(59); @@ -85,7 +85,7 @@ SELECT toIntervalMinute(1) > toIntervalHour(59); SELECT toIntervalMinute(1440) != toIntervalDay(1); SELECT toIntervalMinute(30241) < toIntervalWeek(3); -SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError NO_COMMON_TYPE } SELECT('Comparing hours'); SELECT toIntervalHour(48) > toIntervalHour(2); @@ -96,7 +96,7 @@ SELECT toIntervalHour(48) < toIntervalHour(2); SELECT toIntervalHour(48) < toIntervalDay(2); SELECT toIntervalHour(672) != toIntervalWeek(4); -SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } SELECT('Comparing days'); SELECT toIntervalDay(1) < toIntervalDay(23); @@ -105,14 +105,14 @@ SELECT toIntervalDay(25) > toIntervalWeek(3); SELECT toIntervalDay(1) > toIntervalDay(23); SELECT toIntervalDay(25) < toIntervalWeek(3); -SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError NO_COMMON_TYPE } SELECT('Comparing weeks'); SELECT toIntervalWeek(1) < toIntervalWeek(6); SELECT toIntervalWeek(1) > toIntervalWeek(6); -SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError NO_COMMON_TYPE } SELECT('Comparing months'); SELECT toIntervalMonth(1) < toIntervalMonth(3); @@ -123,7 +123,7 @@ SELECT toIntervalMonth(1) > toIntervalMonth(3); SELECT toIntervalMonth(124) < toIntervalQuarter(5); SELECT toIntervalMonth(36) != toIntervalYear(3); -SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError NO_COMMON_TYPE } SELECT('Comparing quarters'); SELECT toIntervalQuarter(5) > toIntervalQuarter(4); @@ -132,11 +132,11 @@ SELECT toIntervalQuarter(20) = toIntervalYear(5); SELECT toIntervalQuarter(5) < toIntervalQuarter(4); SELECT toIntervalQuarter(20) != toIntervalYear(5); -SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError NO_COMMON_TYPE } SELECT('Comparing years'); SELECT toIntervalYear(1) < toIntervalYear(3); SELECT toIntervalYear(1) > toIntervalYear(3); -SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError NO_COMMON_TYPE } From 8d0c8318ea269ea30707669889932c2b4b66d612 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 9 Aug 2024 00:05:26 +0200 Subject: [PATCH 0655/1722] Apply suggestions from code review --- src/QueryPipeline/QueryPipeline.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 844b9e3b039..c9c0bad7553 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -283,7 +283,7 @@ static void initRowsBeforeAggregation(std::shared_ptr processors, IO if (!processors->empty()) { RowsBeforeStepCounterPtr rows_before_aggregation = std::make_shared(); - for (auto processor : *processors) + for (const auto & processor : *processors) { if (typeid_cast(processor.get()) || typeid_cast(processor.get())) { @@ -545,7 +545,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); - for (const auto context : resources.interpreter_context) + for (const auto & context : resources.interpreter_context) { if (context->getSettingsRef().rows_before_aggregation) { From 6f0f27838745d427966a3c949316a541d9fda7ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 03:45:07 +0200 Subject: [PATCH 0656/1722] Fix trash (low-quality code) in AWS S3 --- src/IO/S3/URI.cpp | 29 ++++++++++++----------------- src/IO/S3/URI.h | 8 ++++---- 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index fead18315d8..446c2aa355b 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -1,8 +1,8 @@ #include -#include -#include -#include "Common/Macros.h" + #if USE_AWS_S3 +#include +#include #include #include #include @@ -10,6 +10,7 @@ #include + namespace DB { @@ -47,14 +48,6 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#path-style-access static const RE2 path_style_pattern("^/([^/]*)/(.*)"); - static constexpr auto S3 = "S3"; - static constexpr auto S3EXPRESS = "S3EXPRESS"; - static constexpr auto COSN = "COSN"; - static constexpr auto COS = "COS"; - static constexpr auto OBS = "OBS"; - static constexpr auto OSS = "OSS"; - static constexpr auto EOS = "EOS"; - if (allow_archive_path_syntax) std::tie(uri_str, archive_pattern) = getURIAndArchivePattern(uri_); else @@ -85,7 +78,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) URIConverter::modifyURI(uri, mapper); } - storage_name = S3; + storage_name = "S3"; if (uri.getHost().empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Host is empty in S3 URI."); @@ -93,11 +86,13 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) /// Extract object version ID from query string. bool has_version_id = false; for (const auto & [query_key, query_value] : uri.getQueryParameters()) + { if (query_key == "versionId") { version_id = query_value; has_version_id = true; } + } /// Poco::URI will ignore '?' when parsing the path, but if there is a versionId in the http parameter, /// '?' can not be used as a wildcard, otherwise it will be ambiguous. @@ -130,14 +125,14 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) boost::to_upper(name); /// For S3Express it will look like s3express-eun1-az1, i.e. contain region and AZ info - if (name != S3 && !name.starts_with(S3EXPRESS) && name != COS && name != OBS && name != OSS && name != EOS) + if (name != "S3" && !name.starts_with("S3EXPRESS") && name != "COS" && name != "OBS" && name != "OSS" && name != "EOS") throw Exception( ErrorCodes::BAD_ARGUMENTS, "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", quoteString(name)); - if (name == COS) - storage_name = COSN; + if (name == "COS") + storage_name = "COSN"; else storage_name = name; } @@ -153,8 +148,8 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) void URI::addRegionToURI(const std::string ®ion) { - if (auto pos = endpoint.find("amazonaws.com"); pos != std::string::npos) - endpoint = endpoint.substr(0, pos) + region + "." + endpoint.substr(pos); + if (auto pos = endpoint.find(".amazonaws.com"); pos != std::string::npos) + endpoint = endpoint.substr(0, pos) + "." + region + endpoint.substr(pos); } void URI::validateBucket(const String & bucket, const Poco::URI & uri) diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index 80e2da96cd4..c8d0b28cd15 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -1,14 +1,14 @@ #pragma once -#include -#include - #include "config.h" #if USE_AWS_S3 +#include +#include #include + namespace DB::S3 { @@ -23,7 +23,7 @@ namespace DB::S3 struct URI { Poco::URI uri; - // Custom endpoint if URI scheme is not S3. + // Custom endpoint if URI scheme, if not S3. std::string endpoint; std::string bucket; std::string key; From 248da0341aca537104486243b84ca230e9c4f9fb Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 8 Aug 2024 15:54:56 +0800 Subject: [PATCH 0657/1722] fixed --- .../HashJoin/HashJoinMethodsImpl.h | 53 ++++++++++++++----- src/Interpreters/joinDispatch.h | 8 +-- ..._join_on_inequal_expression_fast.reference | 1 - 3 files changed, 44 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 5fefe53d145..aedd24630d1 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -1,11 +1,12 @@ #pragma once #include + namespace DB { namespace ErrorCodes { - extern const int UNSUPPORTED_JOIN_KEYS; - extern const int LOGICAL_ERROR; +extern const int UNSUPPORTED_JOIN_KEYS; +extern const int LOGICAL_ERROR; } template size_t HashJoinMethods::insertFromBlockImpl( @@ -156,7 +157,6 @@ Block HashJoinMethods::joinBlockImpl( block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); } } - return remaining_block; } @@ -596,7 +596,7 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter template template -size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( +size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -662,6 +662,8 @@ size_t HashJoinMethods::joinRightColumnsWithAddti { auto & mapped = find_result.getMapped(); find_results.push_back(find_result); + /// We don't add missing in addFoundRowAll here. we will add it after filter is applied. + /// it's different from `joinRightColumns`. if (flag_per_row) addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); else @@ -682,32 +684,54 @@ size_t HashJoinMethods::joinRightColumnsWithAddti for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) { bool any_matched = false; - /// For right join, flag_per_row is true, we need mark used flags for each row. + /// right/full join or multiple disjuncts, we need to mark used flags for each row. if (flag_per_row) { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { if (filter_flags[replicated_row]) { - any_matched = true; if constexpr (join_features.is_semi_join || join_features.is_any_join) { - auto used_once = used_flags.template setUsedOnce( - selected_right_row_it->block, selected_right_row_it->row_num, 0); - if (used_once) + /// For LEFT/INNER SEMI/ANY JOIN, we need to add only first appeared row from left, + if constexpr (join_features.left || join_features.inner) { - total_added_rows += 1; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + if (!any_matched) + { + // For inner join, we need mark each right row'flag, because we only use each right row once. + auto used_once = used_flags.template setUsedOnce( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + any_matched = true; + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } + } + } + else + { + auto used_once = used_flags.template setUsedOnce( + selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (used_once) + { + any_matched = true; + total_added_rows += 1; + added_columns.appendFromBlock( + *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + } } } else if constexpr (join_features.is_anti_join) { + any_matched = true; if constexpr (join_features.right && join_features.need_flags) used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); } else { + any_matched = true; total_added_rows += 1; added_columns.appendFromBlock( *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); @@ -715,6 +739,7 @@ size_t HashJoinMethods::joinRightColumnsWithAddti selected_right_row_it->block, selected_right_row_it->row_num, 0); } } + ++selected_right_row_it; } } @@ -892,7 +917,8 @@ void HashJoinMethods::correctNullabilityInplace( } template -void HashJoinMethods::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) +void HashJoinMethods::correctNullabilityInplace( + ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) { if (nullable) { @@ -908,4 +934,3 @@ void HashJoinMethods::correctNullabilityInplace( JoinCommon::removeColumnNullability(column); } } - diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 5d4bd8f92e5..4aabc49c29b 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -16,11 +16,13 @@ namespace DB /// When only need to match only one row from right table, use HashJoin::MapsOne. For example, LEFT ANY/SEMI/ANTI. /// /// HashJoin::MapsAll will store all rows for each key in the map. It is used when need to match multiple rows from right table. -/// For example, RIGHT ANY/ALL, FULL JOIN, INNER JOIN. +/// For example, LEFT ALL, INNER ALL, RIGHT ALL/ANY. /// /// prefer_use_maps_all is true when there is mixed inequal condition in the join condition. For example, `t1.a = t2.a AND t1.b > t2.b`. /// In this case, we need to use HashJoin::MapsAll to store all rows for each key in the map. We will select all matched rows from the map /// and filter them by `t1.b > t2.b`. +/// +/// flagged indicates whether we need to store flags for each row whether it has been used in the join. See JoinUsedFlags.h. template struct MapGetter; @@ -30,7 +32,7 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; @@ -43,7 +45,7 @@ template struct MapGetter struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; -template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; +template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference index aa8d4103db2..a70e70ef7e9 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -742,4 +742,3 @@ key1 b 2 3 2 key1 c 3 2 1 key1 d 4 7 2 key2 a2 1 1 1 -key4 f 2 3 4 From d0a1ee821b609856c2692abc01d132f8d7a8b88f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 03:51:12 +0200 Subject: [PATCH 0658/1722] You don't know regular expressions --- src/IO/S3/URI.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 446c2aa355b..eea73474c44 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -41,7 +41,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) /// Case when AWS Private Link Interface is being used /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html - static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.]+)\.vpce.amazonaws.com(:\d{1,5})?)"); + static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.]+)\.vpce\.amazonaws\.com(:\d{1,5})?)"); /// Case when bucket name and key represented in path of S3 URL. /// E.g. (https://s3.region.amazonaws.com/bucket-name/key) From 978d36f9fe0e8cc34b6529276b7435b56a16bbb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 03:57:47 +0200 Subject: [PATCH 0659/1722] It's strange that we cared about this --- src/IO/S3/URI.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index eea73474c44..64962f63edb 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -43,7 +43,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.]+)\.vpce\.amazonaws\.com(:\d{1,5})?)"); - /// Case when bucket name and key represented in path of S3 URL. + /// Case when bucket name and key represented in the path of S3 URL. /// E.g. (https://s3.region.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#path-style-access static const RE2 path_style_pattern("^/([^/]*)/(.*)"); @@ -124,13 +124,6 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) } boost::to_upper(name); - /// For S3Express it will look like s3express-eun1-az1, i.e. contain region and AZ info - if (name != "S3" && !name.starts_with("S3EXPRESS") && name != "COS" && name != "OBS" && name != "OSS" && name != "EOS") - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", - quoteString(name)); - if (name == "COS") storage_name = "COSN"; else From dbd4a6d551a9c242c0ce8025ebcdb1304f0d448c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 04:41:06 +0200 Subject: [PATCH 0660/1722] Speed up from 4 sec to 2 sec #52771 --- src/IO/S3/Credentials.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index dfb7727fca4..11779c4dbd5 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -145,12 +145,16 @@ Aws::String AWSEC2MetadataClient::getDefaultCredentialsSecurely() const { String user_agent_string = awsComputeUserAgentString(); auto [new_token, response_code] = getEC2MetadataToken(user_agent_string); - if (response_code == Aws::Http::HttpResponseCode::BAD_REQUEST) + if (response_code == Aws::Http::HttpResponseCode::BAD_REQUEST + || response_code == Aws::Http::HttpResponseCode::REQUEST_NOT_MADE) + { + /// At least the host should be available and reply, otherwise neither IMDSv2 nor IMDSv1 are usable. return {}; + } else if (response_code != Aws::Http::HttpResponseCode::OK || new_token.empty()) { LOG_TRACE(logger, "Calling EC2MetadataService to get token failed, " - "falling back to less secure way. HTTP response code: {}", response_code); + "falling back to a less secure way. HTTP response code: {}", response_code); return getDefaultCredentials(); } From 8bfe4ee23f8b37ff8780e44ede9dc785ac563f75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 04:57:18 +0200 Subject: [PATCH 0661/1722] Speed up requests when IMDS is not available --- src/IO/S3/Credentials.cpp | 6 +++--- src/IO/S3/Credentials.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 11779c4dbd5..9c5f6547933 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -251,7 +251,7 @@ static Aws::String getAWSMetadataEndpoint() return ec2_metadata_service_endpoint; } -std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) +std::shared_ptr createEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration) { auto endpoint = getAWSMetadataEndpoint(); return std::make_shared(client_configuration, endpoint.c_str()); @@ -785,11 +785,11 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. - aws_client_configuration.connectTimeoutMs = 1000; + aws_client_configuration.connectTimeoutMs = 10; aws_client_configuration.requestTimeoutMs = 1000; aws_client_configuration.retryStrategy = std::make_shared(1, 1000); - auto ec2_metadata_client = InitEC2MetadataClient(aws_client_configuration); + auto ec2_metadata_client = createEC2MetadataClient(aws_client_configuration); auto config_loader = std::make_shared(ec2_metadata_client, !credentials_configuration.use_insecure_imds_request); AddProvider(std::make_shared(config_loader)); diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 95297ab0538..042c48ec15a 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -70,7 +70,7 @@ private: LoggerPtr logger; }; -std::shared_ptr InitEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration); +std::shared_ptr createEC2MetadataClient(const Aws::Client::ClientConfiguration & client_configuration); class AWSEC2InstanceProfileConfigLoader : public Aws::Config::AWSProfileConfigLoader { From 9470ceb34d6519c820f7c7ddccbe27b48a046f2a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 05:41:10 +0200 Subject: [PATCH 0662/1722] Minor changes --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/IO/S3/PocoHTTPClient.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 433a0e96d2e..9854bada9ec 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -63,7 +63,7 @@ void throwIfError(const Aws::Utils::Outcome & response) { const auto & err = response.GetError(); throw S3Exception( - fmt::format("{} (Code: {}, s3 exception: {})", + fmt::format("{} (Code: {}, S3 exception: '{}')", err.GetMessage(), static_cast(err.GetErrorType()), err.GetExceptionName()), err.GetErrorType()); } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index aab7a39534d..de43f34d838 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -128,7 +128,7 @@ void PocoHTTPClientConfiguration::updateSchemeAndRegion() } else { - /// In global mode AWS C++ SDK send `us-east-1` but accept switching to another one if being suggested. + /// In global mode AWS C++ SDK sends `us-east-1` but accepts switching to another one if being suggested. region = Aws::Region::AWS_GLOBAL; } } From 54cd980e13d70e84e2c0aad51420a2dd31217131 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 09:44:30 +0200 Subject: [PATCH 0663/1722] Increase queue size and fix tests --- docker/test/stateless/run.sh | 11 +++-------- .../0_stateless/02726_async_insert_flush_queue.sql | 4 +++- .../0_stateless/02726_async_insert_flush_stress.sh | 2 +- 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c81f33ace01..cd2a61cce4e 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -193,8 +193,8 @@ ORDER BY tuple()" # create minio log webhooks for both audit and server logs # use async inserts to avoid creating too many parts -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=300000 +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=300000 max_retries=100 retry=1 @@ -376,12 +376,7 @@ done # collect minio audit and server logs -has_async_inserts=$(clickhouse-client -q "SELECT count() FROM system.asynchronous_inserts WHERE table = 'minio_audit_logs' OR table = 'minio_server_logs'") -if [[ has_async_inserts -eq 1 ]]; then - echo "Waiting for async inserts to flush" - sleep 5 -fi - +clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE" clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql index 97d644fa4d6..5d941adcb81 100644 --- a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -30,7 +30,9 @@ SELECT count() FROM t_async_inserts_flush; SYSTEM FLUSH ASYNC INSERT QUEUE; -SELECT count() FROM system.asynchronous_inserts; +SELECT count() FROM system.asynchronous_inserts +WHERE database = currentDatabase() AND table = 't_async_inserts_flush'; + SELECT count() FROM t_async_inserts_flush; DROP TABLE t_async_inserts_flush; diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh index 876766d0780..61bbbd620f0 100755 --- a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -91,5 +91,5 @@ flush1 $TIMEOUT & wait ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" -${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.asynchronous_inserts" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.asynchronous_inserts WHERE database = currentDatabase() AND table = 'async_inserts'" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"; From cfc10961ed237eac079db6f113330db5391adc1f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 9 Aug 2024 15:56:36 +0800 Subject: [PATCH 0664/1722] fix getName() style for columnlowcardinality and columnunique --- src/Columns/ColumnLowCardinality.h | 2 +- src/Columns/ColumnUnique.h | 2 ++ src/Columns/IColumnUnique.h | 2 +- src/Columns/tests/gtest_column_dump_structure.cpp | 2 +- .../02313_dump_column_structure_low_cardinality.reference | 2 +- tests/queries/0_stateless/02355_column_type_name_lc.reference | 2 +- 6 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 5a23853e961..d9c90e7dc21 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -46,7 +46,7 @@ public: return Base::create(std::move(column_unique), std::move(indexes), is_shared); } - std::string getName() const override { return "LowCardinality(" + getDictionaryPtr()->getName() + ")"; } + std::string getName() const override { return "LowCardinality(" + getDictionary().getNestedColumn()->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } TypeIndex getDataType() const override { return TypeIndex::LowCardinality; } diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index d6cb75679be..8a66f4e02ed 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -48,6 +48,8 @@ private: ColumnUnique(const ColumnUnique & other); public: + std::string getName() const override { return "Unique(" + getNestedColumn()->getName() + ")"; } + MutableColumnPtr cloneEmpty() const override; const ColumnPtr & getNestedColumn() const override; diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index a8e10e5e2b2..52b1bef3009 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -73,7 +73,7 @@ public: /// Returns dictionary hash which is SipHash is applied to each row of nested column. virtual UInt128 getHash() const = 0; - const char * getFamilyName() const override { return "ColumnUnique"; } + const char * getFamilyName() const override { return "Unique"; } TypeIndex getDataType() const override { return getNestedColumn()->getDataType(); } void insert(const Field &) override diff --git a/src/Columns/tests/gtest_column_dump_structure.cpp b/src/Columns/tests/gtest_column_dump_structure.cpp index e00c77798c8..d9647147157 100644 --- a/src/Columns/tests/gtest_column_dump_structure.cpp +++ b/src/Columns/tests/gtest_column_dump_structure.cpp @@ -10,7 +10,7 @@ TEST(IColumn, dumpStructure) { auto type_lc = std::make_shared(std::make_shared()); ColumnPtr column_lc = type_lc->createColumn(); - String expected_structure = "ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1)))"; + String expected_structure = "LowCardinality(size = 0, UInt8(size = 0), Unique(size = 1, String(size = 1)))"; std::vector threads; for (size_t i = 0; i < 6; ++i) diff --git a/tests/queries/0_stateless/02313_dump_column_structure_low_cardinality.reference b/tests/queries/0_stateless/02313_dump_column_structure_low_cardinality.reference index fa7f1799c31..6b1e4743867 100644 --- a/tests/queries/0_stateless/02313_dump_column_structure_low_cardinality.reference +++ b/tests/queries/0_stateless/02313_dump_column_structure_low_cardinality.reference @@ -1 +1 @@ -Array(LowCardinality(String)), Const(size = 1, Array(size = 1, UInt64(size = 1), ColumnLowCardinality(size = 2, UInt8(size = 2), ColumnUnique(size = 3, String(size = 3))))) +Array(LowCardinality(String)), Const(size = 1, Array(size = 1, UInt64(size = 1), LowCardinality(size = 2, UInt8(size = 2), Unique(size = 3, String(size = 3))))) diff --git a/tests/queries/0_stateless/02355_column_type_name_lc.reference b/tests/queries/0_stateless/02355_column_type_name_lc.reference index 234a072299f..50c25a86b2f 100644 --- a/tests/queries/0_stateless/02355_column_type_name_lc.reference +++ b/tests/queries/0_stateless/02355_column_type_name_lc.reference @@ -1 +1 @@ -ColumnLowCardinality +LowCardinality(String) From 47f429a52435d84b3e1cee06202e743c11bcba0d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 20:54:29 +0000 Subject: [PATCH 0665/1722] Proper CMake for libfiu --- CMakeLists.txt | 2 +- contrib/libfiu-cmake/CMakeLists.txt | 27 ++++++++++++++------------- src/CMakeLists.txt | 4 ++-- src/Common/FailPoint.cpp | 6 ++++-- src/Common/FailPoint.h | 9 ++++----- src/Common/config.h.in | 2 +- src/configure_config.cmake | 4 ++-- 7 files changed, 28 insertions(+), 26 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7b4e0484ab1..134f3afd727 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -402,7 +402,7 @@ if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") set(ENABLE_GWP_ASAN OFF) endif () -option (ENABLE_FIU "Enable Fiu" ON) +option (ENABLE_LIBFIU "Enable libfiu" ON) option(WERROR "Enable -Werror compiler option" ON) diff --git a/contrib/libfiu-cmake/CMakeLists.txt b/contrib/libfiu-cmake/CMakeLists.txt index e805491edbb..eab55087c98 100644 --- a/contrib/libfiu-cmake/CMakeLists.txt +++ b/contrib/libfiu-cmake/CMakeLists.txt @@ -1,20 +1,21 @@ -if (NOT ENABLE_FIU) - message (STATUS "Not using fiu") +if (NOT ENABLE_LIBFIU) + message (STATUS "Not using libfiu") return () endif () -set(FIU_DIR "${ClickHouse_SOURCE_DIR}/contrib/libfiu/") +set(LIBFIU_DIR "${ClickHouse_SOURCE_DIR}/contrib/libfiu/") -set(FIU_SOURCES - ${FIU_DIR}/libfiu/fiu.c - ${FIU_DIR}/libfiu/fiu-rc.c - ${FIU_DIR}/libfiu/backtrace.c - ${FIU_DIR}/libfiu/wtable.c +set(LIBFIU_SOURCES + ${LIBFIU_DIR}/libfiu/fiu.c + ${LIBFIU_DIR}/libfiu/fiu-rc.c + ${LIBFIU_DIR}/libfiu/backtrace.c + ${LIBFIU_DIR}/libfiu/wtable.c ) -set(FIU_HEADERS "${FIU_DIR}/libfiu") +set(LIBFIU_HEADERS "${LIBFIU_DIR}/libfiu") -add_library(_fiu ${FIU_SOURCES}) -target_compile_definitions(_fiu PUBLIC DUMMY_BACKTRACE) -target_include_directories(_fiu PUBLIC ${FIU_HEADERS}) -add_library(ch_contrib::fiu ALIAS _fiu) +add_library(_libfiu ${LIBFIU_SOURCES}) +target_compile_definitions(_libfiu PUBLIC DUMMY_BACKTRACE) +target_compile_definitions(_libfiu PUBLIC FIU_ENABLE) +target_include_directories(_libfiu PUBLIC ${LIBFIU_HEADERS}) +add_library(ch_contrib::libfiu ALIAS _libfiu) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 98dd0601a1b..db3ef0f489f 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -353,8 +353,8 @@ target_link_libraries(clickhouse_common_io Poco::Foundation ) -if (TARGET ch_contrib::fiu) - target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::fiu) +if (TARGET ch_contrib::libfiu) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::libfiu) endif() if (TARGET ch_contrib::cpuid) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 0b1ec552d43..b2fcbc77c56 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -7,6 +7,8 @@ #include #include +#include "config.h" + namespace DB { @@ -15,7 +17,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; }; -#if FIU_ENABLE +#if USE_LIBFIU static struct InitFiu { InitFiu() @@ -135,7 +137,7 @@ void FailPointInjection::pauseFailPoint(const String & fail_point_name) void FailPointInjection::enableFailPoint(const String & fail_point_name) { -#if FIU_ENABLE +#if USE_LIBFIU #define SUB_M(NAME, flags, pause) \ if (fail_point_name == FailPoints::NAME) \ { \ diff --git a/src/Common/FailPoint.h b/src/Common/FailPoint.h index b3e1214d597..1af13d08553 100644 --- a/src/Common/FailPoint.h +++ b/src/Common/FailPoint.h @@ -1,17 +1,16 @@ #pragma once -#include "config.h" #include #include #include +#include "config.h" + #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wdocumentation" #pragma clang diagnostic ignored "-Wreserved-macro-identifier" - -#include -#include - +# include +# include #pragma clang diagnostic pop #include diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 6a0090130a3..56a067b06e8 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,7 +59,7 @@ #cmakedefine01 USE_SKIM #cmakedefine01 USE_PRQL #cmakedefine01 USE_ULID -#cmakedefine01 FIU_ENABLE +#cmakedefine01 USE_LIBFIU #cmakedefine01 USE_BCRYPT #cmakedefine01 USE_LIBARCHIVE #cmakedefine01 USE_POCKETFFT diff --git a/src/configure_config.cmake b/src/configure_config.cmake index d22bf674df4..721041bc11b 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -161,8 +161,8 @@ endif() if (TARGET ch_contrib::ssh) set(USE_SSH 1) endif() -if (TARGET ch_contrib::fiu) - set(FIU_ENABLE 1) +if (TARGET ch_contrib::libfiu) + set(USE_LIBFIU 1) endif() if (TARGET ch_contrib::libarchive) set(USE_LIBARCHIVE 1) From 30d8e407723f45ed79dc960604d77e8ee02f3edb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 21:02:04 +0000 Subject: [PATCH 0666/1722] Fix referenced variable for vectorscan in system.build_options --- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index a81bcb08bfc..3f84a7468fd 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -36,7 +36,7 @@ const char * auto_config_build[] "USE_SSL", "@USE_SSL@", "OPENSSL_VERSION", "@OPENSSL_VERSION@", "OPENSSL_IS_BORING_SSL", "@OPENSSL_IS_BORING_SSL@", - "USE_VECTORSCAN", "@ENABLE_VECTORSCAN@", + "USE_VECTORSCAN", "@USE_VECTORSCAN@", "USE_SIMDJSON", "@USE_SIMDJSON@", "USE_ODBC", "@USE_ODBC@", "USE_GRPC", "@USE_GRPC@", From b242a129f811c9838167d98df56d060ceac24b85 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Aug 2024 21:03:42 +0000 Subject: [PATCH 0667/1722] Fix referenced variable for jemalloc in system.build_options --- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 3f84a7468fd..6cecef5a7ad 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -21,7 +21,7 @@ const char * auto_config_build[] "BUILD_COMPILE_DEFINITIONS", "@BUILD_COMPILE_DEFINITIONS@", "USE_EMBEDDED_COMPILER", "@USE_EMBEDDED_COMPILER@", "USE_GLIBC_COMPATIBILITY", "@GLIBC_COMPATIBILITY@", - "USE_JEMALLOC", "@ENABLE_JEMALLOC@", + "USE_JEMALLOC", "@USE_JEMALLOC@", "USE_ICU", "@USE_ICU@", "USE_H3", "@USE_H3@", "USE_MYSQL", "@USE_MYSQL@", From 1e2eea9f6333b165b1b15acef5f489ad067a57f3 Mon Sep 17 00:00:00 2001 From: kruglov Date: Fri, 9 Aug 2024 10:16:15 +0300 Subject: [PATCH 0668/1722] Fixed errors when publication name contents symbols except [a-z_] --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f632e553a0d..01f78673ed8 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -659,7 +659,7 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) { - std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); + std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", doubleQuoteString(publication_name)); tx.exec(query_str); LOG_DEBUG(log, "Dropped publication: {}", publication_name); } @@ -667,7 +667,7 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::addTableToPublication(pqxx::nontransaction & ntx, const String & table_name) { - std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name)); + std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", doubleQuoteString(publication_name), doubleQuoteWithSchema(table_name)); ntx.exec(query_str); LOG_TRACE(log, "Added table {} to publication `{}`", doubleQuoteWithSchema(table_name), publication_name); } From a497a2391455de21dec19b365cc939defdc56b1e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 08:33:11 +0000 Subject: [PATCH 0669/1722] Fix CMake for QPL --- contrib/qpl-cmake/CMakeLists.txt | 4 ---- src/CMakeLists.txt | 2 ++ src/Common/config.h.in | 1 + .../CompressionCodecDeflateQpl.cpp | 8 +++---- src/Compression/CompressionCodecDeflateQpl.h | 6 +++++ src/Compression/CompressionFactory.cpp | 22 +++++++++---------- .../System/StorageSystemBuildOptions.cpp.in | 2 +- src/configure_config.cmake | 3 +++ 8 files changed, 28 insertions(+), 20 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index e62612cff5a..89332ae0f7a 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -728,10 +728,6 @@ add_library(_qpl STATIC ${LIB_DEPS}) target_include_directories(_qpl PUBLIC $ $) - -target_compile_definitions(_qpl - PUBLIC -DENABLE_QPL_COMPRESSION) - target_link_libraries(_qpl PRIVATE ch_contrib::accel-config) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index db3ef0f489f..eba04d93df5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -556,6 +556,8 @@ target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4) if (TARGET ch_contrib::qpl) dbms_target_link_libraries(PUBLIC ch_contrib::qpl) + target_link_libraries (clickhouse_compression PUBLIC ch_contrib::qpl) + target_link_libraries (clickhouse_compression PUBLIC ch_contrib::accel-config) endif () if (TARGET ch_contrib::accel-config) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 56a067b06e8..1680cde22a2 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -32,6 +32,7 @@ #cmakedefine01 USE_IDNA #cmakedefine01 USE_NLP #cmakedefine01 USE_VECTORSCAN +#cmakedefine01 USE_QPL #cmakedefine01 USE_LIBURING #cmakedefine01 USE_AVRO #cmakedefine01 USE_CAPNP diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index f1b5b24e866..c82ee861a6f 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -1,7 +1,3 @@ -#ifdef ENABLE_QPL_COMPRESSION - -#include -#include #include #include #include @@ -11,6 +7,10 @@ #include #include #include +#include +#include + +#if USE_QPL #include "libaccel_config.h" diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 86fd9051bd8..d9abc0fb7e0 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -4,6 +4,11 @@ #include #include #include + +#include "config.h" + +#if USE_QPL + #include namespace Poco @@ -117,3 +122,4 @@ private: }; } +#endif diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 2e7aa0d086f..fbb5664d441 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -1,20 +1,20 @@ -#include "config.h" - #include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include #include -#include -#include -#include +#include +#include +#include #include +#include "config.h" + namespace DB { @@ -179,7 +179,7 @@ void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecZSTDQAT(CompressionCodecFactory & factory); #endif void registerCodecMultiple(CompressionCodecFactory & factory); -#ifdef ENABLE_QPL_COMPRESSION +#if USE_QPL void registerCodecDeflateQpl(CompressionCodecFactory & factory); #endif @@ -209,7 +209,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecGorilla(*this); registerCodecEncrypted(*this); registerCodecFPC(*this); -#ifdef ENABLE_QPL_COMPRESSION +#if USE_QPL registerCodecDeflateQpl(*this); #endif registerCodecGCD(*this); diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 6cecef5a7ad..f7edfb17542 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -62,7 +62,7 @@ const char * auto_config_build[] "USE_ARROW", "@USE_ARROW@", "USE_ORC", "@USE_ORC@", "USE_MSGPACK", "@USE_MSGPACK@", - "USE_QPL", "@ENABLE_QPL@", + "USE_QPL", "@USE_QPL@", "USE_QAT", "@ENABLE_QATLIB@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 721041bc11b..6782cc6d824 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -135,6 +135,9 @@ endif() if (TARGET ch_contrib::vectorscan) set(USE_VECTORSCAN 1) endif() +if (TARGET ch_contrib::qpl) + set(USE_QPL 1) +endif() if (TARGET ch_contrib::avrocpp) set(USE_AVRO 1) endif() From eec5fe087c273aba443d97824da0e7aadfd52cdd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 08:50:52 +0000 Subject: [PATCH 0670/1722] Fix CMake for QATlib --- contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt | 4 ++-- src/CMakeLists.txt | 7 ++----- src/Common/config.h.in | 1 + src/Compression/CompressionCodecZSTDQAT.cpp | 5 ++++- src/Compression/CompressionFactory.cpp | 4 ++-- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- src/configure_config.cmake | 3 +++ 7 files changed, 15 insertions(+), 11 deletions(-) diff --git a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt index 72d21a8572b..fc18092f574 100644 --- a/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt +++ b/contrib/QAT-ZSTD-Plugin-cmake/CMakeLists.txt @@ -27,7 +27,7 @@ if (ENABLE_QAT_OUT_OF_TREE_BUILD) ${QAT_AL_INCLUDE_DIR} ${QAT_USDM_INCLUDE_DIR} ${ZSTD_LIBRARY_DIR}) - target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) else () # In-tree build message(STATUS "Intel QATZSTD in-tree build") @@ -78,7 +78,7 @@ else () # In-tree build ${QAT_USDM_INCLUDE_DIR} ${ZSTD_LIBRARY_DIR} ${LIBQAT_HEADER_DIR}) - target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC -DINTREE) + target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DINTREE) target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $ $) add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin) endif () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index eba04d93df5..43092d10dd2 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -560,12 +560,9 @@ if (TARGET ch_contrib::qpl) target_link_libraries (clickhouse_compression PUBLIC ch_contrib::accel-config) endif () -if (TARGET ch_contrib::accel-config) - dbms_target_link_libraries(PUBLIC ch_contrib::accel-config) -endif () - -if (TARGET ch_contrib::qatzstd_plugin) +if (TARGET ch_contrib::accel-config AND TARGET ch_contrib::qatzstd_plugin) dbms_target_link_libraries(PUBLIC ch_contrib::qatzstd_plugin) + dbms_target_link_libraries(PUBLIC ch_contrib::accel-config) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::qatzstd_plugin) endif () diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 1680cde22a2..e3f8882850f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -33,6 +33,7 @@ #cmakedefine01 USE_NLP #cmakedefine01 USE_VECTORSCAN #cmakedefine01 USE_QPL +#cmakedefine01 USE_QATLIB #cmakedefine01 USE_LIBURING #cmakedefine01 USE_AVRO #cmakedefine01 USE_CAPNP diff --git a/src/Compression/CompressionCodecZSTDQAT.cpp b/src/Compression/CompressionCodecZSTDQAT.cpp index 5a4ef70a30a..e19b7e4a001 100644 --- a/src/Compression/CompressionCodecZSTDQAT.cpp +++ b/src/Compression/CompressionCodecZSTDQAT.cpp @@ -1,4 +1,6 @@ -#ifdef ENABLE_ZSTD_QAT_CODEC +#include "config.h" + +#if USE_QATLIB #include #include @@ -6,6 +8,7 @@ #include #include #include + #include #include diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index fbb5664d441..ac00f571568 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -175,7 +175,7 @@ void registerCodecNone(CompressionCodecFactory & factory); void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecLZ4HC(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); -#ifdef ENABLE_ZSTD_QAT_CODEC +#if USE_QATLIB void registerCodecZSTDQAT(CompressionCodecFactory & factory); #endif void registerCodecMultiple(CompressionCodecFactory & factory); @@ -198,7 +198,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecNone(*this); registerCodecLZ4(*this); registerCodecZSTD(*this); -#ifdef ENABLE_ZSTD_QAT_CODEC +#if USE_QATLIB registerCodecZSTDQAT(*this); #endif registerCodecLZ4HC(*this); diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index f7edfb17542..9e5adbfe825 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -63,7 +63,7 @@ const char * auto_config_build[] "USE_ORC", "@USE_ORC@", "USE_MSGPACK", "@USE_MSGPACK@", "USE_QPL", "@USE_QPL@", - "USE_QAT", "@ENABLE_QATLIB@", + "USE_QATLIB", "@USE_QATLIB@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 6782cc6d824..5b24f79ef6e 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -138,6 +138,9 @@ endif() if (TARGET ch_contrib::qpl) set(USE_QPL 1) endif() +if (TARGET ch_contrib::qatlib) + set(USE_QATLIB 1) +endif() if (TARGET ch_contrib::avrocpp) set(USE_AVRO 1) endif() From 759299910c2892b809735caaf663fe374c315c5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 09:14:00 +0000 Subject: [PATCH 0671/1722] Force new analyzer for test --- tests/queries/0_stateless/03217_filtering_in_storage_merge.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql index 5ecc1e7c672..42d31e95f9c 100644 --- a/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql +++ b/tests/queries/0_stateless/03217_filtering_in_storage_merge.sql @@ -13,4 +13,4 @@ INSERT INTO test_03217_merge_replica_1 SELECT number AS x FROM numbers(10); SYSTEM SYNC REPLICA test_03217_merge_replica_2; -- If the filter on _table is not applied, then the plan will show both replicas -EXPLAIN SELECT _table, count() FROM test_03217_all_replicas WHERE _table = 'test_03217_merge_replica_1' AND x >= 0 GROUP BY _table; +EXPLAIN SELECT _table, count() FROM test_03217_all_replicas WHERE _table = 'test_03217_merge_replica_1' AND x >= 0 GROUP BY _table SETTINGS allow_experimental_analyzer=1; From dc64550536ff249b1c12070ed646bc4321bc68bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 09:14:13 +0000 Subject: [PATCH 0672/1722] Remove wrong check from test --- .../0_stateless/03217_filtering_in_system_tables.reference | 4 ++-- .../0_stateless/03217_filtering_in_system_tables.sql | 7 +++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference index 218fddf92e0..d7ccd989f53 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference @@ -1,4 +1,4 @@ information_schema tables default test_03217_system_tables_replica_1 r1 -1 1 -1 1 +1 +1 diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 72ca7c8684d..0db846bc500 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -10,18 +10,17 @@ CREATE TABLE test_03217_system_tables_replica_2(x UInt32) -- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; - SYSTEM FLUSH LOGS; --- argMin-argMax is necessary to make the test repeatable +-- argMax is necessary to make the test repeatable -- StorageSystemTables -SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 +SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 AND current_database = currentDatabase() AND query LIKE '%SELECT database, table FROM system.tables WHERE database = \'information_schema\' AND table = \'tables\';' AND type = 'QueryFinish'; -- StorageSystemReplicas -SELECT argMin(read_rows, event_time_microseconds), argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 +SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 AND current_database = currentDatabase() AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' AND type = 'QueryFinish'; From 6360687b307ea2ea7c5cf5746d83655b72a73a75 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 11:28:18 +0200 Subject: [PATCH 0673/1722] Try fix flaky 02675_profile_events_from_query_log_and_client --- ...events_from_query_log_and_client.reference | 14 ++++++------ ...rofile_events_from_query_log_and_client.sh | 22 +++++++++++++++---- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index babcecf7004..9dbac8d34f2 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -1,11 +1,11 @@ INSERT TO S3 - [ 0 ] S3Clients: 1 - [ 0 ] S3CompleteMultipartUpload: 1 - [ 0 ] S3CreateMultipartUpload: 1 - [ 0 ] S3HeadObject: 2 - [ 0 ] S3ReadRequestsCount: 2 - [ 0 ] S3UploadPart: 1 - [ 0 ] S3WriteRequestsCount: 3 +Successful write requests 3 +S3Clients 1 +S3CompleteMultipartUpload 1 +S3CreateMultipartUpload 1 +S3HeadObject 2 +S3ReadRequestsCount 2 +S3UploadPart 1 CHECK WITH query_log QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0 CREATE diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index e346d9893a7..cae20be79dc 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -9,7 +9,21 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "INSERT TO S3" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | sort +" 2>&1 | $CLICKHOUSE_LOCAL -q " +WITH '(\\w+): (\\d+)' AS pattern, + (SELECT (groupArray(regexpExtract(line, pattern, 1)), + groupArray(regexpExtract(line, pattern, 2)::UInt64))::Map(String, UInt64) + FROM file(stdin, 'LineAsString', 'line String') + WHERE line LIKE '% S3%' + AND line NOT LIKE '%Microseconds%' + AND line NOT LIKE '%S3DiskConnections%' + AND line NOT LIKE '%S3DiskAddresses') AS pe_map +SELECT untuple(arrayJoin(pe_map) AS pe) +WHERE tupleElement(pe, 1) not like '%WriteRequests%' +UNION ALL +SELECT 'Successful write requests', + (pe_map['S3WriteRequestsCount'] - pe_map['S3WriteRequestsErrors'])::UInt64 +" echo "CHECK WITH query_log" $CLICKHOUSE_CLIENT -nq " @@ -40,19 +54,19 @@ CREATE TABLE times (t DateTime) ENGINE MergeTree ORDER BY t echo "INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" 2>&1 | grep -o -e ' \[ .* \] FileOpen: .* ' echo "READ" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " SELECT '1', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" 2>&1 | grep -o -e ' \[ .* \] FileOpen: .* ' echo "INSERT and READ INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; SELECT '2', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; -" 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' +" 2>&1 | grep -o -e ' \[ .* \] FileOpen: .* ' echo "DROP" $CLICKHOUSE_CLIENT -nq " From e4903858c8ae108b87b726a8056acab10dd6b851 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 09:31:33 +0000 Subject: [PATCH 0674/1722] Add extra check to make sure both replicas are present in system.replicas --- .../0_stateless/03217_filtering_in_system_tables.reference | 2 ++ tests/queries/0_stateless/03217_filtering_in_system_tables.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference index d7ccd989f53..c0761c3f689 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.reference +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.reference @@ -1,4 +1,6 @@ information_schema tables +both default test_03217_system_tables_replica_1 r1 +both default test_03217_system_tables_replica_2 r2 default test_03217_system_tables_replica_1 r1 1 1 diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 0db846bc500..2ce63559b99 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -8,6 +8,8 @@ CREATE TABLE test_03217_system_tables_replica_2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03217_system_tables_replica', 'r2') ORDER BY x; +-- Make sure we can read both replicas +SELECT 'both', database, table, replica_name FROM system.replicas WHERE database = currentDatabase(); -- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; SYSTEM FLUSH LOGS; From 29ce915d00f6a48a7dfa6a70ec9889d47eacf584 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Aug 2024 09:37:16 +0000 Subject: [PATCH 0675/1722] Try to fix bug --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 ++ src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 ++ src/Disks/IO/createReadBufferFromFileBase.cpp | 28 +++++++++++++------ .../Local/LocalObjectStorage.cpp | 9 +++--- src/IO/ReadBuffer.h | 3 ++ .../StorageObjectStorageSource.cpp | 5 +--- 6 files changed, 33 insertions(+), 16 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index bb9761a3905..80dcc1baa14 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -169,6 +169,7 @@ void ReadBufferFromRemoteFSGather::initialize() bool ReadBufferFromRemoteFSGather::nextImpl() { + // LOG_DEBU /// Find first available buffer that fits to given offset. if (!current_buf) initialize(); @@ -230,6 +231,7 @@ void ReadBufferFromRemoteFSGather::reset() { current_object = StoredObject(); current_buf_idx = {}; + // buffer_cemetery_.push_back(current_buf); current_buf.reset(); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 9f1cb681f1a..a8d5bd1797e 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -85,6 +85,8 @@ private: size_t current_buf_idx = 0; SeekableReadBufferPtr current_buf; + std::deque buffer_cemetery_; + LoggerPtr log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index b132e25ac6b..c6152543a41 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -1,14 +1,15 @@ +#include +#include #include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include #include +#include "ReadBufferFromRemoteFSGather.h" #include "config.h" namespace ProfileEvents @@ -77,6 +78,7 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 1"); res = std::make_unique( filename, buffer_size, @@ -88,6 +90,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 2"); + res = std::make_unique( filename, buffer_size, @@ -99,6 +103,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 3"); + #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( @@ -117,6 +123,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 4"); + auto & reader = getThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -131,6 +139,7 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 5"); auto & reader = getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -144,8 +153,11 @@ std::unique_ptr createReadBufferFromFileBase( settings.local_throttler); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); + { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 6"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); + } return res; }; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index c37b47bbc93..d291dcd65cf 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -48,11 +48,12 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = - [=] (bool /* restricted_seek */, const StoredObject & object) - -> std::unique_ptr + auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr { - return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + LOG_DEBUG(&Poco::Logger::get("Get object path"), "Remote Path: {}", object.remote_path); + auto kek = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + LOG_DEBUG(&Poco::Logger::get("Buffer created"), "Remote Path: {}", object.remote_path); + return kek; }; return std::make_unique( diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 73f5335411f..ea41aab84f0 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -103,6 +103,7 @@ public: */ bool ALWAYS_INLINE eof() { + LOG_DEBUG(); return !hasPendingData() && !next(); } @@ -182,6 +183,8 @@ public: while (bytes_copied < n && !eof()) { + auto k = *pos; + LOG_DEBUG(&Poco::Logger::get("Next symbol in read"), "Symbol: {}", k); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); ::memcpy(to + bytes_copied, pos, bytes_to_copy); pos += bytes_to_copy; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 399e4a56ba8..9233986d858 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -413,10 +413,7 @@ std::future StorageObjectStorageSource } std::unique_ptr StorageObjectStorageSource::createReadBuffer( - const ObjectInfo & object_info, - const ObjectStoragePtr & object_storage, - const ContextPtr & context_, - const LoggerPtr & log) + const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { const auto & object_size = object_info.metadata->size_bytes; From da907d535623422b7beb1d8cf3e7389698567e28 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 10:37:48 +0100 Subject: [PATCH 0676/1722] Better parsing --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index cd2a61cce4e..c59d36114ae 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -178,7 +178,7 @@ attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 202 clickhouse-client --query "CREATE TABLE minio_audit_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" @@ -186,7 +186,7 @@ ORDER BY tuple()" clickhouse-client --query "CREATE TABLE minio_server_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(substring(JSONExtractRaw(log, 'time'), 2, 29), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" From a5e06c7c311f142aa8d790cf398af19d86cbce4f Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 9 Aug 2024 11:06:11 +0200 Subject: [PATCH 0677/1722] Fix UB in hopEnd and hopStart This was causing segfaults because of a NULL pointer dereference --- src/Functions/FunctionsTimeWindow.cpp | 7 ++++++- .../01049_window_view_window_functions.reference | 2 ++ .../0_stateless/01049_window_view_window_functions.sql | 3 +++ 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 77d740803be..faea9c6ba58 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -622,7 +622,12 @@ struct TimeWindowImpl { auto type = WhichDataType(arguments[0].type); if (type.isTuple()) - return std::static_pointer_cast(arguments[0].type)->getElement(0); + { + const auto & tuple_elems = std::static_pointer_cast(arguments[0].type)->getElements(); + if (tuple_elems.empty()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Tuple passed to {} should not be empty", function_name); + return tuple_elems[0]; + } else if (type.isUInt32()) return std::make_shared(); else diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index 2d49664b280..47d1ccc57dd 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -67,3 +67,5 @@ SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 2020-01-10 00:00:00 SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); 2019-01-10 00:00:00 +SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.sql b/tests/queries/0_stateless/01049_window_view_window_functions.sql index 617019bd2c6..3638dd1a3b2 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.sql +++ b/tests/queries/0_stateless/01049_window_view_window_functions.sql @@ -36,3 +36,6 @@ SELECT hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, I SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); + +SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } From 2657e2b3ef2f98802fd2b8ebcd359fe756b709c6 Mon Sep 17 00:00:00 2001 From: Graham Campbell Date: Fri, 9 Aug 2024 11:08:41 +0100 Subject: [PATCH 0678/1722] Do not apply redundant sorting removal when there's an offset --- .../Optimizations/removeRedundantSorting.cpp | 8 ++-- .../02496_remove_redundant_sorting.reference | 37 +++++++++++++++++ .../02496_remove_redundant_sorting.sh | 22 ++++++++++ ...emove_redundant_sorting_analyzer.reference | 41 +++++++++++++++++++ 4 files changed, 105 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index 7cac7bee6ec..f0094f0f8d2 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -59,9 +60,10 @@ public: if (typeid_cast(current_step) || typeid_cast(current_step) /// (1) if there are LIMITs on top of ORDER BY, the ORDER BY is non-removable - || typeid_cast(current_step) /// (2) if ORDER BY is with FILL WITH, it is non-removable - || typeid_cast(current_step) /// (3) ORDER BY will change order of previous sorting - || typeid_cast(current_step)) /// (4) aggregation change order + || typeid_cast(current_step) /// (2) OFFSET on top of ORDER BY, the ORDER BY is non-removable + || typeid_cast(current_step) /// (3) if ORDER BY is with FILL WITH, it is non-removable + || typeid_cast(current_step) /// (4) ORDER BY will change order of previous sorting + || typeid_cast(current_step)) /// (5) aggregation change order { logStep("nodes_affect_order/push", current_node); nodes_affect_order.push_back(current_node); diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 77ef213b36d..a0a1fd60812 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -527,3 +527,40 @@ Expression (Projection) 2 4 1 3 0 2 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT SUM(a) AS a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + GROUP BY 2 + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression (Projection) + Expression (Before ORDER BY) + Aggregating + Expression (Before GROUP BY) + Offset + Expression (Projection) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + Expression (Projection) + Expression (Before ORDER BY) + Aggregating + Expression (Before GROUP BY) + ReadFromStorage (Values) +-- execute +0 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index 646e2501a99..d59b4387101 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -329,3 +329,25 @@ FROM ORDER BY number DESC )" run_query "$query" + +echo "-- presence of an inner OFFSET retains the ORDER BY" +query="WITH + t1 AS ( + SELECT SUM(a) AS a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + GROUP BY 2 + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2" +run_query "$query" diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index b6a2e3182df..58441de5f22 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -533,3 +533,44 @@ Expression (Project names) 2 4 1 3 0 2 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT SUM(a) AS a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + GROUP BY 2 + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression (Project names) + Expression (Projection) + Aggregating + Expression (Before GROUP BY) + Expression (Change column names to column identifiers) + Expression (Project names) + Offset + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + Expression (Projection) + Expression (Change column names to column identifiers) + Expression (Project names) + Expression (Projection) + Aggregating + Expression (Before GROUP BY) + Expression (Change column names to column identifiers) + ReadFromStorage (Values) +-- execute +0 From dccb6bdd88ef26244ddb1c9de8d1232140036294 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 9 Aug 2024 18:33:05 +0800 Subject: [PATCH 0679/1722] fix failed uts --- .../Formats/Impl/ORCBlockOutputFormat.cpp | 47 +++++++------------ .../Formats/Impl/ORCBlockOutputFormat.h | 5 -- 2 files changed, 17 insertions(+), 35 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index bd89ae0fa86..4a7a23158ff 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -280,20 +280,28 @@ void ORCBlockOutputFormat::writeColumn( DataTypePtr & type, const PaddedPODArray * null_bytemap) { - orc_column.numElements = column.size(); + size_t rows = column.size(); + orc_column.resize(rows); + orc_column.numElements = rows; + + /// Calculate orc_column.hasNulls if (null_bytemap) - { orc_column.hasNulls = !memoryIsZero(null_bytemap->data(), 0, null_bytemap->size()); - if (orc_column.hasNulls) - { - orc_column.notNull.resize(null_bytemap->size()); - for (size_t i = 0; i < null_bytemap->size(); ++i) - orc_column.notNull[i] = !(*null_bytemap)[i]; - } - } else orc_column.hasNulls = false; + /// Fill orc_column.notNull + if (orc_column.hasNulls) + { + for (size_t i = 0; i < rows; ++i) + orc_column.notNull[i] = !(*null_bytemap)[i]; + } + else + { + for (size_t i = 0; i < rows; ++i) + orc_column.notNull[i] = 1; + } + /// ORC doesn't have unsigned types, so cast everything to signed and sign-extend to Int64 to /// make the ORC library calculate min and max correctly. switch (type->getTypeId()) @@ -516,27 +524,6 @@ void ORCBlockOutputFormat::writeColumn( } } -size_t ORCBlockOutputFormat::getColumnSize(const IColumn & column, DataTypePtr & type) -{ - if (type->getTypeId() == TypeIndex::Array) - { - auto nested_type = assert_cast(*type).getNestedType(); - const IColumn & nested_column = assert_cast(column).getData(); - return std::max(column.size(), getColumnSize(nested_column, nested_type)); - } - - return column.size(); -} - -size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk) -{ - size_t columns_num = chunk.getNumColumns(); - size_t max_column_size = 0; - for (size_t i = 0; i != columns_num; ++i) - max_column_size = std::max(max_column_size, getColumnSize(*chunk.getColumns()[i], data_types[i])); - return max_column_size; -} - void ORCBlockOutputFormat::consume(Chunk chunk) { if (!writer) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index 28837193d1a..06ecac9b820 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -69,11 +69,6 @@ private: void writeColumn(orc::ColumnVectorBatch & orc_column, const IColumn & column, DataTypePtr & type, const PaddedPODArray * null_bytemap); - /// These two functions are needed to know maximum nested size of arrays to - /// create an ORC Batch with the appropriate size - size_t getColumnSize(const IColumn & column, DataTypePtr & type); - size_t getMaxColumnSize(Chunk & chunk); - void prepareWriter(); const FormatSettings format_settings; From 3357275fa8c55bcc5371b4ff9c9a5d80e51ab689 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 9 Aug 2024 18:33:45 +0800 Subject: [PATCH 0680/1722] Fix MSAN issue caused by incorrect date format. --- src/IO/ReadHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..dd4aef23a25 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1432,7 +1432,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, date_broken_down_length + 1 + size)); else return false; } From 35f19522e745ef2267b4c6f99dfc5d7c1f7e78c3 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 9 Aug 2024 12:56:14 +0200 Subject: [PATCH 0681/1722] fix fuzzer --- src/Functions/FunctionsConversion.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 43ebe573582..c25bc44450f 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1579,7 +1579,7 @@ struct ConvertImpl IntervalKind to = typeid_cast(result_type.get())->getKind(); IntervalKind from = typeid_cast(arguments[0].type.get())->getKind(); - if (from == to) + if (from == to || arguments[0].column->empty()) return arguments[0].column; const auto &map = getGranularityMap(); From 4ced1f37e8e478ef806bd623a411916c18169dfe Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 11:56:53 +0100 Subject: [PATCH 0682/1722] Escape quote --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c59d36114ae..acdcda753d7 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -178,7 +178,7 @@ attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 202 clickhouse-client --query "CREATE TABLE minio_audit_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '\"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" @@ -186,7 +186,7 @@ ORDER BY tuple()" clickhouse-client --query "CREATE TABLE minio_server_logs ( log String, - event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') + event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '\"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC') ) ENGINE = MergeTree ORDER BY tuple()" From ade1228b9578d5c0d7124a9d5c40ac3207e48074 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 13:01:00 +0200 Subject: [PATCH 0683/1722] Fix order --- ...rofile_events_from_query_log_and_client.reference | 2 +- ...02675_profile_events_from_query_log_and_client.sh | 12 +++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index 9dbac8d34f2..448eca3e5b1 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -1,11 +1,11 @@ INSERT TO S3 -Successful write requests 3 S3Clients 1 S3CompleteMultipartUpload 1 S3CreateMultipartUpload 1 S3HeadObject 2 S3ReadRequestsCount 2 S3UploadPart 1 +Successful write requests 3 CHECK WITH query_log QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0 CREATE diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index cae20be79dc..6d770b308b5 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -18,11 +18,13 @@ WITH '(\\w+): (\\d+)' AS pattern, AND line NOT LIKE '%Microseconds%' AND line NOT LIKE '%S3DiskConnections%' AND line NOT LIKE '%S3DiskAddresses') AS pe_map -SELECT untuple(arrayJoin(pe_map) AS pe) -WHERE tupleElement(pe, 1) not like '%WriteRequests%' -UNION ALL -SELECT 'Successful write requests', - (pe_map['S3WriteRequestsCount'] - pe_map['S3WriteRequestsErrors'])::UInt64 +SELECT * FROM ( + SELECT untuple(arrayJoin(pe_map) AS pe) + WHERE tupleElement(pe, 1) not like '%WriteRequests%' + UNION ALL + SELECT 'Successful write requests', + (pe_map['S3WriteRequestsCount'] - pe_map['S3WriteRequestsErrors'])::UInt64 +) ORDER BY 1 " echo "CHECK WITH query_log" From 36c0c4562b8622b84012a12e29175f272bda2b0b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 13:14:30 +0200 Subject: [PATCH 0684/1722] Fix race in WithRetries --- src/Backups/WithRetries.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Backups/WithRetries.cpp b/src/Backups/WithRetries.cpp index 181e6331ac9..9f22085f5a9 100644 --- a/src/Backups/WithRetries.cpp +++ b/src/Backups/WithRetries.cpp @@ -68,13 +68,19 @@ const WithRetries::KeeperSettings & WithRetries::getKeeperSettings() const WithRetries::FaultyKeeper WithRetries::getFaultyZooKeeper() const { - /// We need to create new instance of ZooKeeperWithFaultInjection each time a copy a pointer to ZooKeeper client there + zkutil::ZooKeeperPtr current_zookeeper; + { + std::lock_guard lock(zookeeper_mutex); + current_zookeeper = zookeeper; + } + + /// We need to create new instance of ZooKeeperWithFaultInjection each time and copy a pointer to ZooKeeper client there /// The reason is that ZooKeeperWithFaultInjection may reset the underlying pointer and there could be a race condition /// when the same object is used from multiple threads. auto faulty_zookeeper = ZooKeeperWithFaultInjection::createInstance( settings.keeper_fault_injection_probability, settings.keeper_fault_injection_seed, - zookeeper, + current_zookeeper, log->name(), log); From ca4041847e4aa8acccd6ea31c0a18f2160c0dc7a Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 9 Aug 2024 19:15:41 +0800 Subject: [PATCH 0685/1722] Add tests --- src/IO/ReadHelpers.cpp | 4 ++-- ...215_fix_datetime_implicit_conversion.reference | 1 + .../03215_fix_datetime_implicit_conversion.sql | 15 +++++++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference create mode 100644 tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index dd4aef23a25..e69b4187b37 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1402,7 +1402,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", String(s, already_read_length)); else return false; } @@ -1432,7 +1432,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D s_pos[size] = 0; if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, date_broken_down_length + 1 + size)); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, size)); else return false; } diff --git a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql new file mode 100644 index 00000000000..70a8a3432a6 --- /dev/null +++ b/tests/queries/0_stateless/03215_fix_datetime_implicit_conversion.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS tab SYNC; + +CREATE TABLE tab +( + a DateTime, + pk String +) Engine = MergeTree() ORDER BY pk; + +INSERT INTO tab select cast(number, 'DateTime'), generateUUIDv4() FROM system.numbers LIMIT 1; + +SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'; +SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'; -- { serverError CANNOT_PARSE_DATETIME } +SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'; -- { serverError TYPE_MISMATCH } + +DROP TABLE IF EXISTS tab SYNC; From 1e3ccbc3ec81c5b9d79a034159181f1f6bdb195c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 9 Aug 2024 11:22:44 +0000 Subject: [PATCH 0686/1722] add perf test for subcolumns --- .../optimize_functions_to_subcolumns.xml | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 tests/performance/optimize_functions_to_subcolumns.xml diff --git a/tests/performance/optimize_functions_to_subcolumns.xml b/tests/performance/optimize_functions_to_subcolumns.xml new file mode 100644 index 00000000000..a246aae7950 --- /dev/null +++ b/tests/performance/optimize_functions_to_subcolumns.xml @@ -0,0 +1,27 @@ + + + 1 + 4 + + + + CREATE TABLE t_subcolumns (a Array(UInt64), s Nullable(String), m Map(String, UInt64)) ENGINE = MergeTree ORDER BY tuple() + + + + INSERT INTO t_subcolumns SELECT range(number % 20), toString(number), mapFromArrays(range(number % 20), range(number % 20)) FROM numbers_mt(50000000) + + + + OPTIMIZE TABLE t_subcolumns FINAL + + + SELECT count() FROM t_subcolumns WHERE NOT ignore(length(a)) + SELECT count() FROM t_subcolumns WHERE notEmpty(a) + SELECT count() FROM t_subcolumns WHERE NOT ignore(length(m)) + SELECT count() FROM t_subcolumns WHERE notEmpty(m) + SELECT count() FROM t_subcolumns WHERE isNotNull(s) + SELECT count(s) FROM t_subcolumns + + DROP TABLE t_subcolumns + From e8f2f65e62c65878463879396b7ebdceed48c5e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2024 11:51:23 +0000 Subject: [PATCH 0687/1722] Avoid converting type to string and back in _CAST --- src/Functions/CastOverloadResolver.cpp | 23 +++++++++++++++---- src/Functions/CastOverloadResolver.h | 6 ++++- src/Functions/toBool.cpp | 3 +-- src/Interpreters/ActionsDAG.cpp | 12 +++++----- src/Interpreters/castColumn.cpp | 6 ++--- .../optimizeUseAggregateProjection.cpp | 21 ++++++----------- src/Processors/Transforms/WindowTransform.cpp | 17 ++------------ src/Storages/MergeTree/KeyCondition.cpp | 7 ++---- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 20 +++++----------- ...rojection_with_normalized_states.reference | 1 + ...gate_projection_with_normalized_states.sql | 2 ++ 11 files changed, 52 insertions(+), 66 deletions(-) diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp index 49f63073aaf..6cb4d492fd8 100644 --- a/src/Functions/CastOverloadResolver.cpp +++ b/src/Functions/CastOverloadResolver.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +36,7 @@ FunctionBasePtr createFunctionBaseCast( class CastOverloadResolverImpl : public IFunctionOverloadResolver { public: - const char * getNameImpl() const + static const char * getNameImpl(CastType cast_type, bool internal) { if (cast_type == CastType::accurate) return "accurateCast"; @@ -49,7 +50,7 @@ public: String getName() const override { - return getNameImpl(); + return getNameImpl(cast_type, internal); } size_t getNumberOfArguments() const override { return 2; } @@ -79,10 +80,22 @@ public: } } + static FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional diagnostic) + { + if (cast_type == CastType::accurateOrNull && !isVariant(to)) + to = makeNullable(to); + + ColumnsWithTypeAndName arguments; + arguments.emplace_back(std::move(from)); + arguments.emplace_back().type = std::make_unique(); + + return createFunctionBaseCast(nullptr, getNameImpl(cast_type, true), arguments, to, diagnostic, cast_type); + } + protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { - return createFunctionBaseCast(context, getNameImpl(), arguments, return_type, diagnostic, cast_type); + return createFunctionBaseCast(context, getNameImpl(cast_type, internal), arguments, return_type, diagnostic, cast_type); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -130,9 +143,9 @@ private: }; -FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional diagnostic) +FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional diagnostic) { - return CastOverloadResolverImpl::create(ContextPtr{}, type, true, diagnostic); + return CastOverloadResolverImpl::createInternalCast(std::move(from), std::move(to), cast_type, std::move(diagnostic)); } REGISTER_FUNCTION(CastOverloadResolvers) diff --git a/src/Functions/CastOverloadResolver.h b/src/Functions/CastOverloadResolver.h index 7d98f774812..66f9d6cfcaf 100644 --- a/src/Functions/CastOverloadResolver.h +++ b/src/Functions/CastOverloadResolver.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -11,6 +12,9 @@ namespace DB class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; +class IFunctionBase; +using FunctionBasePtr = std::shared_ptr; + enum class CastType : uint8_t { nonAccurate, @@ -24,6 +28,6 @@ struct CastDiagnostic std::string column_to; }; -FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional diagnostic); +FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional diagnostic); } diff --git a/src/Functions/toBool.cpp b/src/Functions/toBool.cpp index 6f2c436c1ea..ac595d313e3 100644 --- a/src/Functions/toBool.cpp +++ b/src/Functions/toBool.cpp @@ -54,8 +54,7 @@ namespace } }; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); - auto func_cast = func_builder_cast->build(cast_args); + auto func_cast = createInternalCast(arguments[0], result_type, CastType::nonAccurate, {}); return func_cast->execute(cast_args, result_type, arguments[0].column->size()); } }; diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index df1c0aa1f2a..2a594839c6a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -301,11 +301,11 @@ const ActionsDAG::Node & ActionsDAG::addCast(const Node & node_to_cast, const Da column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); column.type = std::make_shared(); - const auto * cast_type_constant_node = &addColumn(std::move(column)); + const auto * cast_type_constant_node = &addColumn(column); ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); + auto func_base_cast = createInternalCast(ColumnWithTypeAndName{node_to_cast.result_type, node_to_cast.result_name}, cast_type, CastType::nonAccurate, {}); - return addFunction(func_builder_cast, std::move(children), result_name); + return addFunction(func_base_cast, std::move(children), result_name); } const ActionsDAG::Node & ActionsDAG::addFunctionImpl( @@ -1547,11 +1547,11 @@ ActionsDAG ActionsDAG::makeConvertingActions( const auto * left_arg = dst_node; CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name}; - FunctionOverloadResolverPtr func_builder_cast - = createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic)); + ColumnWithTypeAndName left_column{nullptr, dst_node->result_type, {}}; + auto func_base_cast = createInternalCast(std::move(left_column), res_elem.type, CastType::nonAccurate, std::move(diagnostic)); NodeRawConstPtrs children = { left_arg, right_arg }; - dst_node = &actions_dag.addFunction(func_builder_cast, std::move(children), {}); + dst_node = &actions_dag.addFunction(func_base_cast, std::move(children), {}); } if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column))) diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 906dfb84b14..a779c9bc34d 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -26,11 +26,9 @@ static ColumnPtr castColumn(CastType cast_type, const ColumnWithTypeAndName & ar "" } }; - auto get_cast_func = [cast_type, &arguments] + auto get_cast_func = [from = arg, to = type, cast_type] { - - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(cast_type, {}); - return func_builder_cast->build(arguments); + return createInternalCast(from, to, cast_type, {}); }; FunctionBasePtr func_cast = cache ? cache->getOrSet(cast_type, from_name, to_name, std::move(get_cast_func)) : get_cast_func(); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 52d1931c51e..b31ee7ea53c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -255,20 +255,13 @@ static void appendAggregateFunctions( const auto * node = input; - if (node->result_name != aggregate.column_name) - { - if (DataTypeAggregateFunction::strictEquals(type, node->result_type)) - { - node = &proj_dag.addAlias(*node, aggregate.column_name); - } - else - { - /// Cast to aggregate types specified in query if it's not - /// strictly the same as the one specified in projection. This - /// is required to generate correct results during finalization. - node = &proj_dag.addCast(*node, type, aggregate.column_name); - } - } + if (!DataTypeAggregateFunction::strictEquals(type, node->result_type)) + /// Cast to aggregate types specified in query if it's not + /// strictly the same as the one specified in projection. This + /// is required to generate correct results during finalization. + node = &proj_dag.addCast(*node, type, aggregate.column_name); + else if (node->result_name != aggregate.column_name) + node = &proj_dag.addAlias(*node, aggregate.column_name); proj_dag_outputs.push_back(node); } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index c26cd7cc8c3..c27c230c741 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2337,22 +2337,9 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction argument_types[2]->getName()); } - const auto from_name = argument_types[2]->getName(); - const auto to_name = argument_types[0]->getName(); - ColumnsWithTypeAndName arguments + auto get_cast_func = [from = argument_types[2], to = argument_types[0]] { - { argument_types[2], "" }, - { - DataTypeString().createColumnConst(0, to_name), - std::make_shared(), - "" - } - }; - - auto get_cast_func = [&arguments] - { - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::accurate, {}); - return func_builder_cast->build(arguments); + return createInternalCast({from, {}}, to, CastType::accurate, {}); }; func_cast = get_cast_func(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dfb43c4e75d..aa7a498d5a3 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1956,11 +1956,8 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme auto common_type_maybe_nullable = (key_expr_type_is_nullable && !common_type->isNullable()) ? DataTypePtr(std::make_shared(common_type)) : common_type; - ColumnsWithTypeAndName arguments{ - {nullptr, key_expr_type, ""}, - {DataTypeString().createColumnConst(1, common_type_maybe_nullable->getName()), common_type_maybe_nullable, ""}}; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); - auto func_cast = func_builder_cast->build(arguments); + + auto func_cast = createInternalCast({key_expr_type, {}}, common_type_maybe_nullable, CastType::nonAccurate, {}); /// If we know the given range only contains one value, then we treat all functions as positive monotonic. if (!single_point && !func_cast->hasInformationAboutMonotonicity()) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 36ff6c0a4bd..9c82817e8cb 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -152,23 +152,15 @@ const ActionsDAG::Node & addFunction( const ActionsDAG::Node & addCast( const ActionsDAGPtr & dag, const ActionsDAG::Node & node_to_cast, - const String & type_name, + const DataTypePtr & to_type, OriginalToNewNodeMap & node_remap) { - if (node_to_cast.result_type->getName() == type_name) + if (!node_to_cast.result_type->equals(*to_type)) return node_to_cast; - Field cast_type_constant_value(type_name); - - ColumnWithTypeAndName column; - column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); - column.type = std::make_shared(); - - const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); - ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {}); - - return addFunction(dag, func_builder_cast, std::move(children), node_remap); + const auto & new_node = dag->addCast(node_to_cast, to_type, {}); + node_remap[new_node.result_name] = {dag.get(), &new_node}; + return new_node; } /// Normalizes the filter node by adding AND with a constant true. @@ -332,7 +324,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction /// Build AND(last_step_result_node, true) const auto & and_node = addAndTrue(last_step_dag, *last_step_result_node_info.node, node_remap); /// Build CAST(and_node, type of PREWHERE column) - const auto & cast_node = addCast(last_step_dag, and_node, output->result_type->getName(), node_remap); + const auto & cast_node = addCast(last_step_dag, and_node, output->result_type, node_remap); /// Add alias for the result with the name of the PREWHERE column const auto & prewhere_result_node = last_step_dag->addAlias(cast_node, output->result_name); last_step_dag->addOrReplaceInOutputs(prewhere_result_node); diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference index 25aa9dc5dec..37993873983 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference @@ -1,2 +1,3 @@ 3 950 990 500 2000 +[950] [999] diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql index 5375823aa8e..956bf3711a2 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql @@ -29,4 +29,6 @@ FROM cluster('test_cluster_two_shards', currentDatabase(), r) WHERE a = 'x' settings prefer_localhost_replica=0; +SELECT quantilesTimingMerge(0.95)(q), quantilesTimingMerge(toInt64(1))(q) FROM remote('127.0.0.{1,2}', currentDatabase(), r); + DROP TABLE r; From 3d850f8ceb0ca5cfae26e8faa7c4d900cc4e8fda Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 13:58:02 +0200 Subject: [PATCH 0688/1722] fix --- src/Processors/Sources/ShellCommandSource.cpp | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 23359367a9b..f55a3713215 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -70,17 +70,16 @@ static void makeFdBlocking(int fd) static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_milliseconds) { + auto logger = getLogger("TimeoutReadBufferFromFileDescriptor"); + auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; + int res; while (true) { Stopwatch watch; - auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; - LOG_TEST( - getLogger("TimeoutReadBufferFromFileDescriptor"), - "Polling descriptors: {}", - fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); + LOG_TEST(logger, "Polling descriptors: {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", ")); res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); @@ -92,11 +91,7 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond const auto elapsed = watch.elapsedMilliseconds(); if (timeout_milliseconds <= elapsed) { - LOG_TEST( - getLogger("TimeoutReadBufferFromFileDescriptor"), - "Timeout exceeded: elapsed={}, timeout={}", - elapsed, - timeout_milliseconds); + LOG_TEST(logger, "Timeout exceeded: elapsed={}, timeout={}", elapsed, timeout_milliseconds); break; } timeout_milliseconds -= elapsed; @@ -107,9 +102,8 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond } } - auto describe_fd = [](const auto & pollfd) { return fmt::format("(fd={}, flags={})", pollfd.fd, fcntl(pollfd.fd, F_GETFL)); }; LOG_TEST( - getLogger("TimeoutReadBufferFromFileDescriptor"), + logger, "Poll for descriptors: {} returned {}", fmt::join(std::span(pfds, pfds + num) | std::views::transform(describe_fd), ", "), res); From c3ab8266ebf86da96712acf97ee0efe9e924d777 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rodolphe=20Dug=C3=A9=20de=20Bernonville?= Date: Fri, 9 Aug 2024 14:05:18 +0200 Subject: [PATCH 0689/1722] odbc: get http retry from server configuration --- src/BridgeHelper/XDBCBridgeHelper.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 5f4c7fd8381..0630a0f24e3 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -100,6 +100,7 @@ protected: auto buf = BuilderRWBufferFromHTTP(getPingURI()) .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withTimeouts(getHTTPTimeouts()) + .withSettings(getContext()->getReadSettings()) .create(credentials); return checkString(PING_OK_ANSWER, *buf); @@ -206,6 +207,7 @@ protected: .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withTimeouts(getHTTPTimeouts()) + .withSettings(getContext()->getReadSettings()) .create(credentials); bool res = false; @@ -232,6 +234,7 @@ protected: .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withTimeouts(getHTTPTimeouts()) + .withSettings(getContext()->getReadSettings()) .create(credentials); std::string character; From 1875e8d9cd581871c247e3a4cc58f1f57ffa0659 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 9 Aug 2024 14:06:52 +0200 Subject: [PATCH 0690/1722] Fix UB in tumbleEnd and tumbleStart This was causing segfaults because of a NULL pointer dereference --- src/Functions/FunctionsTimeWindow.cpp | 7 ++++++- .../01049_window_view_window_functions.reference | 2 ++ .../0_stateless/01049_window_view_window_functions.sql | 2 ++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index faea9c6ba58..88b85c48326 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -267,7 +267,12 @@ struct TimeWindowImpl { auto type = WhichDataType(arguments[0].type); if (type.isTuple()) - return std::static_pointer_cast(arguments[0].type)->getElement(0); + { + const auto & tuple_elems = std::static_pointer_cast(arguments[0].type)->getElements(); + if (tuple_elems.empty()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Tuple passed to {} should not be empty", function_name); + return tuple_elems[0]; + } else if (type.isUInt32()) return std::make_shared(); else diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index 47d1ccc57dd..073301104d2 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -69,3 +69,5 @@ SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DA 2019-01-10 00:00:00 SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleEnd(tuple()); -- { serverError ILLEGAL_COLUMN } diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.sql b/tests/queries/0_stateless/01049_window_view_window_functions.sql index 3638dd1a3b2..fb2b4b4949a 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.sql +++ b/tests/queries/0_stateless/01049_window_view_window_functions.sql @@ -39,3 +39,5 @@ SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DA SELECT hopStart(tuple()); -- { serverError ILLEGAL_COLUMN } SELECT hopEnd(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleStart(tuple()); -- { serverError ILLEGAL_COLUMN } +SELECT tumbleEnd(tuple()); -- { serverError ILLEGAL_COLUMN } From 20563bc6cbc6e70eb6926c5f21fded356270f40f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 12:19:44 +0000 Subject: [PATCH 0691/1722] Make test work with ReplicatedDatabase in test --- .../0_stateless/03217_filtering_in_system_tables.sql | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql index 2ce63559b99..eb506dfe39a 100644 --- a/tests/queries/0_stateless/03217_filtering_in_system_tables.sql +++ b/tests/queries/0_stateless/03217_filtering_in_system_tables.sql @@ -9,9 +9,11 @@ CREATE TABLE test_03217_system_tables_replica_2(x UInt32) ORDER BY x; -- Make sure we can read both replicas -SELECT 'both', database, table, replica_name FROM system.replicas WHERE database = currentDatabase(); +-- The replica name might be altered because of `_functional_tests_helper_database_replicated_replace_args_macros`, +-- thus we need to use `left` +SELECT 'both', database, table, left(replica_name, 2) FROM system.replicas WHERE database = currentDatabase(); -- If filtering is not done correctly on database-table column, then this query report to read 2 rows, which are the above tables -SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name = 'r1'; +SELECT database, table, left(replica_name, 2) FROM system.replicas WHERE database = currentDatabase() AND table = 'test_03217_system_tables_replica_1' AND replica_name LIKE 'r1%'; SYSTEM FLUSH LOGS; -- argMax is necessary to make the test repeatable @@ -24,5 +26,5 @@ SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 -- StorageSystemReplicas SELECT argMax(read_rows, event_time_microseconds) FROM system.query_log WHERE 1 AND current_database = currentDatabase() - AND query LIKE '%SELECT database, table, replica_name FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name = \'r1\';' + AND query LIKE '%SELECT database, table, left(replica_name, 2) FROM system.replicas WHERE database = currentDatabase() AND table = \'test_03217_system_tables_replica_1\' AND replica_name LIKE \'r1\%\';' AND type = 'QueryFinish'; From 96b54df1638043e27887099df5f7cf310d1e7fa4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 9 Aug 2024 15:04:03 +0200 Subject: [PATCH 0692/1722] fix bugprone-macro-parentheses --- src/Interpreters/SystemLog.cpp | 4 +++- src/Interpreters/SystemLog.h | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index d4403b72583..832c39bfaf8 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -284,11 +284,13 @@ ASTPtr getCreateTableQueryClean(const StorageID & table_id, ContextPtr context) SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config) { +/// NOLINTBEGIN(bugprone-macro-parentheses) #define CREATE_PUBLIC_MEMBERS(log_type, member, descr) \ member = createSystemLog(global_context, "system", #member, config, #member, descr); \ LIST_OF_ALL_SYSTEM_LOGS(CREATE_PUBLIC_MEMBERS) #undef CREATE_PUBLIC_MEMBERS +/// NOLINTEND(bugprone-macro-parentheses) if (session_log) global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); @@ -333,7 +335,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf std::vector SystemLogs::getAllLogs() const { #define GET_RAW_POINTERS(log_type, member, descr) \ - member.get(), \ + (member).get(), \ std::vector result = { LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 24ef6a18eb8..9e1af3578bd 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -60,11 +60,13 @@ namespace DB }; */ +/// NOLINTBEGIN(bugprone-macro-parentheses) #define FORWARD_DECLARATION(log_type, member, descr) \ class log_type; \ LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) #undef FORWARD_DECLARATION +/// NOLINTEND(bugprone-macro-parentheses) /// System logs should be destroyed in destructor of the last Context and before tables, From 21be195dd8f841c3cfaf453bb7faba02627b9c0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 13:14:09 +0000 Subject: [PATCH 0693/1722] Revert unnecessary change --- .../clickhouse_path/format_schemas/test.capnp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp index 247e7b9ceca..44f1961205b 100644 --- a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/test.capnp @@ -7,4 +7,4 @@ struct TestRecordStruct val1 @2 : Text; val2 @3 : Float32; val3 @4 : UInt8; -} +} \ No newline at end of file From 8cf5f6d6168342a69b188b17588566a4ac85fa69 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 9 Aug 2024 13:20:05 +0000 Subject: [PATCH 0694/1722] Add empty cell to reports when time is missing --- tests/ci/report.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 0b6c818aed0..15b1512896a 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -770,10 +770,12 @@ def create_test_html_report( row.append(f'{test_result.status}') colspan += 1 + row.append("") if test_result.time is not None: has_test_time = True - row.append(f"{test_result.time}") - colspan += 1 + row.append(str(test_result.time)) + row.append("") + colspan += 1 if test_result.log_urls is not None: has_log_urls = True From b757522fc4ac545451acc398ab230323fb7c0fd3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 14:20:57 +0100 Subject: [PATCH 0695/1722] fix build --- programs/keeper/Keeper.cpp | 2 +- src/Server/HTTPHandlerFactory.cpp | 41 ++----------------------- src/Server/PrometheusRequestHandler.cpp | 4 +-- src/Server/PrometheusRequestHandler.h | 7 +++-- 4 files changed, 10 insertions(+), 44 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index ae51a62ff9c..a447a9e50f6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -515,7 +515,7 @@ try "Prometheus: http://" + address.toString(), std::make_unique( std::move(my_http_context), - createKeeperPrometheusHandlerFactory(config_getter(), async_metrics, "PrometheusHandler-factory"), + createKeeperPrometheusHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 0ee45783d52..fc31ad2874e 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -122,7 +122,8 @@ static inline auto createHandlersFactoryFromConfig( } else if (handler_type == "prometheus") { - main_handler_factory->addHandler(createPrometheusHandlerFactoryForHTTPRule(config, prefix + "." + key, async_metrics)); + main_handler_factory->addHandler( + createPrometheusHandlerFactoryForHTTPRule(server, config, prefix + "." + key, async_metrics)); } else if (handler_type == "replicas_status") { @@ -199,19 +200,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); else if (name == "PrometheusHandler-factory") -<<<<<<< HEAD - { - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(config, metrics_writer, name); - } -||||||| 02b8d563e3a - { - auto metrics_writer = std::make_shared(config, "prometheus", async_metrics); - return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); - } -======= return createPrometheusHandlerFactory(server, config, async_metrics, name); ->>>>>>> master throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); } @@ -298,34 +287,8 @@ void addDefaultHandlersFactory( ); factory.addHandler(query_handler); -<<<<<<< HEAD - /// We check that prometheus handler will be served on current (default) port. - /// Otherwise it will be created separately, see createHandlerFactory(...). - if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) - { - auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator - = [writer]() -> std::unique_ptr { return std::make_unique(writer); }; - auto prometheus_handler = std::make_shared>(std::move(creator)); - prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - prometheus_handler->allowGetAndHeadRequest(); -||||||| 02b8d563e3a - /// We check that prometheus handler will be served on current (default) port. - /// Otherwise it will be created separately, see createHandlerFactory(...). - if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0) - { - auto writer = std::make_shared(config, "prometheus", async_metrics); - auto creator = [&server, writer] () -> std::unique_ptr - { - return std::make_unique(server, writer); - }; - auto prometheus_handler = std::make_shared>(std::move(creator)); - prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics")); - prometheus_handler->allowGetAndHeadRequest(); -======= /// createPrometheusHandlerFactoryForHTTPRuleDefaults() can return nullptr if prometheus protocols must not be served on http port. if (auto prometheus_handler = createPrometheusHandlerFactoryForHTTPRuleDefaults(server, config, async_metrics)) ->>>>>>> master factory.addHandler(prometheus_handler); } diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 52cda92d9b4..ae1fb6d629e 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -95,7 +95,7 @@ public: class PrometheusRequestHandler::ImplWithContext : public Impl { public: - explicit ImplWithContext(PrometheusRequestHandler & parent) : Impl(parent), default_settings(parent.server.context()->getSettingsRef()) { } + explicit ImplWithContext(PrometheusRequestHandler & parent) : Impl(parent), default_settings(server().context()->getSettingsRef()) { } virtual void handlingRequestWithContext(HTTPServerRequest & request, HTTPServerResponse & response) = 0; @@ -353,7 +353,7 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe if (request.getVersion() == HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); - setResponseDefaultHeaders(response); + setResponseDefaultHeaders(response); impl->beforeHandlingRequest(request); impl->handleRequest(request, response); diff --git a/src/Server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h index 7aeed11d6b8..281ecf5260e 100644 --- a/src/Server/PrometheusRequestHandler.h +++ b/src/Server/PrometheusRequestHandler.h @@ -15,8 +15,11 @@ class WriteBufferFromHTTPServerResponse; class PrometheusRequestHandler : public HTTPRequestHandler { public: - PrometheusRequestHandler(const PrometheusRequestHandlerConfig & config_, - const AsynchronousMetrics & async_metrics_, std::shared_ptr metrics_writer_); + PrometheusRequestHandler( + IServer & server_, + const PrometheusRequestHandlerConfig & config_, + const AsynchronousMetrics & async_metrics_, + std::shared_ptr metrics_writer_); ~PrometheusRequestHandler() override; void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event_) override; From 65ebcd6f21b26144cb47e6b71c939517b1fb38a2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Aug 2024 13:55:47 +0000 Subject: [PATCH 0696/1722] Fixing test. --- .../0_stateless/01656_test_query_log_factories_info.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference index 47b3133ceca..44531c19ab7 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference @@ -17,7 +17,7 @@ used_functions ['repeat'] arraySort(used_data_type_families) -['Array','Int32','Nullable','String'] +['Int32','Nullable','String'] used_database_engines ['Atomic'] From a74cc601cd11ab32df61c46f9950c051fd8bb4da Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 16:04:27 +0200 Subject: [PATCH 0697/1722] A catch-all URL style for S3 --- src/IO/S3/URI.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 64962f63edb..9c80b377661 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -136,7 +136,16 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) validateBucket(bucket, uri); } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket or key name are invalid in S3 URI."); + { + /// Custom endpoint, e.g. a public domain of Cloudflare R2, + /// which could be served by a custom server-side code. + storage_name = "S3"; + bucket = "default"; + is_virtual_hosted_style = false; + endpoint = uri.getScheme() + "://" + uri.getAuthority(); + if (!uri.getPath().empty()) + key = uri.getPath().substr(1); + } } void URI::addRegionToURI(const std::string ®ion) From 43a38fb5f0563f50d38cf5d988db3b181b64f606 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Aug 2024 15:11:08 +0100 Subject: [PATCH 0698/1722] rm redundant file --- programs/server/config.d/listen.xml | 3 --- 1 file changed, 3 deletions(-) delete mode 100644 programs/server/config.d/listen.xml diff --git a/programs/server/config.d/listen.xml b/programs/server/config.d/listen.xml deleted file mode 100644 index f94e5c88568..00000000000 --- a/programs/server/config.d/listen.xml +++ /dev/null @@ -1,3 +0,0 @@ - - :: - From 9a6d98cd203ee9a6cdde4450aaf72d109a9719c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 16:15:28 +0200 Subject: [PATCH 0699/1722] Update test --- .../integration/test_odbc_interaction/test.py | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 0d0d7a0afb1..9d4ca5ad49f 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -51,9 +51,9 @@ create_table_sql_nullable_template = """ """ -def skip_test_msan(instance): - if instance.is_built_with_memory_sanitizer(): - pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") +def skip_test_sanitizers(instance): + if instance.is_built_with_sanitizer(): + pytest.skip("Sanitizers cannot work with third-party shared libraries") def get_mysql_conn(): @@ -208,7 +208,7 @@ def started_cluster(): def test_mysql_odbc_select_nullable(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) mysql_setup = node1.odbc_drivers["MySQL"] table_name = "test_insert_nullable_select" @@ -248,7 +248,7 @@ def test_mysql_odbc_select_nullable(started_cluster): def test_mysql_simple_select_works(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) mysql_setup = node1.odbc_drivers["MySQL"] @@ -331,7 +331,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nulla def test_mysql_insert(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) mysql_setup = node1.odbc_drivers["MySQL"] table_name = "test_insert" @@ -374,7 +374,7 @@ def test_mysql_insert(started_cluster): def test_sqlite_simple_select_function_works(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) sqlite_setup = node1.odbc_drivers["SQLite3"] sqlite_db = sqlite_setup["Database"] @@ -438,7 +438,7 @@ def test_sqlite_simple_select_function_works(started_cluster): def test_sqlite_table_function(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) sqlite_setup = node1.odbc_drivers["SQLite3"] sqlite_db = sqlite_setup["Database"] @@ -470,7 +470,7 @@ def test_sqlite_table_function(started_cluster): def test_sqlite_simple_select_storage_works(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) sqlite_setup = node1.odbc_drivers["SQLite3"] sqlite_db = sqlite_setup["Database"] @@ -503,7 +503,7 @@ def test_sqlite_simple_select_storage_works(started_cluster): def test_sqlite_odbc_hashed_dictionary(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] node1.exec_in_container( @@ -586,7 +586,7 @@ def test_sqlite_odbc_hashed_dictionary(started_cluster): def test_sqlite_odbc_cached_dictionary(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] node1.exec_in_container( @@ -635,7 +635,7 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) try: conn = get_postgres_conn(started_cluster) @@ -663,7 +663,7 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) try: conn = get_postgres_conn(started_cluster) @@ -685,7 +685,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): def test_no_connection_pooling(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) try: conn = get_postgres_conn(started_cluster) @@ -717,7 +717,7 @@ def test_no_connection_pooling(started_cluster): def test_postgres_insert(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) conn = get_postgres_conn(started_cluster) @@ -754,7 +754,7 @@ def test_postgres_insert(started_cluster): def test_odbc_postgres_date_data_type(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) try: conn = get_postgres_conn(started_cluster) @@ -783,7 +783,7 @@ def test_odbc_postgres_date_data_type(started_cluster): def test_odbc_postgres_conversions(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) try: conn = get_postgres_conn(started_cluster) @@ -841,7 +841,7 @@ def test_odbc_postgres_conversions(started_cluster): def test_odbc_cyrillic_with_varchar(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) conn = get_postgres_conn(started_cluster) cursor = conn.cursor() @@ -868,7 +868,7 @@ def test_odbc_cyrillic_with_varchar(started_cluster): def test_many_connections(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) conn = get_postgres_conn(started_cluster) cursor = conn.cursor() @@ -894,7 +894,7 @@ def test_many_connections(started_cluster): def test_concurrent_queries(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) conn = get_postgres_conn(started_cluster) cursor = conn.cursor() @@ -948,7 +948,7 @@ def test_concurrent_queries(started_cluster): def test_odbc_long_column_names(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) conn = get_postgres_conn(started_cluster) cursor = conn.cursor() @@ -986,7 +986,7 @@ def test_odbc_long_column_names(started_cluster): def test_odbc_long_text(started_cluster): - skip_test_msan(node1) + skip_test_sanitizers(node1) conn = get_postgres_conn(started_cluster) cursor = conn.cursor() From 9d91b600caa0d5cddcf4d13a8c80ac85c9077fca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 16:27:35 +0200 Subject: [PATCH 0700/1722] Add a test --- tests/integration/test_s3_imds/test_simple.py | 2 +- .../03221_s3_imds_decent_timeout.reference | 1 + .../0_stateless/03221_s3_imds_decent_timeout.sh | 16 ++++++++++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03221_s3_imds_decent_timeout.reference create mode 100755 tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh diff --git a/tests/integration/test_s3_imds/test_simple.py b/tests/integration/test_s3_imds/test_simple.py index 0dacac2b0b9..4884c824f99 100644 --- a/tests/integration/test_s3_imds/test_simple.py +++ b/tests/integration/test_s3_imds/test_simple.py @@ -56,7 +56,7 @@ def test_credentials_from_metadata(): ) expected_logs = [ - "Calling EC2MetadataService to get token failed, falling back to less secure way", + "Calling EC2MetadataService to get token failed, falling back to a less secure way", "Getting default credentials for ec2 instance from resolver:8080", "Calling EC2MetadataService resource, /latest/meta-data/iam/security-credentials returned credential string myrole", "Calling EC2MetadataService resource /latest/meta-data/iam/security-credentials/myrole", diff --git a/tests/queries/0_stateless/03221_s3_imds_decent_timeout.reference b/tests/queries/0_stateless/03221_s3_imds_decent_timeout.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03221_s3_imds_decent_timeout.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh b/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh new file mode 100755 index 00000000000..fb55539d04a --- /dev/null +++ b/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# ^ requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Inaccessible IMDS should not introduce large delays, so this query should reply quickly at least sometimes: +while true +do + # This host (likely) drops packets sent to it (does not reply), so it is good for testing timeouts. + # At the same time, we expect that google.com does not drop packets and quickly replies with 404, which is a non-retriable error for S3. + AWS_EC2_METADATA_SERVICE_ENDPOINT='https://10.255.255.255/' ${CLICKHOUSE_LOCAL} --time --query "SELECT * FROM s3('https://google.com/test')" |& grep -v -F 404 | + ${CLICKHOUSE_LOCAL} --input-format TSV "SELECT c1::Float64 < 1 FROM table" | grep 1 && break +done From 79cfffbaf88c3c34213d5144fd20f7888b6cabe0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 15:57:16 +0100 Subject: [PATCH 0701/1722] Even bigger queue --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index acdcda753d7..a030be92506 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -193,8 +193,8 @@ ORDER BY tuple()" # create minio log webhooks for both audit and server logs # use async inserts to avoid creating too many parts -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=300000 -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=300000 +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 max_retries=100 retry=1 From e3fded6c94b7fe81af750a6976d704dbeea7a4a5 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 9 Aug 2024 15:13:55 +0000 Subject: [PATCH 0702/1722] Log peration error for zk multi request --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 064ac2261ec..1a9ed4f1ee7 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1570,7 +1570,7 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, size_t failed_op_index_, const Coordination::Requests & requests_, const Coordination::Responses & responses_) - : KeeperException(exception_code, "Transaction failed: Op #{}, path", failed_op_index_), + : KeeperException(exception_code, "Transaction failed ({}): Op #{}, path", exception_code, failed_op_index_), requests(requests_), responses(responses_), failed_op_index(failed_op_index_) { addMessage(getPathForFirstFailedOp()); From 97eded0ac7aa41a9320729b418c8ab2ff1821202 Mon Sep 17 00:00:00 2001 From: kruglov Date: Fri, 9 Aug 2024 17:38:24 +0300 Subject: [PATCH 0703/1722] Fixed test_dependent_loading. event_time_microseconds has two dates connected with "\n" --- .../test_postgresql_replica_database_engine_2/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 75edb22aab1..7fdd17625a9 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1127,9 +1127,13 @@ def test_dependent_loading(started_cluster): nested_time = instance.query( f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{uuid}_nested' and message not like '%like%'" ).strip() - time = instance.query( - f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{table}' and message not like '%like%'" - ).strip() + time = ( + instance.query( + f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{table}' and message not like '%like%'" + ) + .strip() + .split("\n")[-1] + ) instance.query( f"SELECT toDateTime64('{nested_time}', 6) < toDateTime64('{time}', 6)" ) From 6ded5e1c8b994ad2332468e605b17a74e8d5675f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 9 Aug 2024 23:50:03 +0800 Subject: [PATCH 0704/1722] Some fixups --- src/IO/ReadHelpers.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index e69b4187b37..b484f80250d 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1399,10 +1399,8 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D size_t size = buf.read(s_pos, remaining_date_size); if (size != remaining_date_size) { - s_pos[size] = 0; - if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", String(s, already_read_length)); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", std::string_view(s, already_read_length + size)); else return false; } @@ -1429,10 +1427,8 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D if (size != time_broken_down_length) { - s_pos[size] = 0; - if constexpr (throw_exception) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", String(s, size)); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", std::string_view(s, size)); else return false; } From a3d8db6e1eb27d6a8fa81bbf43c8ffb171714c0b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 9 Aug 2024 19:05:37 +0200 Subject: [PATCH 0705/1722] updates due to review --- .../data-types/special-data-types/interval.md | 21 +++++++------- src/DataTypes/getLeastSupertype.cpp | 13 ++++----- src/DataTypes/getLeastSupertype.h | 21 ++------------ src/Functions/FunctionsConversion.cpp | 28 +++---------------- 4 files changed, 24 insertions(+), 59 deletions(-) diff --git a/docs/zh/sql-reference/data-types/special-data-types/interval.md b/docs/zh/sql-reference/data-types/special-data-types/interval.md index e16f6d5f84f..e05869b2df8 100644 --- a/docs/zh/sql-reference/data-types/special-data-types/interval.md +++ b/docs/zh/sql-reference/data-types/special-data-types/interval.md @@ -55,28 +55,29 @@ SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY └─────────────────────┴───────────────────────────────┘ ``` -也å¯ä»¥åŒæ™‚使用多個間隔: +ä¸åŒç±»åž‹çš„é—´éš”ä¸èƒ½åˆå¹¶ã€‚ ä½ ä¸èƒ½ä½¿ç”¨è¯¸å¦‚ `4 DAY 1 HOUR` 的时间间隔. 以å°äºŽæˆ–等于时间间隔最å°å•ä½çš„å•ä½æ¥æŒ‡å®šé—´éš”,例如,时间间隔 `1 day and an hour` å¯ä»¥è¡¨ç¤ºä¸º `25 HOUR` 或 `90000 SECOND`. + +ä½ ä¸èƒ½å¯¹ `Interval` 类型的值执行算术è¿ç®—,但你å¯ä»¥å‘ `Date` 或 `DateTime` æ•°æ®ç±»åž‹çš„值添加ä¸åŒç±»åž‹çš„时间间隔,例如: ``` sql -SELECT now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) +SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR ``` ``` text -┌───current_date_time─┬─plus(current_date_time, plus(toIntervalDay(4), toIntervalHour(3)))─┠-│ 2024-08-08 18:31:39 │ 2024-08-12 21:31:39 │ -└─────────────────────┴────────────────────────────────────────────────────────────────────┘ +┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┠+│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ +└─────────────────────┴────────────────────────────────────────────────────────┘ ``` -並比較ä¸åŒç›´æ•¸çš„值: +以下查询将导致异常: ``` sql -SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); +select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) ``` ``` text -┌─less(toIntervalMicrosecond(179999999), toIntervalMinute(3))─┠-│ 1 │ -└─────────────────────────────────────────────────────────────┘ +Received exception from server (version 19.14.1): +Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. ``` ## å¦è¯·å‚阅 {#see-also} diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 674284460dc..8bcec49815f 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -230,8 +230,7 @@ void convertUInt64toInt64IfPossible(const DataTypes & types, TypeIndexSet & type DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet &types_set) { - const auto& granularity_map = getGranularityMap(); - int min_granularity = std::get<0>(granularity_map.at(IntervalKind::Kind::Year)); + auto min_interval = IntervalKind::Kind::Year; DataTypePtr smallest_type; bool is_higher_interval = false; // For Years, Quarters and Months @@ -240,18 +239,18 @@ DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet & { if (const auto * interval_type = typeid_cast(type.get())) { - int current_granularity = std::get<0>(granularity_map.at(interval_type->getKind())); - if (current_granularity > 8) + auto current_interval = interval_type->getKind().kind; + if (current_interval > IntervalKind::Kind::Week) is_higher_interval = true; - if (current_granularity < min_granularity) + if (current_interval < min_interval) { - min_granularity = current_granularity; + min_interval = current_interval; smallest_type = type; } } } - if (is_higher_interval && min_granularity <= 8) + if (is_higher_interval && min_interval <= IntervalKind::Kind::Week) throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); if (smallest_type) diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index c584eb83011..5ea2b6417b2 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -50,24 +50,9 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types); DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); -/// A map that enumerated all interval kinds in ascending order with a conversion value to a next interval -inline const std::unordered_map> & getGranularityMap() -{ - static std::unordered_map> granularity_map = - { - {IntervalKind::Kind::Nanosecond, {1, 1000}}, - {IntervalKind::Kind::Microsecond, {2, 1000}}, - {IntervalKind::Kind::Millisecond, {3, 1000}}, - {IntervalKind::Kind::Second, {4, 60}}, - {IntervalKind::Kind::Minute, {5, 60}}, - {IntervalKind::Kind::Hour, {6, 24}}, - {IntervalKind::Kind::Day, {7, 7}}, - {IntervalKind::Kind::Week, {8, 4}}, - {IntervalKind::Kind::Month, {9, 3}}, - {IntervalKind::Kind::Quarter, {10, 4}}, - {IntervalKind::Kind::Year, {11, 1}} - }; - return granularity_map; +/// A vector that shows the conversion rates to the next Interval type starting from NanoSecond +static std::vector interval_conversions = {1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4, 1}; + } } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index c25bc44450f..25c6bbcbfef 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1582,42 +1582,22 @@ struct ConvertImpl if (from == to || arguments[0].column->empty()) return arguments[0].column; - const auto &map = getGranularityMap(); Int64 conversion_factor = 1; Int64 result_value; - int from_position = map.at(from).first; - int to_position = map.at(to).first; // Positions of each interval according to granurality map + int from_position = static_cast(from.kind); + int to_position = static_cast(to.kind); // Positions of each interval according to granurality map if (from_position < to_position) { for (int i = from_position - 1; i <= to_position; ++i) - { - // Find the kind that matches this position - for (const auto &entry : map) - { - if (entry.second.first == i) - { - conversion_factor *= entry.second.second; - break; - } - } - } + conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) / conversion_factor; } else { for (int i = from_position - 1; i >= to_position; --i) - { - for (const auto &entry : map) - { - if (entry.second.first == i) - { - conversion_factor *= entry.second.second; - break; - } - } - } + conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) * conversion_factor; } From c13d348d1e8a467b9d16fc83214ef574752092e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 19:56:50 +0200 Subject: [PATCH 0706/1722] Fix test `00900_long_parquet_load` --- tests/queries/0_stateless/00900_long_parquet_load.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 1bafb033f56..3a7022ac0cf 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-debug +# Tags: long, no-fasttest, no-debug, no-asan, no-msan, no-tsan # # Load all possible .parquet files found in submodules. From a25accdae3cf420beaeb8ca25a9ac32070c397b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 20:20:09 +0200 Subject: [PATCH 0707/1722] Fix a test --- programs/client/Client.cpp | 3 +++ src/Client/ClientBase.cpp | 3 --- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 1d99d223ee9..631914dee5c 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1164,6 +1164,9 @@ void Client::processOptions(const OptionsDescription & options_description, /// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.) client_context = global_context; initClientContext(); + + /// Allow to pass-through unknown settings to the server. + client_context->getAccessControl().allowAllSettings(); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a00a9499237..473db8e9678 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -689,9 +689,6 @@ void ClientBase::initClientContext() client_context->setQueryKindInitial(); client_context->setQueryKind(query_kind); client_context->setQueryParameters(query_parameters); - - /// Allow to pass-through unknown settings to the server. - client_context->getAccessControl().allowAllSettings(); } bool ClientBase::isRegularFile(int fd) From 776eb7ca75b50ee11da5b26d9dd0bb7544916151 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 20:38:24 +0200 Subject: [PATCH 0708/1722] Fix unit test --- src/Core/MySQL/MySQLGtid.cpp | 2 -- src/Core/MySQL/tests/gtest_MySQLGtid.cpp | 17 ++++++++--------- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/Core/MySQL/MySQLGtid.cpp b/src/Core/MySQL/MySQLGtid.cpp index 7916f882979..28b583a0cfe 100644 --- a/src/Core/MySQL/MySQLGtid.cpp +++ b/src/Core/MySQL/MySQLGtid.cpp @@ -24,9 +24,7 @@ void GTIDSet::tryMerge(size_t i) void GTIDSets::parse(String gtid_format) { if (gtid_format.empty()) - { return; - } std::vector gtid_sets; boost::split(gtid_sets, gtid_format, [](char c) { return c == ','; }); diff --git a/src/Core/MySQL/tests/gtest_MySQLGtid.cpp b/src/Core/MySQL/tests/gtest_MySQLGtid.cpp index e31a87aaa39..e5a2fe44e5c 100644 --- a/src/Core/MySQL/tests/gtest_MySQLGtid.cpp +++ b/src/Core/MySQL/tests/gtest_MySQLGtid.cpp @@ -10,20 +10,19 @@ GTEST_TEST(GTIDSetsContains, Tests) contained1, contained2, contained3, contained4, contained5, not_contained1, not_contained2, not_contained3, not_contained4, not_contained5, not_contained6; - gtid_set.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); - contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + gtid_set.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60"); + contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60"); contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:2-3:11:47-49"); contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:11"); - contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:47-49:60"); - contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:60"); + contained4.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:2-16:47-49:60"); + contained5.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:60"); - not_contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-50, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + not_contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-50, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60"); not_contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:0-3:11:47-49"); not_contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:99"); - not_contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:46-49:60"); - not_contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:99"); - not_contained6.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60, 00000000-0000-0000-0000-000000000000"); - + not_contained4.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:2-16:46-49:60"); + not_contained5.parse("FBC30C64-F8C9-4DDF-8CDD-066208EB433B:99"); + not_contained6.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, FBC30C64-F8C9-4DDF-8CDD-066208EB433B:1-19:47-49:60, 00000000-0000-0000-0000-000000000000"); ASSERT_TRUE(gtid_set.contains(contained1)); ASSERT_TRUE(gtid_set.contains(contained2)); From c65c3768baad44a7f8a58913385acb6cfd369735 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 21:15:18 +0200 Subject: [PATCH 0709/1722] Fix .NET --- src/IO/ReadHelpers.h | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 753560620e3..580aa51b238 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -258,6 +258,14 @@ inline void readBoolText(bool & x, ReadBuffer & buf) char tmp = '0'; readChar(tmp, buf); x = tmp != '0'; + + if (!buf.eof() && isAlphaASCII(*buf.position())) + { + if (tmp == 't') + assertString("rue", buf); + else if (tmp == 'T') + assertString("RUE", buf); + } } template From b42159479145664628cf876cda61012e2f56b21d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 21:15:27 +0200 Subject: [PATCH 0710/1722] Fix backups --- src/Access/AccessBackup.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index 90effdab70f..d9ee89b45ce 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -93,7 +93,7 @@ namespace break; } - UUID id = parse(line); + UUID id = parse(line.substr(0, line.find('\t'))); line.clear(); String queries; From ae5982f92affaaa664899c31b3507d893b8773aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 21:42:16 +0200 Subject: [PATCH 0711/1722] Fix crap --- src/Common/parseRemoteDescription.cpp | 15 ++++++++----- src/Storages/StorageExternalDistributed.cpp | 24 ++++++++++----------- 2 files changed, 22 insertions(+), 17 deletions(-) diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index df3820b11f9..6a53098362d 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -79,11 +79,16 @@ std::vector parseRemoteDescription( /// Look for the corresponding closing bracket for (m = i + 1; m < r; ++m) { - if (description[m] == '{') ++cnt; - if (description[m] == '}') --cnt; - if (description[m] == '.' && description[m-1] == '.') last_dot = m; - if (description[m] == separator) have_splitter = true; - if (cnt == 0) break; + if (description[m] == '{') + ++cnt; + if (description[m] == '}') + --cnt; + if (description[m] == '.' && description[m-1] == '.') + last_dot = m; + if (description[m] == separator) + have_splitter = true; + if (cnt == 0) + break; } if (cnt != 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}': incorrect brace sequence in first argument", func_name); diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 951c87807bb..4277387bc5d 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -111,15 +111,17 @@ void registerStorageExternalDistributed(StorageFactory & factory) std::unordered_set shards; ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end()); + String addresses_expr = checkAndGetLiteralArgument(engine_args[1], "addresses"); + Strings shards_addresses = get_addresses(addresses_expr); auto engine_name = checkAndGetLiteralArgument(engine_args[0], "engine_name"); if (engine_name == "URL") { - auto configuration = StorageURL::getConfiguration(inner_engine_args, context); - auto shards_addresses = get_addresses(configuration.addresses_expr); auto format_settings = StorageURL::getFormatSettingsFromArgs(args); for (const auto & shard_address : shards_addresses) { + inner_engine_args.at(0) = std::make_shared(shard_address); + auto configuration = StorageURL::getConfiguration(inner_engine_args, context); auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses); if (uri_options.size() > 1) { @@ -140,13 +142,12 @@ void registerStorageExternalDistributed(StorageFactory & factory) else if (engine_name == "MySQL") { MySQLSettings mysql_settings; - auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings); - auto shards_addresses = get_addresses(configuration.addresses_expr); for (const auto & shard_address : shards_addresses) { - auto current_configuration{configuration}; - current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306); - auto pool = createMySQLPoolWithFailover(current_configuration, mysql_settings); + inner_engine_args.at(0) = std::make_shared(shard_address); + auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings); + configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306); + auto pool = createMySQLPoolWithFailover(configuration, mysql_settings); shards.insert(std::make_shared( args.table_id, std::move(pool), configuration.database, configuration.table, /* replace_query = */ false, /* on_duplicate_clause = */ "", @@ -157,14 +158,13 @@ void registerStorageExternalDistributed(StorageFactory & factory) #if USE_LIBPQXX else if (engine_name == "PostgreSQL") { - auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context); - auto shards_addresses = get_addresses(configuration.addresses_expr); for (const auto & shard_address : shards_addresses) { - auto current_configuration{configuration}; - current_configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432); + inner_engine_args.at(0) = std::make_shared(shard_address); + auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context); + configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432); auto pool = std::make_shared( - current_configuration, + configuration, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, settings.postgresql_connection_pool_retries, From f4173546a93ab72986d1239a812ac0bc7fda11cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Aug 2024 21:47:01 +0200 Subject: [PATCH 0712/1722] Remove obsolete test --- src/IO/tests/gtest_s3_uri.cpp | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 0ec28f80072..7216c8077e3 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -206,11 +206,6 @@ TEST(S3UriTest, validPatterns) } } -TEST_P(S3UriTest, invalidPatterns) -{ - ASSERT_ANY_THROW(S3::URI new_uri(GetParam())); -} - TEST(S3UriTest, versionIdChecks) { for (const auto& test_case : TestCases) @@ -223,19 +218,4 @@ TEST(S3UriTest, versionIdChecks) } } -INSTANTIATE_TEST_SUITE_P( - S3, - S3UriTest, - testing::Values( - "https:///", - "https://.s3.amazonaws.com/key", - "https://s3.amazonaws.com/key", - "https://jokserfn.s3amazonaws.com/key", - "https://s3.amazonaws.com//", - "https://amazonaws.com/", - "https://amazonaws.com//", - "https://amazonaws.com//key")); - -} - #endif From b5afddb1af0a9aeb4738cf3fb7b7242361469028 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 9 Aug 2024 22:56:25 +0200 Subject: [PATCH 0713/1722] Update optimize_functions_to_subcolumns.xml --- tests/performance/optimize_functions_to_subcolumns.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/optimize_functions_to_subcolumns.xml b/tests/performance/optimize_functions_to_subcolumns.xml index a246aae7950..146af1605c4 100644 --- a/tests/performance/optimize_functions_to_subcolumns.xml +++ b/tests/performance/optimize_functions_to_subcolumns.xml @@ -1,6 +1,5 @@ - 1 4 From 61befa33a40d4ec78ba124107e700d14f9be9599 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 9 Aug 2024 21:10:05 +0000 Subject: [PATCH 0714/1722] Fix crash on parquet column type mismatch --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 108 ++++++++++++------ .../02841_parquet_filter_pushdown.reference | 2 + .../02841_parquet_filter_pushdown.sql | 6 + 3 files changed, 83 insertions(+), 33 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index bc5e8292192..1268f1df5f6 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -54,7 +55,7 @@ namespace ErrorCodes } \ } while (false) -/// Decode min/max value from column chunk statistics. +/// Decode min/max value from column chunk statistics. Returns Null if missing or unsupported. /// /// There are two questionable decisions in this implementation: /// * We parse the value from the encoded byte string instead of casting the parquet::Statistics @@ -62,7 +63,7 @@ namespace ErrorCodes /// * We dispatch based on the parquet logical+converted+physical type instead of the ClickHouse type. /// The idea is that this is similar to what we'll have to do when reimplementing Parquet parsing in /// ClickHouse instead of using Arrow (for speed). So, this is an exercise in parsing Parquet manually. -static std::optional decodePlainParquetValueSlow(const std::string & data, parquet::Type::type physical_type, const parquet::ColumnDescriptor & descr) +static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type::type physical_type, const parquet::ColumnDescriptor & descr) { using namespace parquet; @@ -118,8 +119,6 @@ static std::optional decodePlainParquetValueSlow(const std::string & data if (data.size() != size || size < 1 || size > 32) throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected decimal size: {} (actual {})", size, data.size()); - /// For simplicity, widen all decimals to 256-bit. It should compare correctly with values - /// of different bitness. Int256 val = 0; memcpy(&val, data.data(), size); if (big_endian) @@ -128,7 +127,20 @@ static std::optional decodePlainParquetValueSlow(const std::string & data if (size < 32 && (val >> (size * 8 - 1)) != 0) val |= ~((Int256(1) << (size * 8)) - 1); - return Field(DecimalField(Decimal256(val), static_cast(scale))); + auto narrow = [&]() -> Field { + using T = typename D::NativeType; + T x = 0; + memcpy(&x, &val, sizeof(T)); + return Field(DecimalField(D(x), static_cast(scale))); + }; + if (size <= 4) + return narrow.template operator()(); + else if (size <= 8) + return narrow.template operator()(); + else if (size <= 16) + return narrow.template operator()(); + else + return narrow.template operator()(); } while (false); @@ -213,14 +225,15 @@ static std::optional decodePlainParquetValueSlow(const std::string & data /// TODO: Remove this workaround either when we implement our own Parquet decoder that /// doesn't have this bug, or if it's fixed in Arrow. if (data.empty()) - return std::nullopt; + return Field(); return Field(data); } - /// This one's deprecated in Parquet. + /// This type is deprecated in Parquet. + /// TODO: But turns out it's still used in practice, we should support it. if (physical_type == Type::type::INT96) - throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Parquet INT96 type is deprecated and not supported"); + return Field(); /// Integers. @@ -283,15 +296,12 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa continue; auto stats = it->second; - auto default_value = [&]() -> Field - { - DataTypePtr type = header.getByPosition(idx).type; - if (type->lowCardinality()) - type = assert_cast(*type).getDictionaryType(); - if (type->isNullable()) - type = assert_cast(*type).getNestedType(); - return type->getDefault(); - }; + DataTypePtr type = header.getByPosition(idx).type; + if (type->lowCardinality()) + type = assert_cast(*type).getDictionaryType(); + if (type->isNullable()) + type = assert_cast(*type).getNestedType(); + Field default_value = type->getDefault(); /// Only primitive fields are supported, not arrays, maps, tuples, or Nested. /// Arrays, maps, and Nested can't be meaningfully supported because Parquet only has min/max @@ -299,8 +309,8 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa /// Same limitation for tuples, but maybe it would make sense to have some kind of tuple /// expansion in KeyCondition to accept ranges per element instead of whole tuple. - std::optional min; - std::optional max; + Field min; + Field max; if (stats->HasMinMax()) { try @@ -315,6 +325,39 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa } } + /// If the data type in parquet file substantially differs from the requested data type, + /// it's sometimes correct to just typecast the min/max values. + /// Other times it's incorrect, e.g.: + /// INSERT INTO FUNCTION file('t.parquet', Parquet, 'x String') VALUES ('1'), ('100'), ('2'); + /// SELECT * FROM file('t.parquet', Parquet, 'x Int64') WHERE x >= 3; + /// If we just typecast min/max from string to integer, this query will incorrectly return empty result. + /// Allow conversion in some simple cases and ignore the min/max values otherwise. + auto min_type = min.getType(); + auto max_type = max.getType(); + min = convertFieldToType(min, *type); + max = convertFieldToType(max, *type); + auto ok_cast = [&](Field::Types::Which from, Field::Types::Which to) -> bool + { + if (from == to) + return true; + /// Decimal -> wider decimal. + if (Field::isDecimal(from) || Field::isDecimal(to)) + return Field::isDecimal(from) && Field::isDecimal(to) && to >= from; + /// Integer -> IP. + if (to == Field::Types::IPv4 || to == Field::Types::IPv6) + return from == Field::Types::UInt64 || from == Field::Types::Int64; + /// Disable index for everything else, especially string <-> number. + return false; + }; + if (!(ok_cast(min_type, min.getType()) && ok_cast(max_type, max.getType())) && + !(min == max) && + !(min_type == Field::Types::Int64 && min.getType() == Field::Types::UInt64 && min.get() >= 0) && + !(max_type == Field::Types::UInt64 && max.getType() == Field::Types::Int64 && max.get() <= UInt64(INT64_MAX))) + { + min = Field(); + max = Field(); + } + /// In Range, NULL is represented as positive or negative infinity (represented by a special /// kind of Field, different from floating-point infinities). @@ -328,7 +371,7 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa { /// Single-point range containing either the default value of one of the infinities. if (null_as_default) - hyperrectangle[idx].right = hyperrectangle[idx].left = default_value(); + hyperrectangle[idx].right = hyperrectangle[idx].left = default_value; else hyperrectangle[idx].right = hyperrectangle[idx].left; continue; @@ -339,32 +382,31 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa if (null_as_default) { /// Make sure the range contains the default value. - Field def = default_value(); - if (min.has_value() && applyVisitor(FieldVisitorAccurateLess(), def, *min)) - min = def; - if (max.has_value() && applyVisitor(FieldVisitorAccurateLess(), *max, def)) - max = def; + if (!min.isNull() && applyVisitor(FieldVisitorAccurateLess(), default_value, min)) + min = default_value; + if (!max.isNull() && applyVisitor(FieldVisitorAccurateLess(), max, default_value)) + max = default_value; } else { /// Make sure the range reaches infinity on at least one side. - if (min.has_value() && max.has_value()) - min.reset(); + if (!min.isNull() && !max.isNull()) + min = Field(); } } else { /// If the column doesn't have nulls, exclude both infinities. - if (!min.has_value()) + if (min.isNull()) hyperrectangle[idx].left_included = false; - if (!max.has_value()) + if (max.isNull()) hyperrectangle[idx].right_included = false; } - if (min.has_value()) - hyperrectangle[idx].left = std::move(min.value()); - if (max.has_value()) - hyperrectangle[idx].right = std::move(max.value()); + if (!min.isNull()) + hyperrectangle[idx].left = std::move(min); + if (!max.isNull()) + hyperrectangle[idx].right = std::move(max); } return hyperrectangle; diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown.reference b/tests/queries/0_stateless/02841_parquet_filter_pushdown.reference index 4adf418bcc7..8003b9cb626 100644 --- a/tests/queries/0_stateless/02841_parquet_filter_pushdown.reference +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown.reference @@ -71,3 +71,5 @@ d256 Nullable(Decimal(76, 40)) 500 244750 500 244750 500 244750 +42 +100 diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql b/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql index 950485d53f0..52caee50b32 100644 --- a/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown.sql @@ -131,3 +131,9 @@ select count(), sum(number) from file('02841.parquet', Parquet, 'number UInt64, select count(), sum(number) from file('02841.parquet') where indexHint(string_or_null == ''); -- quirk with infinities select count(), sum(number) from file('02841.parquet', Parquet, 'number UInt64, string_or_null String') where indexHint(string_or_null == ''); select count(), sum(number) from file('02841.parquet', Parquet, 'number UInt64, nEgAtIvE_oR_nUlL Int64') where indexHint(nEgAtIvE_oR_nUlL > -50) settings input_format_parquet_case_insensitive_column_matching = 1; + +-- Bad type conversions. +insert into function file('02841.parquet') select 42 as x; +select * from file('02841.parquet', Parquet, 'x Nullable(String)') where x not in (1); +insert into function file('t.parquet', Parquet, 'x String') values ('1'), ('100'), ('2'); +select * from file('t.parquet', Parquet, 'x Int64') where x >= 3; From c61eef4a7659e4856cc3266d8d7dd28b4e095d2b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 9 Aug 2024 21:17:49 +0000 Subject: [PATCH 0715/1722] Reimplement Dynamic type --- docs/en/sql-reference/data-types/dynamic.md | 123 +- src/Columns/ColumnDynamic.cpp | 881 ++++-- src/Columns/ColumnDynamic.h | 164 +- src/Columns/ColumnVariant.cpp | 29 +- src/Columns/ColumnVariant.h | 11 +- src/Columns/tests/gtest_column_dynamic.cpp | 428 ++- src/DataTypes/DataTypeDynamic.cpp | 79 +- src/DataTypes/DataTypeFactory.cpp | 6 + src/DataTypes/DataTypeFactory.h | 1 + src/DataTypes/DataTypeTuple.cpp | 13 +- src/DataTypes/DataTypesBinaryEncoding.cpp | 2 +- .../Serializations/SerializationDynamic.cpp | 361 ++- .../Serializations/SerializationDynamic.h | 8 +- .../SerializationDynamicElement.cpp | 127 +- .../SerializationDynamicElement.h | 8 +- .../SerializationVariantElement.cpp | 41 +- .../SerializationVariantElement.h | 6 +- src/Formats/JSONExtractTree.cpp | 31 +- src/Functions/FunctionsConversion.cpp | 326 +-- src/Functions/dynamicType.cpp | 91 +- tests/queries/0_stateless/00000_test.sql | 43 + ...03033_dynamic_text_serialization.reference | 26 +- .../03033_dynamic_text_serialization.sql | 4 +- .../03034_dynamic_conversions.reference | 18 +- .../0_stateless/03034_dynamic_conversions.sql | 6 +- .../03035_dynamic_sorting.reference | 715 +++-- .../0_stateless/03035_dynamic_sorting.sql | 67 +- ...ed_subcolumns_compact_merge_tree.reference | 20 + ...d_shared_subcolumns_compact_merge_tree.sql | 43 + ...ic_read_shared_subcolumns_memory.reference | 20 + ..._dynamic_read_shared_subcolumns_memory.sql | 43 + ..._read_shared_subcolumns_small.reference.j2 | 2460 +++++++++++++++++ ...ynamic_read_shared_subcolumns_small.sql.j2 | 46 + ...hared_subcolumns_wide_merge_tree.reference | 20 + ...read_shared_subcolumns_wide_merge_tree.sql | 43 + ..._1_horizontal_compact_merge_tree.reference | 94 +- ...merges_1_horizontal_compact_merge_tree.sql | 29 +- ...s_1_horizontal_compact_wide_tree.reference | 94 +- ..._merges_1_horizontal_compact_wide_tree.sql | 28 +- ...es_1_vertical_compact_merge_tree.reference | 94 +- ...c_merges_1_vertical_compact_merge_tree.sql | 31 +- ...erges_1_vertical_wide_merge_tree.reference | 94 +- ...amic_merges_1_vertical_wide_merge_tree.sql | 28 +- .../03037_dynamic_merges_small.reference.j2 | 376 ++- .../03037_dynamic_merges_small.sql.j2 | 28 +- ...ynamic_merges_compact_horizontal.reference | 84 +- ...sted_dynamic_merges_compact_horizontal.sql | 33 +- ..._dynamic_merges_compact_vertical.reference | 84 +- ...nested_dynamic_merges_compact_vertical.sql | 33 +- ...8_nested_dynamic_merges_small.reference.j2 | 340 ++- .../03038_nested_dynamic_merges_small.sql.j2 | 36 +- ...d_dynamic_merges_wide_horizontal.reference | 84 +- ..._nested_dynamic_merges_wide_horizontal.sql | 33 +- ...ted_dynamic_merges_wide_vertical.reference | 84 +- ...38_nested_dynamic_merges_wide_vertical.sql | 33 +- ...type_alters_1_compact_merge_tree.reference | 123 +- ...namic_type_alters_1_compact_merge_tree.sql | 20 +- ...040_dynamic_type_alters_1_memory.reference | 94 +- ...ic_type_alters_1_wide_merge_tree.reference | 157 +- .../03041_dynamic_type_check_table.sh | 2 +- .../03150_dynamic_type_mv_insert.reference | 50 +- .../03150_dynamic_type_mv_insert.sql | 7 + ...151_dynamic_type_scale_max_types.reference | 48 +- .../03151_dynamic_type_scale_max_types.sql | 8 +- .../03152_dynamic_type_simple.reference | 2 +- .../0_stateless/03152_dynamic_type_simple.sql | 8 + .../0_stateless/03153_dynamic_type_empty.sql | 2 + .../03159_dynamic_type_all_types.sql | 4 +- .../03172_dynamic_binary_serialization.sh | 4 +- ...3200_memory_engine_alter_dynamic.reference | 20 +- .../03200_memory_engine_alter_dynamic.sql | 4 +- 71 files changed, 6725 insertions(+), 1878 deletions(-) create mode 100644 tests/queries/0_stateless/00000_test.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference create mode 100644 tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index 8be81471377..f9befd166fe 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -14,7 +14,7 @@ To declare a column of `Dynamic` type, use the following syntax: Dynamic(max_types=N) ``` -Where `N` is an optional parameter between `1` and `255` indicating how many different data types can be stored inside a column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all new types will be converted to type `String`. Default value of `max_types` is `32`. +Where `N` is an optional parameter between `0` and `254` indicating how many different data types can be stored as separate subcolumns inside a column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all values with new types will be stored together in a special shared data structure in binary form. Default value of `max_types` is `32`. :::note The Dynamic data type is an experimental feature. To use it, set `allow_experimental_dynamic_type = 1`. @@ -224,41 +224,43 @@ SELECT d::Dynamic(max_types=5) as d2, dynamicType(d2) FROM test; └───────┴────────────────┘ ``` -If `K < N`, then the values with the rarest types are converted to `String`: +If `K < N`, then the values with the rarest types will be inserted into a single special subcolumn, but still will be accessible: ```text CREATE TABLE test (d Dynamic(max_types=4)) ENGINE = Memory; INSERT INTO test VALUES (NULL), (42), (43), ('42.42'), (true), ([1, 2, 3]); -SELECT d, dynamicType(d), d::Dynamic(max_types=2) as d2, dynamicType(d2) FROM test; +SELECT d, dynamicType(d), d::Dynamic(max_types=2) as d2, dynamicType(d2), isDynamicElementInSharedData(d2) FROM test; ``` ```text -┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┠-│ á´ºáµá´¸á´¸ │ None │ á´ºáµá´¸á´¸ │ None │ -│ 42 │ Int64 │ 42 │ Int64 │ -│ 43 │ Int64 │ 43 │ Int64 │ -│ 42.42 │ String │ 42.42 │ String │ -│ true │ Bool │ true │ String │ -│ [1,2,3] │ Array(Int64) │ [1,2,3] │ String │ -└─────────┴────────────────┴─────────┴─────────────────┘ +┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┬─isDynamicElementInSharedData(d2)─┠+│ á´ºáµá´¸á´¸ │ None │ á´ºáµá´¸á´¸ │ None │ false │ +│ 42 │ Int64 │ 42 │ Int64 │ false │ +│ 43 │ Int64 │ 43 │ Int64 │ false │ +│ 42.42 │ String │ 42.42 │ String │ false │ +│ true │ Bool │ true │ Bool │ true │ +│ [1,2,3] │ Array(Int64) │ [1,2,3] │ Array(Int64) │ true │ +└─────────┴────────────────┴─────────┴─────────────────┴──────────────────────────────────┘ ``` -If `K=1`, all types are converted to `String`: +Functions `isDynamicElementInSharedData` returns `true` for rows that are stored in a special shared data structure inside `Dynamic` and as we can see, resulting column contains only 2 types that are not stored in shared data structure. + +If `K=0`, all types will be inserted into single special subcolumn: ```text CREATE TABLE test (d Dynamic(max_types=4)) ENGINE = Memory; INSERT INTO test VALUES (NULL), (42), (43), ('42.42'), (true), ([1, 2, 3]); -SELECT d, dynamicType(d), d::Dynamic(max_types=1) as d2, dynamicType(d2) FROM test; +SELECT d, dynamicType(d), d::Dynamic(max_types=0) as d2, dynamicType(d2), isDynamicElementInSharedData(d2) FROM test; ``` ```text -┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┠-│ á´ºáµá´¸á´¸ │ None │ á´ºáµá´¸á´¸ │ None │ -│ 42 │ Int64 │ 42 │ String │ -│ 43 │ Int64 │ 43 │ String │ -│ 42.42 │ String │ 42.42 │ String │ -│ true │ Bool │ true │ String │ -│ [1,2,3] │ Array(Int64) │ [1,2,3] │ String │ -└─────────┴────────────────┴─────────┴─────────────────┘ +┌─d───────┬─dynamicType(d)─┬─d2──────┬─dynamicType(d2)─┬─isDynamicElementInSharedData(d2)─┠+│ á´ºáµá´¸á´¸ │ None │ á´ºáµá´¸á´¸ │ None │ false │ +│ 42 │ Int64 │ 42 │ Int64 │ true │ +│ 43 │ Int64 │ 43 │ Int64 │ true │ +│ 42.42 │ String │ 42.42 │ String │ true │ +│ true │ Bool │ true │ Bool │ true │ +│ [1,2,3] │ Array(Int64) │ [1,2,3] │ Array(Int64) │ true │ +└─────────┴────────────────┴─────────┴─────────────────┴──────────────────────────────────┘ ``` ## Reading Dynamic type from the data @@ -411,17 +413,17 @@ SELECT d, dynamicType(d) FROM test ORDER by d; ## Reaching the limit in number of different data types stored inside Dynamic -`Dynamic` data type can store only limited number of different data types inside. By default, this limit is 32, but you can change it in type declaration using syntax `Dynamic(max_types=N)` where N is between 1 and 255 (due to implementation details, it's impossible to have more than 255 different data types inside Dynamic). -When the limit is reached, all new data types inserted to `Dynamic` column will be casted to `String` and stored as `String` values. +`Dynamic` data type can store only limited number of different data types as separate subcolumns. By default, this limit is 32, but you can change it in type declaration using syntax `Dynamic(max_types=N)` where N is between 0 and 254 (due to implementation details, it's impossible to have more than 254 different data types that can be stored as separate subcolumns inside Dynamic). +When the limit is reached, all new data types inserted to `Dynamic` column will be inserted into a single shared data structure that stores values with different data types in binary form. Let's see what happens when the limit is reached in different scenarios. ### Reaching the limit during data parsing -During parsing of `Dynamic` values from the data, when the limit is reached for current block of data, all new values will be inserted as `String` values: +During parsing of `Dynamic` values from the data, when the limit is reached for current block of data, all new values will be inserted into shared data structure: ```sql -SELECT d, dynamicType(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', ' +SELECT d, dynamicType(d), isDynamicElementInSharedData(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', ' {"d" : 42} {"d" : [1, 2, 3]} {"d" : "Hello, World!"} @@ -432,22 +434,22 @@ SELECT d, dynamicType(d) FROM format(JSONEachRow, 'd Dynamic(max_types=3)', ' ``` ```text -┌─d──────────────────────────┬─dynamicType(d)─┠-│ 42 │ Int64 │ -│ [1,2,3] │ Array(Int64) │ -│ Hello, World! │ String │ -│ 2020-01-01 │ String │ -│ ["str1", "str2", "str3"] │ String │ -│ {"a" : 1, "b" : [1, 2, 3]} │ String │ -└────────────────────────────┴────────────────┘ +┌─d──────────────────────┬─dynamicType(d)─────────────────┬─isDynamicElementInSharedData(d)─┠+│ 42 │ Int64 │ false │ +│ [1,2,3] │ Array(Int64) │ false │ +│ Hello, World! │ String │ false │ +│ 2020-01-01 │ Date │ true │ +│ ['str1','str2','str3'] │ Array(String) │ true │ +│ (1,[1,2,3]) │ Tuple(a Int64, b Array(Int64)) │ true │ +└────────────────────────┴────────────────────────────────┴─────────────────────────────────┘ ``` -As we can see, after inserting 3 different data types `Int64`, `Array(Int64)` and `String` all new types were converted to `String`. +As we can see, after inserting 3 different data types `Int64`, `Array(Int64)` and `String` all new types were inserted into special shared data structure. ### During merges of data parts in MergeTree table engines -During merge of several data parts in MergeTree table the `Dynamic` column in the resulting data part can reach the limit of different data types inside and won't be able to store all types from source parts. -In this case ClickHouse chooses what types will remain after merge and what types will be casted to `String`. In most cases ClickHouse tries to keep the most frequent types and cast the rarest types to `String`, but it depends on the implementation. +During merge of several data parts in MergeTree table the `Dynamic` column in the resulting data part can reach the limit of different data types that can be stored in separate subcolumns inside and won't be able to store all types as subcolumns from source parts. +In this case ClickHouse chooses what types will remain as separate subcolumns after merge and what types will be inserted into shared data structure. In most cases ClickHouse tries to keep the most frequent types and store the rarest types in shared data structure, but it depends on the implementation. Let's see an example of such merge. First, let's create a table with `Dynamic` column, set the limit of different data types to `3` and insert values with `5` different types: @@ -463,17 +465,17 @@ INSERT INTO test SELECT number, 'str_' || toString(number) FROM numbers(1); Each insert will create a separate data pert with `Dynamic` column containing single type: ```sql -SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; +SELECT count(), dynamicType(d), isDynamicElementInSharedData(d), _part FROM test GROUP BY _part, dynamicType(d), isDynamicElementInSharedData(d) ORDER BY _part, count(); ``` ```text -┌─count()─┬─dynamicType(d)──────┬─_part─────┠-│ 5 │ UInt64 │ all_1_1_0 │ -│ 4 │ Array(UInt64) │ all_2_2_0 │ -│ 3 │ Date │ all_3_3_0 │ -│ 2 │ Map(UInt64, UInt64) │ all_4_4_0 │ -│ 1 │ String │ all_5_5_0 │ -└─────────┴─────────────────────┴───────────┘ +┌─count()─┬─dynamicType(d)──────┬─isDynamicElementInSharedData(d)─┬─_part─────┠+│ 5 │ UInt64 │ false │ all_1_1_0 │ +│ 4 │ Array(UInt64) │ false │ all_2_2_0 │ +│ 3 │ Date │ false │ all_3_3_0 │ +│ 2 │ Map(UInt64, UInt64) │ false │ all_4_4_0 │ +│ 1 │ String │ false │ all_5_5_0 │ +└─────────┴─────────────────────┴─────────────────────────────────┴───────────┘ ``` Now, let's merge all parts into one and see what will happen: @@ -481,18 +483,20 @@ Now, let's merge all parts into one and see what will happen: ```sql SYSTEM START MERGES test; OPTIMIZE TABLE test FINAL; -SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) ORDER BY _part; +SELECT count(), dynamicType(d), isDynamicElementInSharedData(d), _part FROM test GROUP BY _part, dynamicType(d), isDynamicElementInSharedData(d) ORDER BY _part, count() desc; ``` ```text -┌─count()─┬─dynamicType(d)─┬─_part─────┠-│ 5 │ UInt64 │ all_1_5_2 │ -│ 6 │ String │ all_1_5_2 │ -│ 4 │ Array(UInt64) │ all_1_5_2 │ -└─────────┴────────────────┴───────────┘ +┌─count()─┬─dynamicType(d)──────┬─isDynamicElementInSharedData(d)─┬─_part─────┠+│ 5 │ UInt64 │ false │ all_1_5_2 │ +│ 4 │ Array(UInt64) │ false │ all_1_5_2 │ +│ 3 │ Date │ false │ all_1_5_2 │ +│ 2 │ Map(UInt64, UInt64) │ true │ all_1_5_2 │ +│ 1 │ String │ true │ all_1_5_2 │ +└─────────┴─────────────────────┴─────────────────────────────────┴───────────┘ ``` -As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` and casted all other types to `String`. +As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` as subcolumns and inserted all other types into shared data. ## JSONExtract functions with Dynamic @@ -509,22 +513,23 @@ SELECT JSONExtract('{"a" : [1, 2, 3]}', 'a', 'Dynamic') AS dynamic, dynamicType( ``` ```sql -SELECT JSONExtract('{"obj" : {"a" : 42, "b" : "Hello", "c" : [1,2,3]}}', 'obj', 'Map(String, Variant(UInt32, String, Array(UInt32)))') AS map_of_dynamics, mapApply((k, v) -> (k, variantType(v)), map_of_dynamics) AS map_of_dynamic_types``` +SELECT JSONExtract('{"obj" : {"a" : 42, "b" : "Hello", "c" : [1,2,3]}}', 'obj', 'Map(String, Dynamic)') AS map_of_dynamics, mapApply((k, v) -> (k, dynamicType(v)), map_of_dynamics) AS map_of_dynamic_types +``` ```text -┌─map_of_dynamics──────────────────┬─map_of_dynamic_types────────────────────────────┠-│ {'a':42,'b':'Hello','c':[1,2,3]} │ {'a':'UInt32','b':'String','c':'Array(UInt32)'} │ -└──────────────────────────────────┴─────────────────────────────────────────────────┘ +┌─map_of_dynamics──────────────────┬─map_of_dynamic_types────────────────────────────────────┠+│ {'a':42,'b':'Hello','c':[1,2,3]} │ {'a':'Int64','b':'String','c':'Array(Nullable(Int64))'} │ +└──────────────────────────────────┴─────────────────────────────────────────────────────────┘ ``` ```sql -SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Variant(UInt32, String, Array(UInt32))') AS dynamics, arrayMap(x -> (x.1, variantType(x.2)), dynamics) AS dynamic_types``` +SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Dynamic') AS dynamics, arrayMap(x -> (x.1, dynamicType(x.2)), dynamics) AS dynamic_types``` ``` ```text -┌─dynamics───────────────────────────────┬─dynamic_types─────────────────────────────────────────┠-│ [('a',42),('b','Hello'),('c',[1,2,3])] │ [('a','UInt32'),('b','String'),('c','Array(UInt32)')] │ -└────────────────────────────────────────┴───────────────────────────────────────────────────────┘ +┌─dynamics───────────────────────────────┬─dynamic_types─────────────────────────────────────────────────┠+│ [('a',42),('b','Hello'),('c',[1,2,3])] │ [('a','Int64'),('b','String'),('c','Array(Nullable(Int64))')] │ +└────────────────────────────────────────┴───────────────────────────────────────────────────────────────┘ ``` ### Binary output format diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index a92d54dd675..454f7956f48 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1,16 +1,21 @@ #include #include +#include #include #include #include +#include #include #include #include #include #include #include -#include +#include +#include +#include +#include #include namespace DB @@ -22,31 +27,77 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; } - -ColumnDynamic::ColumnDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_dynamic_types_) +namespace { - /// Create empty Variant. - variant_info.variant_type = std::make_shared(DataTypes{}); - variant_info.variant_name = variant_info.variant_type->getName(); - variant_column = variant_info.variant_type->createColumn(); + +/// Static default format settings to avoid creating it every time. +const FormatSettings & getFormatSettings() +{ + static const FormatSettings settings; + return settings; +} + +} + +/// Shared variant will contain String values but we cannot use usual String type +/// because we can have regular variant with type String. +/// To solve it, we use String type with custom name for shared variant. +DataTypePtr ColumnDynamic::getSharedVariantDataType() +{ + return DataTypeFactory::instance().getCustom("String", std::make_unique(std::make_unique(getSharedVariantTypeName()))); +} + +ColumnDynamic::ColumnDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_dynamic_types_), global_max_dynamic_types(max_dynamic_types) +{ + /// Create Variant with shared variant. + setVariantType(std::make_shared(DataTypes{getSharedVariantDataType()})); } ColumnDynamic::ColumnDynamic( - MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_) + MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_) + : variant_column(std::move(variant_column_)) + , max_dynamic_types(max_dynamic_types_) + , global_max_dynamic_types(global_max_dynamic_types_) + , statistics(statistics_) +{ + createVariantInfo(variant_type_); +} + +ColumnDynamic::ColumnDynamic( + MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_) : variant_column(std::move(variant_column_)) , variant_info(variant_info_) , max_dynamic_types(max_dynamic_types_) + , global_max_dynamic_types(global_max_dynamic_types_) , statistics(statistics_) { } -ColumnDynamic::MutablePtr ColumnDynamic::create(MutableColumnPtr variant_column, const DataTypePtr & variant_type, size_t max_dynamic_types_, const Statistics & statistics_) +void ColumnDynamic::setVariantType(const DataTypePtr & variant_type) +{ + if (variant_column && !empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting specific variant type is allowed only for empty dynamic column"); + + variant_column = variant_type->createColumn(); + createVariantInfo(variant_type); +} + +void ColumnDynamic::setMaxDynamicPaths(size_t max_dynamic_type_) +{ + if (variant_column && !empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting specific max_dynamic_type parameter is allowed only for empty dynamic column"); + + max_dynamic_types = max_dynamic_type_; +} + +void ColumnDynamic::createVariantInfo(const DataTypePtr & variant_type) { - VariantInfo variant_info; variant_info.variant_type = variant_type; variant_info.variant_name = variant_type->getName(); const auto & variants = assert_cast(*variant_type).getVariants(); + variant_info.variant_names.clear(); variant_info.variant_names.reserve(variants.size()); + variant_info.variant_name_to_discriminator.clear(); variant_info.variant_name_to_discriminator.reserve(variants.size()); for (ColumnVariant::Discriminator discr = 0; discr != variants.size(); ++discr) { @@ -54,30 +105,26 @@ ColumnDynamic::MutablePtr ColumnDynamic::create(MutableColumnPtr variant_column, variant_info.variant_name_to_discriminator[variant_name] = discr; } - return create(std::move(variant_column), variant_info, max_dynamic_types_, statistics_); + if (!variant_info.variant_name_to_discriminator.contains(getSharedVariantTypeName())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Variant in Dynamic column doesn't contain shared variant"); } -bool ColumnDynamic::addNewVariant(const DB::DataTypePtr & new_variant) +bool ColumnDynamic::addNewVariant(const DataTypePtr & new_variant, const String & new_variant_name) { /// Check if we already have such variant. - if (variant_info.variant_name_to_discriminator.contains(new_variant->getName())) + if (variant_info.variant_name_to_discriminator.contains(new_variant_name)) return true; - /// Check if we reached maximum number of variants. - if (variant_info.variant_names.size() >= max_dynamic_types) + /// Check if we reached maximum number of variants (don't count shared variant). + if (variant_info.variant_names.size() - 1 == max_dynamic_types) { - /// ColumnDynamic can have max_dynamic_types number of variants only when it has String as a variant. - /// Otherwise we won't be able to cast new variants to Strings. - if (!variant_info.variant_name_to_discriminator.contains("String")) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Maximum number of variants reached, but no String variant exists"); + /// Dynamic column should always have shared variant. + if (!variant_info.variant_name_to_discriminator.contains(getSharedVariantTypeName())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Maximum number of variants reached, but no shared variant exists"); return false; } - /// If we have (max_dynamic_types - 1) number of variants and don't have String variant, we can add only String variant. - if (variant_info.variant_names.size() == max_dynamic_types - 1 && new_variant->getName() != "String" && !variant_info.variant_name_to_discriminator.contains("String")) - return false; - const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); DataTypes all_variants = current_variants; all_variants.push_back(new_variant); @@ -86,21 +133,15 @@ bool ColumnDynamic::addNewVariant(const DB::DataTypePtr & new_variant) return true; } -void ColumnDynamic::addStringVariant() +void extendVariantColumn( + IColumn & variant_column, + const DataTypePtr & old_variant_type, + const DataTypePtr & new_variant_type, + std::unordered_map old_variant_name_to_discriminator) { - if (!addNewVariant(std::make_shared())) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add String variant to Dynamic column, it's a bug"); -} - -void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DB::DataTypePtr & new_variant_type) -{ - const DataTypes & current_variants = assert_cast(variant_info.variant_type.get())->getVariants(); + const DataTypes & current_variants = assert_cast(old_variant_type.get())->getVariants(); const DataTypes & new_variants = assert_cast(new_variant_type.get())->getVariants(); - Names new_variant_names; - new_variant_names.reserve(new_variants.size()); - std::unordered_map new_variant_name_to_discriminator; - new_variant_name_to_discriminator.reserve(new_variants.size()); std::vector> new_variant_columns_and_discriminators_to_add; new_variant_columns_and_discriminators_to_add.reserve(new_variants.size() - current_variants.size()); std::vector current_to_new_discriminators; @@ -108,26 +149,26 @@ void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DB::DataTypePt for (ColumnVariant::Discriminator discr = 0; discr != new_variants.size(); ++discr) { - const auto & name = new_variant_names.emplace_back(new_variants[discr]->getName()); - new_variant_name_to_discriminator[name] = discr; - - auto current_it = variant_info.variant_name_to_discriminator.find(name); - if (current_it == variant_info.variant_name_to_discriminator.end()) + auto current_it = old_variant_name_to_discriminator.find(new_variants[discr]->getName()); + if (current_it == old_variant_name_to_discriminator.end()) new_variant_columns_and_discriminators_to_add.emplace_back(new_variants[discr]->createColumn(), discr); else current_to_new_discriminators[current_it->second] = discr; } - variant_info.variant_type = new_variant_type; - variant_info.variant_name = new_variant_type->getName(); - variant_info.variant_names = new_variant_names; - variant_info.variant_name_to_discriminator = new_variant_name_to_discriminator; - assert_cast(*variant_column).extend(current_to_new_discriminators, std::move(new_variant_columns_and_discriminators_to_add)); + assert_cast(variant_column).extend(current_to_new_discriminators, std::move(new_variant_columns_and_discriminators_to_add)); +} + +void ColumnDynamic::updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type) +{ + extendVariantColumn(*variant_column, variant_info.variant_type, new_variant_type, variant_info.variant_name_to_discriminator); + createVariantInfo(new_variant_type); + /// Clear mappings cache because now with new Variant we will have new mappings. variant_mappings_cache.clear(); } -std::vector * ColumnDynamic::combineVariants(const DB::ColumnDynamic::VariantInfo & other_variant_info) +std::vector * ColumnDynamic::combineVariants(const ColumnDynamic::VariantInfo & other_variant_info) { /// Check if we already have global discriminators mapping for other Variant in cache. /// It's used to not calculate the same mapping each call of insertFrom with the same columns. @@ -153,21 +194,14 @@ std::vector * ColumnDynamic::combineVariants(const { const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); - /// We cannot combine Variants if total number of variants exceeds max_dynamic_types. - if (current_variants.size() + num_new_variants > max_dynamic_types) + /// We cannot combine Variants if total number of variants exceeds max_dynamic_types (don't count shared variant). + if (current_variants.size() + num_new_variants - 1 > max_dynamic_types) { /// Remember that we cannot combine our variant with this one, so we will not try to do it again. variants_with_failed_combination.insert(other_variant_info.variant_name); return nullptr; } - /// We cannot combine Variants if total number of variants reaches max_dynamic_types and we don't have String variant. - if (current_variants.size() + num_new_variants == max_dynamic_types && !variant_info.variant_name_to_discriminator.contains("String") && !other_variant_info.variant_name_to_discriminator.contains("String")) - { - variants_with_failed_combination.insert(other_variant_info.variant_name); - return nullptr; - } - DataTypes all_variants = current_variants; all_variants.insert(all_variants.end(), other_variants.begin(), other_variants.end()); auto new_variant_type = std::make_shared(all_variants); @@ -185,40 +219,93 @@ std::vector * ColumnDynamic::combineVariants(const return &it->second; } -void ColumnDynamic::insert(const DB::Field & x) +void ColumnDynamic::insert(const Field & x) { - /// Check if we can insert field without Variant extension. - if (variant_column->tryInsert(x)) + if (x.isNull()) + { + insertDefault(); return; + } + + auto & variant_col = getVariantColumn(); + auto shared_variant_discr = getSharedVariantDiscriminator(); + /// Check if we can insert field into existing variants and avoid Variant extension. + for (size_t i = 0; i != variant_col.getNumVariants(); ++i) + { + if (i != shared_variant_discr && variant_col.getVariantByGlobalDiscriminator(i).tryInsert(x)) + { + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(i)); + variant_col.getOffsets().push_back(variant_col.getVariantByGlobalDiscriminator(i).size() - 1); + return; + } + } /// If we cannot insert field into current variant column, extend it with new variant for this field from its type. - if (addNewVariant(applyVisitor(FieldToDataType(), x))) + auto field_data_type = applyVisitor(FieldToDataType(), x); + auto field_data_type_name = field_data_type->getName(); + if (addNewVariant(field_data_type, field_data_type_name)) { - /// Now we should be able to insert this field into extended variant column. - variant_column->insert(x); + /// Insert this field into newly added variant. + auto discr = variant_info.variant_name_to_discriminator[field_data_type_name]; + variant_col.getVariantByGlobalDiscriminator(discr).insert(x); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(discr)); + variant_col.getOffsets().push_back(variant_col.getVariantByGlobalDiscriminator(discr).size() - 1); } else { /// We reached maximum number of variants and couldn't add new variant. - /// This case should be really rare in real use cases. - /// We should always be able to add String variant and cast inserted value to String. - addStringVariant(); - variant_column->insert(toString(x)); + /// In this case we add the value of this new variant into special shared variant. + /// We store values in shared variant in binary form with binary encoded type. + auto & shared_variant = getSharedVariant(); + auto & chars = shared_variant.getChars(); + WriteBufferFromVector value_buf(chars, AppendModeTag()); + encodeDataType(field_data_type, value_buf); + getVariantSerialization(field_data_type, field_data_type_name)->serializeBinary(x, value_buf, getFormatSettings()); + value_buf.finalize(); + chars.push_back(0); + shared_variant.getOffsets().push_back(chars.size()); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(shared_variant_discr)); + variant_col.getOffsets().push_back(shared_variant.size() - 1); } } -bool ColumnDynamic::tryInsert(const DB::Field & x) +bool ColumnDynamic::tryInsert(const Field & x) { /// We can insert any value into Dynamic column. insert(x); return true; } +Field ColumnDynamic::operator[](size_t n) const +{ + Field res; + get(n, res); + return res; +} + +void ColumnDynamic::get(size_t n, Field & res) const +{ + const auto & variant_col = getVariantColumn(); + /// Check if value is not in shared variant. + if (variant_col.globalDiscriminatorAt(n) != getSharedVariantDiscriminator()) + { + variant_col.get(n, res); + return; + } + + /// We should deeserialize value from shared variant. + const auto & shared_variant = getSharedVariant(); + auto value_data = shared_variant.getDataAt(variant_col.offsetAt(n)); + ReadBufferFromMemory buf(value_data.data, value_data.size); + auto type = decodeDataType(buf); + getVariantSerialization(type)->deserializeBinary(res, buf, getFormatSettings()); +} + #if !defined(DEBUG_OR_SANITIZER_BUILD) -void ColumnDynamic::insertFrom(const DB::IColumn & src_, size_t n) +void ColumnDynamic::insertFrom(const IColumn & src_, size_t n) #else -void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) +void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) #endif { const auto & dynamic_src = assert_cast(src_); @@ -231,6 +318,28 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) } auto & variant_col = assert_cast(*variant_column); + const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); + auto src_global_discr = src_variant_col.globalDiscriminatorAt(n); + auto src_offset = src_variant_col.offsetAt(n); + + /// Check if we insert from shared variant and process it separately. + if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) + { + auto & src_shared_variant = dynamic_src.getSharedVariant(); + auto value = src_shared_variant.getDataAt(src_offset); + /// Decode data type of this value. + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have this variant and deserialize value into variant from shared variant data. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + variant_col.deserializeBinaryIntoVariant(it->second, getVariantSerialization(type, type_name), buf, getFormatSettings()); + /// Otherwise just insert it into our shared variant. + else + variant_col.insertIntoVariantFrom(getSharedVariantDiscriminator(), src_shared_variant, src_offset); + + return; + } /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) @@ -241,8 +350,6 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) /// We cannot combine 2 Variant types as total number of variants exceeds the limit. /// We need to insert single value, try to add only corresponding variant. - const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); - auto src_global_discr = src_variant_col.globalDiscriminatorAt(n); /// NULL doesn't require Variant extension. if (src_global_discr == ColumnVariant::NULL_DISCRIMINATOR) @@ -260,19 +367,18 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) } /// We reached maximum number of variants and couldn't add new variant. - /// We should always be able to add String variant and cast inserted value to String. - addStringVariant(); - auto tmp_variant_column = src_variant_col.getVariantByGlobalDiscriminator(src_global_discr).cloneEmpty(); - tmp_variant_column->insertFrom(src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); - auto tmp_string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - auto string_variant_discr = variant_info.variant_name_to_discriminator["String"]; - variant_col.insertIntoVariantFrom(string_variant_discr, *tmp_string_column, 0); + /// Insert this value into shared variant. + insertValueIntoSharedVariant( + src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), + variant_type, + dynamic_src.variant_info.variant_names[src_global_discr], + src_offset); } #if !defined(DEBUG_OR_SANITIZER_BUILD) -void ColumnDynamic::insertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +void ColumnDynamic::insertRangeFrom(const IColumn & src_, size_t start, size_t length) #else -void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) #endif { if (start + length > src_.size()) @@ -293,156 +399,206 @@ void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, si /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping); + size_t prev_size = variant_col.size(); + auto shared_variant_discr = getSharedVariantDiscriminator(); + variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping, shared_variant_discr); + + /// We should process insertion from srs shared variant separately, because it can contain + /// values that should be extracted into our variants. insertRangeFrom above didn't insert + /// values into our shared variant (we specified shared_variant_discr as special skip discriminator). + + /// Check if srs shared variant is empty, nothing to do in this case. + if (dynamic_src.getSharedVariant().empty()) + return; + + /// Iterate over src discriminators and process insertion from src shared variant. + const auto & src_variant_column = dynamic_src.getVariantColumn(); + const auto src_shared_variant_discr = dynamic_src.getSharedVariantDiscriminator(); + const auto src_shared_variant_local_discr = src_variant_column.localDiscriminatorByGlobal(src_shared_variant_discr); + const auto & src_local_discriminators = src_variant_column.getLocalDiscriminators(); + const auto & src_offsets = src_variant_column.getOffsets(); + const auto & src_shared_variant = assert_cast(src_variant_column.getVariantByLocalDiscriminator(src_shared_variant_local_discr)); + + auto & local_discriminators = variant_col.getLocalDiscriminators(); + auto & offsets = variant_col.getOffsets(); + const auto shared_variant_local_discr = variant_col.localDiscriminatorByGlobal(shared_variant_discr); + auto & shared_variant = assert_cast(variant_col.getVariantByLocalDiscriminator(shared_variant_local_discr)); + for (size_t i = 0; i != length; ++i) + { + if (src_local_discriminators[start + i] == src_shared_variant_local_discr) + { + chassert(local_discriminators[prev_size + i] == shared_variant_local_discr); + auto value = src_shared_variant.getDataAt(src_offsets[start + i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have variant with this type. In this case we should extract + /// the value from src shared variant and insert it into this variant. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + { + auto local_discr = variant_col.localDiscriminatorByGlobal(it->second); + auto & variant = variant_col.getVariantByLocalDiscriminator(local_discr); + getVariantSerialization(type, type_name)->deserializeBinary(variant, buf, getFormatSettings()); + /// Local discriminators were already filled in ColumnVariant::insertRangeFrom and this row should contain + /// shared_variant_local_discr. Change it to local discriminator of the found variant and update offsets. + local_discriminators[prev_size + i] = local_discr; + offsets[prev_size + i] = variant.size() - 1; + } + /// Otherwise, insert this value into shared variant. + else + { + shared_variant.insertData(value.data, value.size); + /// Update variant offset. + offsets[prev_size + i] = shared_variant.size() - 1; + } + } + } + return; } /// We cannot combine 2 Variant types as total number of variants exceeds the limit. - /// In this case we will add most frequent variants from this range and insert them as usual, - /// all other variants will be converted to String. - /// TODO: instead of keeping all current variants and just adding new most frequent variants - /// from source columns we can also try to replace rarest existing variants with frequent - /// variants from source column (so we will avoid casting new frequent variants to String - /// and keeping rare existing ones). It will require rewriting of existing data in Variant - /// column but will improve usability of Dynamic column for example during squashing blocks - /// during insert. - - const auto & src_variant_column = dynamic_src.getVariantColumn(); - - /// Calculate ranges for each variant in current range. - std::vector> variants_ranges(dynamic_src.variant_info.variant_names.size(), {0, 0}); - /// If we insert the whole column, no need to iterate through the range, we can just take variant sizes. - if (start == 0 && length == dynamic_src.size()) - { - for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) - variants_ranges[i] = {0, src_variant_column.getVariantByGlobalDiscriminator(i).size()}; - } - /// Otherwise we need to iterate through discriminators and calculate the range for each variant. - else - { - const auto & local_discriminators = src_variant_column.getLocalDiscriminators(); - const auto & offsets = src_variant_column.getOffsets(); - size_t end = start + length; - for (size_t i = start; i != end; ++i) - { - auto discr = src_variant_column.globalDiscriminatorByLocal(local_discriminators[i]); - if (discr != ColumnVariant::NULL_DISCRIMINATOR) - { - if (!variants_ranges[discr].second) - variants_ranges[discr].first = offsets[i]; - ++variants_ranges[discr].second; - } - } - } - + /// In this case we will add most frequent variants and insert them as usual, + /// all other variants will be inserted into shared variant. const auto & src_variants = assert_cast(*dynamic_src.variant_info.variant_type).getVariants(); - /// List of variants that will be converted to String. - std::vector variants_to_convert_to_string; /// Mapping from global discriminators of src_variant to the new variant we will create. std::vector other_to_new_discriminators; other_to_new_discriminators.reserve(dynamic_src.variant_info.variant_names.size()); - /// Check if we cannot add any more new variants. In this case we will convert all new variants to String. - if (variant_info.variant_names.size() == max_dynamic_types || (variant_info.variant_names.size() == max_dynamic_types - 1 && !variant_info.variant_name_to_discriminator.contains("String"))) + /// Check if we cannot add any more new variants. In this case we will insert all new variants into shared variant. + if (variant_info.variant_names.size() - 1 == max_dynamic_types) { - addStringVariant(); + auto shared_variant_discr = getSharedVariantDiscriminator(); for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) { auto it = variant_info.variant_name_to_discriminator.find(dynamic_src.variant_info.variant_names[i]); if (it == variant_info.variant_name_to_discriminator.end()) - { - variants_to_convert_to_string.push_back(i); - other_to_new_discriminators.push_back(variant_info.variant_name_to_discriminator["String"]); - } + other_to_new_discriminators.push_back(shared_variant_discr); else - { other_to_new_discriminators.push_back(it->second); - } } } - /// We still can add some new variants, but not all of them. Let's choose the most frequent variants in specified range. + /// We still can add some new variants, but not all of them. Let's choose the most frequent variants. else { + /// Create list of pairs and sort it. std::vector> new_variants_with_sizes; new_variants_with_sizes.reserve(dynamic_src.variant_info.variant_names.size()); - for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) + const auto & src_variant_column = dynamic_src.getVariantColumn(); + for (const auto & [name, discr] : dynamic_src.variant_info.variant_name_to_discriminator) { - const auto & variant_name = dynamic_src.variant_info.variant_names[i]; - if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name)) - new_variants_with_sizes.emplace_back(variants_ranges[i].second, i); + if (!variant_info.variant_name_to_discriminator.contains(name)) + new_variants_with_sizes.emplace_back(src_variant_column.getVariantByGlobalDiscriminator(discr).size(), discr); } std::sort(new_variants_with_sizes.begin(), new_variants_with_sizes.end(), std::greater()); DataTypes new_variants = assert_cast(*variant_info.variant_type).getVariants(); - if (!variant_info.variant_name_to_discriminator.contains("String")) - new_variants.push_back(std::make_shared()); - + /// Add new variants from sorted list until we reach max_dynamic_types. for (const auto & [_, discr] : new_variants_with_sizes) { - if (new_variants.size() != max_dynamic_types) - new_variants.push_back(src_variants[discr]); - else - variants_to_convert_to_string.push_back(discr); + if (new_variants.size() - 1 == max_dynamic_types) + break; + new_variants.push_back(src_variants[discr]); } auto new_variant_type = std::make_shared(new_variants); updateVariantInfoAndExpandVariantColumn(new_variant_type); - auto string_variant_discriminator = variant_info.variant_name_to_discriminator.at("String"); + auto shared_variant_discr = getSharedVariantDiscriminator(); for (const auto & variant_name : dynamic_src.variant_info.variant_names) { auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it == variant_info.variant_name_to_discriminator.end()) - other_to_new_discriminators.push_back(string_variant_discriminator); + other_to_new_discriminators.push_back(shared_variant_discr); else other_to_new_discriminators.push_back(it->second); } } - /// Convert to String all variants that couldn't be added. - std::unordered_map variants_converted_to_string; - variants_converted_to_string.reserve(variants_to_convert_to_string.size()); - for (auto discr : variants_to_convert_to_string) - { - auto [variant_start, variant_length] = variants_ranges[discr]; - const auto & variant = src_variant_column.getVariantPtrByGlobalDiscriminator(discr); - if (variant_start == 0 && variant_length == variant->size()) - variants_converted_to_string[discr] = castColumn(ColumnWithTypeAndName(variant, src_variants[discr], ""), std::make_shared()); - else - variants_converted_to_string[discr] = castColumn(ColumnWithTypeAndName(variant->cut(variant_start, variant_length), src_variants[discr], ""), std::make_shared()); - } - + /// Iterate over the range and perform insertion. + const auto & src_variant_column = dynamic_src.getVariantColumn(); const auto & src_local_discriminators = src_variant_column.getLocalDiscriminators(); const auto & src_offsets = src_variant_column.getOffsets(); const auto & src_variant_columns = src_variant_column.getVariants(); + const auto src_shared_variant_discr = dynamic_src.getSharedVariantDiscriminator(); + const auto src_shared_variant_local_discr = src_variant_column.localDiscriminatorByGlobal(src_shared_variant_discr); + const auto & src_shared_variant = assert_cast(*src_variant_columns[src_shared_variant_local_discr]); + auto & local_discriminators = variant_col.getLocalDiscriminators(); + local_discriminators.reserve(local_discriminators.size() + length); + auto & offsets = variant_col.getOffsets(); + offsets.reserve(offsets.size() + length); + auto & variant_columns = variant_col.getVariants(); + const auto shared_variant_discr = getSharedVariantDiscriminator(); + const auto shared_variant_local_discr = variant_col.localDiscriminatorByGlobal(shared_variant_discr); + auto & shared_variant = assert_cast(*variant_columns[shared_variant_local_discr]); size_t end = start + length; for (size_t i = start; i != end; ++i) { - auto local_discr = src_local_discriminators[i]; - if (local_discr == ColumnVariant::NULL_DISCRIMINATOR) + auto src_local_discr = src_local_discriminators[i]; + auto src_offset = src_offsets[i]; + if (src_local_discr == ColumnVariant::NULL_DISCRIMINATOR) { - variant_col.insertDefault(); + local_discriminators.push_back(ColumnVariant::NULL_DISCRIMINATOR); + offsets.emplace_back(); } else { - auto global_discr = src_variant_column.globalDiscriminatorByLocal(local_discr); - auto to_global_discr = other_to_new_discriminators[global_discr]; - auto it = variants_converted_to_string.find(global_discr); - if (it == variants_converted_to_string.end()) + /// Process insertion from src shared variant separately. + if (src_local_discr == src_shared_variant_local_discr) { - variant_col.insertIntoVariantFrom(to_global_discr, *src_variant_columns[local_discr], src_offsets[i]); + auto value = src_shared_variant.getDataAt(src_offset); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have variant with this type. In this case we should extract + /// the value from src shared variant and insert it into this variant. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + { + auto local_discr = variant_col.localDiscriminatorByGlobal(it->second); + getVariantSerialization(type, type_name)->deserializeBinary(*variant_columns[local_discr], buf, getFormatSettings()); + local_discriminators.push_back(local_discr); + offsets.push_back(variant_columns[local_discr]->size() - 1); + } + /// Otherwise, insert this value into shared variant. + else + { + shared_variant.insertData(value.data, value.size); + local_discriminators.push_back(shared_variant_local_discr); + offsets.push_back(shared_variant.size() - 1); + } } + /// Insertion from usual variant. else { - variant_col.insertIntoVariantFrom(to_global_discr, *it->second, src_offsets[i] - variants_ranges[global_discr].first); + auto src_global_discr = src_variant_column.globalDiscriminatorByLocal(src_local_discr); + auto global_discr = other_to_new_discriminators[src_global_discr]; + /// Check if we need to insert this value into shared variant. + if (global_discr == shared_variant_discr) + { + serializeValueIntoSharedVariant( + shared_variant, + *src_variant_columns[src_local_discr], + src_variants[src_global_discr], + getVariantSerialization(src_variants[src_global_discr], dynamic_src.variant_info.variant_names[src_global_discr]), + src_offset); + local_discriminators.push_back(shared_variant_local_discr); + offsets.push_back(shared_variant.size() - 1); + } + else + { + auto local_discr = variant_col.localDiscriminatorByGlobal(global_discr); + variant_columns[local_discr]->insertFrom(*src_variant_columns[src_local_discr], src_offset); + local_discriminators.push_back(local_discr); + offsets.push_back(variant_columns[local_discr]->size() - 1); + } } } } } #if !defined(DEBUG_OR_SANITIZER_BUILD) -void ColumnDynamic::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +void ColumnDynamic::insertManyFrom(const IColumn & src_, size_t position, size_t length) #else -void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size_t length) #endif { const auto & dynamic_src = assert_cast(src_); @@ -455,6 +611,36 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, } auto & variant_col = assert_cast(*variant_column); + const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); + auto src_global_discr = src_variant_col.globalDiscriminatorAt(position); + auto src_offset = src_variant_col.offsetAt(position); + + /// Check if we insert from shared variant and process it separately. + if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) + { + auto & src_shared_variant = dynamic_src.getSharedVariant(); + auto value = src_shared_variant.getDataAt(src_offset); + /// Decode data type of this value. + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + /// Check if we have this variant and deserialize value into variant from shared variant data. + if (auto it = variant_info.variant_name_to_discriminator.find(type_name); it != variant_info.variant_name_to_discriminator.end()) + { + /// Deserialize value into temporary column and use it in insertManyIntoVariantFrom. + auto tmp_column = type->createColumn(); + tmp_column->reserve(1); + getVariantSerialization(type, type_name)->deserializeBinary(*tmp_column, buf, getFormatSettings()); + variant_col.insertManyIntoVariantFrom(it->second, *tmp_column, 0, length); + } + /// Otherwise just insert it into our shared variant. + else + { + variant_col.insertManyIntoVariantFrom(getSharedVariantDiscriminator(), src_shared_variant, src_offset, length); + } + + return; + } /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) @@ -465,8 +651,6 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, /// We cannot combine 2 Variant types as total number of variants exceeds the limit. /// We need to insert single value, try to add only corresponding variant. - const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); - auto src_global_discr = src_variant_col.globalDiscriminatorAt(position); if (src_global_discr == ColumnVariant::NULL_DISCRIMINATOR) { insertDefault(); @@ -481,21 +665,51 @@ void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, return; } - addStringVariant(); - auto tmp_variant_column = src_variant_col.getVariantByGlobalDiscriminator(src_global_discr).cloneEmpty(); - tmp_variant_column->insertFrom(src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position)); - auto tmp_string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - auto string_variant_discr = variant_info.variant_name_to_discriminator["String"]; - variant_col.insertManyIntoVariantFrom(string_variant_discr, *tmp_string_column, 0, length); + /// We reached maximum number of variants and couldn't add new variant. + /// Insert this value into shared variant. + /// Create temporary string column, serialize value into it and use it in insertManyIntoVariantFrom. + auto tmp_shared_variant = ColumnString::create(); + serializeValueIntoSharedVariant( + *tmp_shared_variant, + src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), + variant_type, + getVariantSerialization(variant_type, dynamic_src.variant_info.variant_names[src_global_discr]), + src_offset); + + variant_col.insertManyIntoVariantFrom(getSharedVariantDiscriminator(), *tmp_shared_variant, 0, length); } +void ColumnDynamic::insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n) +{ + auto & variant_col = getVariantColumn(); + auto & shared_variant = getSharedVariant(); + serializeValueIntoSharedVariant(shared_variant, src, type, getVariantSerialization(type, type_name), n); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(getSharedVariantDiscriminator())); + variant_col.getOffsets().push_back(shared_variant.size() - 1); +} -StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, const char *& begin) const +void ColumnDynamic::serializeValueIntoSharedVariant( + ColumnString & shared_variant, + const IColumn & src, + const DataTypePtr & type, + const SerializationPtr & serialization, + size_t n) +{ + auto & chars = shared_variant.getChars(); + WriteBufferFromVector value_buf(chars, AppendModeTag()); + encodeDataType(type, value_buf); + serialization->serializeBinary(src, n, value_buf, getFormatSettings()); + value_buf.finalize(); + chars.push_back(0); + shared_variant.getOffsets().push_back(chars.size()); +} + +StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const { /// We cannot use Variant serialization here as it serializes discriminator + value, /// but Dynamic doesn't have fixed mapping discriminator <-> variant type /// as different Dynamic column can have different Variants. - /// Instead, we serialize null bit + variant type in binary format (size + bytes) + value. + /// Instead, we serialize null bit + variant type and value in binary format (size + data). const auto & variant_col = assert_cast(*variant_column); auto discr = variant_col.globalDiscriminatorAt(n); StringRef res; @@ -509,19 +723,29 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co return res; } - const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); - String variant_type_binary_data = encodeDataType(variant_type); - size_t variant_type_binary_data_size = variant_type_binary_data.size(); - char * pos = arena.allocContinue(sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(), begin); - memcpy(pos, &null_bit, sizeof(UInt8)); - memcpy(pos + sizeof(UInt8), &variant_type_binary_data_size, sizeof(size_t)); - memcpy(pos + sizeof(UInt8) + sizeof(size_t), variant_type_binary_data.data(), variant_type_binary_data.size()); - res.data = pos; - res.size = sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(); + WriteBufferFromOwnString buf; + StringRef type_and_value; + /// If we have value from shared variant, it's already stored in the desired format. + if (discr == getSharedVariantDiscriminator()) + { + type_and_value = getSharedVariant().getDataAt(variant_col.offsetAt(n)); + } + /// For regular variants serialize its type and value in binary format. + else + { + const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); + encodeDataType(variant_type, buf); + getVariantSerialization(variant_type, variant_info.variant_names[discr]) + ->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings()); + type_and_value = buf.str(); + } - auto value_ref = variant_col.getVariantByGlobalDiscriminator(discr).serializeValueIntoArena(variant_col.offsetAt(n), arena, begin); - res.data = value_ref.data - res.size; - res.size += value_ref.size; + char * pos = arena.allocContinue(sizeof(UInt8) + sizeof(size_t) + type_and_value.size, begin); + memcpy(pos, &null_bit, sizeof(UInt8)); + memcpy(pos + sizeof(UInt8), &type_and_value.size, sizeof(size_t)); + memcpy(pos + sizeof(UInt8) + sizeof(size_t), type_and_value.data, type_and_value.size); + res.data = pos; + res.size = sizeof(UInt8) + sizeof(size_t) + type_and_value.size; return res; } @@ -536,39 +760,36 @@ const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) return pos; } - /// Read variant type in binary format. - const size_t variant_type_binary_data_size = unalignedLoad(pos); - pos += sizeof(variant_type_binary_data_size); - String variant_type_binary_data; - variant_type_binary_data.resize(variant_type_binary_data_size); - memcpy(variant_type_binary_data.data(), pos, variant_type_binary_data_size); - pos += variant_type_binary_data_size; - auto variant_type = decodeDataType(variant_type_binary_data); + /// Read variant type and value in binary format. + const size_t type_and_value_size = unalignedLoad(pos); + pos += sizeof(type_and_value_size); + std::string_view type_and_value(pos, type_and_value_size); + pos += type_and_value_size; + + ReadBufferFromMemory buf(type_and_value.data(), type_and_value.size()); + auto variant_type = decodeDataType(buf); auto variant_name = variant_type->getName(); /// If we already have such variant, just deserialize it into corresponding variant column. auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it != variant_info.variant_name_to_discriminator.end()) { - auto discr = it->second; - return variant_col.deserializeVariantAndInsertFromArena(discr, pos); + variant_col.deserializeBinaryIntoVariant(it->second, getVariantSerialization(variant_type, variant_name), buf, getFormatSettings()); } - - /// If we don't have such variant, add it. - if (likely(addNewVariant(variant_type))) + /// If we don't have such variant, try to add it. + else if (likely(addNewVariant(variant_type))) { auto discr = variant_info.variant_name_to_discriminator[variant_name]; - return variant_col.deserializeVariantAndInsertFromArena(discr, pos); + variant_col.deserializeBinaryIntoVariant(discr, getVariantSerialization(variant_type, variant_name), buf, getFormatSettings()); + } + /// Otherwise insert this value into shared variant. + else + { + auto & shared_variant = getSharedVariant(); + shared_variant.insertData(type_and_value.data(), type_and_value.size()); + variant_col.getLocalDiscriminators().push_back(variant_col.localDiscriminatorByGlobal(getSharedVariantDiscriminator())); + variant_col.getOffsets().push_back(shared_variant.size() - 1); } - /// We reached maximum number of variants and couldn't add new variant. - /// We should always be able to add String variant and cast inserted value to String. - addStringVariant(); - /// Create temporary column of this variant type and deserialize value into it. - auto tmp_variant_column = variant_type->createColumn(); - pos = tmp_variant_column->deserializeAndInsertFromArena(pos); - /// Cast temporary column to String and insert this value into String variant. - auto str_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - variant_col.insertIntoVariantFrom(variant_info.variant_name_to_discriminator["String"], *str_column, 0); return pos; } @@ -579,14 +800,10 @@ const char * ColumnDynamic::skipSerializedInArena(const char * pos) const if (null_bit) return pos; - const size_t variant_type_binary_data_size = unalignedLoad(pos); - pos += sizeof(variant_type_binary_data_size); - String variant_type_binary_data; - variant_type_binary_data.resize(variant_type_binary_data_size); - memcpy(variant_type_binary_data.data(), pos, variant_type_binary_data_size); - pos += variant_type_binary_data_size; - auto tmp_variant_column = decodeDataType(variant_type_binary_data)->createColumn(); - return tmp_variant_column->skipSerializedInArena(pos); + const size_t type_and_value_size = unalignedLoad(pos); + pos += sizeof(type_and_value_size); + pos += type_and_value_size; + return pos; } void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const @@ -604,9 +821,9 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const } #if !defined(DEBUG_OR_SANITIZER_BUILD) -int ColumnDynamic::compareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +int ColumnDynamic::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const #else -int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const #endif { const auto & left_variant = assert_cast(*variant_column); @@ -614,7 +831,9 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int const auto & right_variant = assert_cast(*right_dynamic.variant_column); auto left_discr = left_variant.globalDiscriminatorAt(n); + auto left_shared_variant_discr = getSharedVariantDiscriminator(); auto right_discr = right_variant.globalDiscriminatorAt(m); + auto right_shared_variant_discr = right_dynamic.getSharedVariantDiscriminator(); /// Check if we have NULLs and return result based on nan_direction_hint. if (left_discr == ColumnVariant::NULL_DISCRIMINATOR && right_discr == ColumnVariant::NULL_DISCRIMINATOR) @@ -624,12 +843,125 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int else if (right_discr == ColumnVariant::NULL_DISCRIMINATOR) return -nan_direction_hint; - /// If rows have different types, we compare type names. - if (variant_info.variant_names[left_discr] != right_dynamic.variant_info.variant_names[right_discr]) - return variant_info.variant_names[left_discr] < right_dynamic.variant_info.variant_names[right_discr] ? -1 : 1; + /// Check if both values are in shared variant. + if (left_discr == left_shared_variant_discr && right_discr == right_shared_variant_discr) + { + /// Extract type names from both values. + auto left_value = getSharedVariant().getDataAt(left_variant.offsetAt(n)); + ReadBufferFromMemory buf_left(left_value.data, left_value.size); + auto left_data_type = decodeDataType(buf_left); + auto left_data_type_name = left_data_type->getName(); - /// If rows have the same types, compare actual values from corresponding variants. - return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); + auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); + ReadBufferFromMemory buf_right(right_value.data, right_value.size); + auto right_data_type = decodeDataType(buf_right); + auto right_data_type_name = right_data_type->getName(); + + /// If rows have different types, we compare type names. + if (left_data_type_name != right_data_type_name) + return left_data_type_name < right_data_type_name ? -1 : 1; + + /// If rows have the same type, we compare actual values. + /// We have both values serialized in binary format, so we need to + /// create temporary column, insert both values into it and compare. + auto tmp_column = left_data_type->createColumn(); + const auto & serialization = getVariantSerialization(left_data_type, left_data_type_name); + serialization->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); + serialization->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); + return tmp_column->compareAt(0, 1, *tmp_column, nan_direction_hint); + } + /// Check if only left value is in shared data. + else if (left_discr == left_shared_variant_discr) + { + /// Extract left type name from the value. + auto left_value = getSharedVariant().getDataAt(left_variant.offsetAt(n)); + ReadBufferFromMemory buf_left(left_value.data, left_value.size); + auto left_data_type = decodeDataType(buf_left); + auto left_data_type_name = left_data_type->getName(); + + /// If rows have different types, we compare type names. + if (left_data_type_name != right_dynamic.variant_info.variant_names[right_discr]) + return left_data_type_name < right_dynamic.variant_info.variant_names[right_discr] ? -1 : 1; + + /// If rows have the same type, we compare actual values. + /// We have left value serialized in binary format, we need to + /// create temporary column, insert the value into it and compare. + auto tmp_column = left_data_type->createColumn(); + getVariantSerialization(left_data_type, left_data_type_name)->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); + return tmp_column->compareAt(0, right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); + } + /// Check if only right value is in shared data. + else if (right_discr == right_shared_variant_discr) + { + /// Extract right type name from the value. + auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); + ReadBufferFromMemory buf_right(right_value.data, right_value.size); + auto right_data_type = decodeDataType(buf_right); + auto right_data_type_name = right_data_type->getName(); + + /// If rows have different types, we compare type names. + if (variant_info.variant_names[left_discr] != right_data_type_name) + return variant_info.variant_names[left_discr] < right_data_type_name ? -1 : 1; + + /// If rows have the same type, we compare actual values. + /// We have right value serialized in binary format, we need to + /// create temporary column, insert the value into it and compare. + auto tmp_column = right_data_type->createColumn(); + getVariantSerialization(right_data_type, right_data_type_name)->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); + return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), 0, *tmp_column, nan_direction_hint); + } + /// Otherwise both values are regular variants. + else + { + /// If rows have different types, we compare type names. + if (variant_info.variant_names[left_discr] != right_dynamic.variant_info.variant_names[right_discr]) + return variant_info.variant_names[left_discr] < right_dynamic.variant_info.variant_names[right_discr] ? -1 : 1; + + /// If rows have the same types, compare actual values from corresponding variants. + return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); + } +} + +struct ColumnDynamic::ComparatorBase +{ + const ColumnDynamic & parent; + int nan_direction_hint; + + ComparatorBase(const ColumnDynamic & parent_, int nan_direction_hint_) + : parent(parent_), nan_direction_hint(nan_direction_hint_) + { + } + + ALWAYS_INLINE int compare(size_t lhs, size_t rhs) const + { + return parent.compareAt(lhs, rhs, parent, nan_direction_hint); + } +}; + +void ColumnDynamic::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const +{ + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + getPermutationImpl(limit, res, ComparatorAscendingUnstable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + getPermutationImpl(limit, res, ComparatorAscendingStable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + getPermutationImpl(limit, res, ComparatorDescendingUnstable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) + getPermutationImpl(limit, res, ComparatorDescendingStable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); +} + +void ColumnDynamic::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, DB::EqualRanges & equal_ranges) const +{ + auto comparator_equal = ComparatorEqual(*this, nan_direction_hint); + + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingUnstable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingStable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingUnstable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) + updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingStable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort()); } ColumnPtr ColumnDynamic::compress() const @@ -637,14 +969,16 @@ ColumnPtr ColumnDynamic::compress() const ColumnPtr variant_compressed = variant_column->compress(); size_t byte_size = variant_compressed->byteSize(); return ColumnCompressed::create(size(), byte_size, - [my_variant_compressed = std::move(variant_compressed), my_variant_info = variant_info, my_max_dynamic_types = max_dynamic_types, my_statistics = statistics]() mutable + [my_variant_compressed = std::move(variant_compressed), my_variant_info = variant_info, my_max_dynamic_types = max_dynamic_types, my_global_max_dynamic_types = global_max_dynamic_types, my_statistics = statistics]() mutable { - return ColumnDynamic::create(my_variant_compressed->decompress(), my_variant_info, my_max_dynamic_types, my_statistics); + return ColumnDynamic::create(my_variant_compressed->decompress(), my_variant_info, my_max_dynamic_types, my_global_max_dynamic_types, my_statistics); }); } void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) { + LOG_DEBUG(getLogger("ColumnDynamic"), "takeDynamicStructureFromSourceColumns"); + if (!empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Dynamic column"); @@ -663,6 +997,9 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source /// First, collect all variants from all source columns and calculate total sizes. std::unordered_map total_sizes; DataTypes all_variants; + /// Add shared variant type in advance; + all_variants.push_back(getSharedVariantDataType()); + total_sizes[getSharedVariantTypeName()] = 0; for (const auto & source_column : source_columns) { @@ -671,7 +1008,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source const auto & source_variant_info = source_dynamic.getVariantInfo(); const auto & source_variants = assert_cast(*source_variant_info.variant_type).getVariants(); /// During deserialization from MergeTree we will have variant sizes statistics from the whole data part. - const auto & source_statistics = source_dynamic.getStatistics(); + const auto & source_statistics = source_dynamic.getStatistics(); for (size_t i = 0; i != source_variants.size(); ++i) { const auto & variant_name = source_variant_info.variant_names[i]; @@ -682,37 +1019,67 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source all_variants.push_back(source_variants[i]); it = total_sizes.emplace(variant_name, 0).first; } - auto statistics_it = source_statistics.data.find(variant_name); - size_t size = statistics_it == source_statistics.data.end() ? source_variant_column.getVariantByGlobalDiscriminator(i).size() : statistics_it->second; + size_t size = source_variant_column.getVariantByGlobalDiscriminator(i).size(); + if (source_statistics) + { + auto statistics_it = source_statistics->variants_statistics.find(variant_name); + if (statistics_it != source_statistics->variants_statistics.end()) + size = statistics_it->second; + } + it->second += size; } + + /// Use add variants from shared variant statistics. It can help extracting + /// frequent variants from shared variant to usual variants. + if (source_statistics) + { + for (const auto & [variant_name, size] : source_statistics->shared_variants_statistics) + { + auto it = total_sizes.find(variant_name); + /// Add this variant to the list of all variants if we didn't see it yet. + if (it == total_sizes.end()) + { + all_variants.push_back(DataTypeFactory::instance().get(variant_name)); + it = total_sizes.emplace(variant_name, 0).first; + } + it->second += size; + } + } } DataTypePtr result_variant_type; - /// Check if the number of all variants exceeds the limit. - if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_sizes.contains("String"))) + Statistics new_statistics(Statistics::Source::MERGE); + /// Check if the number of all dynamic types exceeds the limit. + if (all_variants.size() - 1 > global_max_dynamic_types) { /// Create list of variants with their sizes and sort it. std::vector> variants_with_sizes; variants_with_sizes.reserve(all_variants.size()); for (const auto & variant : all_variants) - variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant); + { + if (variant->getName() != getSharedVariantTypeName()) + variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant); + } std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); - /// Take first max_dynamic_types variants from sorted list. + /// Take first max_dynamic_types variants from sorted list and fill shared_variants_statistics with the rest. DataTypes result_variants; - result_variants.reserve(max_dynamic_types); - /// Add String variant in advance. - result_variants.push_back(std::make_shared()); - for (const auto & [_, variant] : variants_with_sizes) + result_variants.reserve(global_max_dynamic_types + 1); + for (const auto & [size, variant] : variants_with_sizes) { - if (result_variants.size() == max_dynamic_types) - break; - - if (variant->getName() != "String") + /// Add variant to the resulting variants list until we reach max_dynamic_types. + if (result_variants.size() < global_max_dynamic_types) result_variants.push_back(variant); + /// Add all remaining variants into shared_variants_statistics until we reach its max size. + else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) + new_statistics.shared_variants_statistics[variant->getName()] = size; + else + break; } + /// Add shared variant. + result_variants.push_back(getSharedVariantDataType()); result_variant_type = std::make_shared(result_variants); } else @@ -720,26 +1087,16 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source result_variant_type = std::make_shared(all_variants); } - /// Now we have resulting Variant and can fill variant info. - variant_info.variant_type = result_variant_type; - variant_info.variant_name = result_variant_type->getName(); - const auto & result_variants = assert_cast(*result_variant_type).getVariants(); - variant_info.variant_names.clear(); - variant_info.variant_names.reserve(result_variants.size()); - variant_info.variant_name_to_discriminator.clear(); - variant_info.variant_name_to_discriminator.reserve(result_variants.size()); - statistics.data.clear(); - statistics.data.reserve(result_variants.size()); - statistics.source = Statistics::Source::MERGE; - for (size_t i = 0; i != result_variants.size(); ++i) - { - auto variant_name = result_variants[i]->getName(); - variant_info.variant_names.push_back(variant_name); - variant_info.variant_name_to_discriminator[variant_name] = i; - statistics.data[variant_name] = total_sizes[variant_name]; - } + /// Now we have resulting Variant and can fill variant info and create merge statistics. + setVariantType(result_variant_type); + new_statistics.variants_statistics.reserve(variant_info.variant_names.size()); + for (const auto & variant_name : variant_info.variant_names) + new_statistics.variants_statistics[variant_name] = total_sizes[variant_name]; + statistics = std::make_shared(std::move(new_statistics)); - variant_column = variant_info.variant_type->createColumn(); + /// Reduce max_dynamic_types to the number of selected variants (without shared variant), so there will be no possibility + /// to extend selected variants on inerts into this column during merges. + max_dynamic_types = variant_info.variant_names.size() - 1; /// Now we have the resulting Variant that will be used in all merged columns. /// Variants can also contain Dynamic columns inside, we should collect diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index e92cabd3db9..8b815e2b015 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -19,11 +20,15 @@ namespace DB * * When new values are inserted into Dynamic column, the internal Variant * type and column are extended if the inserted value has new type. + * When the limit on number of dynamic types is exceeded, all values + * with new types are inserted into special shared variant with type String + * that contains values and their types in binary format. */ class ColumnDynamic final : public COWHelper, ColumnDynamic> { public: - /// + static constexpr const char * SHARED_VARIANT_TYPE_NAME = "SharedVariant"; + struct Statistics { enum class Source @@ -32,12 +37,27 @@ public: MERGE, /// Statistics were calculated during merge of several MergeTree parts. }; + Statistics(Source source_) : source(source_) {} + /// Source of the statistics. Source source; - /// Statistics data: (variant name) -> (total variant size in data part). - std::unordered_map data; + /// Statistics data for usual variants: (variant name) -> (total variant size in data part). + std::unordered_map variants_statistics; + /// Statistics data for variants from shared variant: (variant name) -> (total variant size in data part). + /// For shared variant we store statistics only for first 256 variants (should cover almost all cases and it's not expensive). + static constexpr const size_t MAX_SHARED_VARIANT_STATISTICS_SIZE = 256; + std::unordered_map shared_variants_statistics; }; + using StatisticsPtr = std::shared_ptr; + + struct ComparatorBase; + using ComparatorAscendingUnstable = ComparatorAscendingUnstableImpl; + using ComparatorAscendingStable = ComparatorAscendingStableImpl; + using ComparatorDescendingUnstable = ComparatorDescendingUnstableImpl; + using ComparatorDescendingStable = ComparatorDescendingStableImpl; + using ComparatorEqual = ComparatorEqualImpl; + private: friend class COWHelper, ColumnDynamic>; @@ -54,28 +74,32 @@ private: }; explicit ColumnDynamic(size_t max_dynamic_types_); - ColumnDynamic(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_ = {}); + ColumnDynamic(MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}); + ColumnDynamic(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}); public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ using Base = COWHelper, ColumnDynamic>; - static Ptr create(const ColumnPtr & variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_ = {}) + static Ptr create(const ColumnPtr & variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) { - return ColumnDynamic::create(variant_column_->assumeMutable(), variant_info_, max_dynamic_types_, statistics_); + return ColumnDynamic::create(variant_column_->assumeMutable(), variant_info_, max_dynamic_types_, global_max_dynamic_types_, statistics_); } - static MutablePtr create(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, const Statistics & statistics_ = {}) + static MutablePtr create(MutableColumnPtr variant_column_, const VariantInfo & variant_info_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) { - return Base::create(std::move(variant_column_), variant_info_, max_dynamic_types_, statistics_); + return Base::create(std::move(variant_column_), variant_info_, max_dynamic_types_, global_max_dynamic_types_, statistics_); } - static MutablePtr create(MutableColumnPtr variant_column_, const DataTypePtr & variant_type, size_t max_dynamic_types_, const Statistics & statistics_ = {}); - - static ColumnPtr create(ColumnPtr variant_column_, const DataTypePtr & variant_type, size_t max_dynamic_types_, const Statistics & statistics_ = {}) + static MutablePtr create(MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) { - return create(variant_column_->assumeMutable(), variant_type, max_dynamic_types_, statistics_); + return Base::create(std::move(variant_column_), variant_type_, max_dynamic_types_, global_max_dynamic_types_, statistics_); + } + + static ColumnPtr create(ColumnPtr variant_column_, const DataTypePtr & variant_type, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_ = {}) + { + return create(variant_column_->assumeMutable(), variant_type, max_dynamic_types_, global_max_dynamic_types_, statistics_); } static MutablePtr create(size_t max_dynamic_types_) @@ -83,7 +107,7 @@ public: return Base::create(max_dynamic_types_); } - std::string getName() const override { return "Dynamic(max_types=" + std::to_string(max_dynamic_types) + ")"; } + std::string getName() const override { return "Dynamic(max_types=" + std::to_string(global_max_dynamic_types) + ")"; } const char * getFamilyName() const override { @@ -98,12 +122,12 @@ public: MutableColumnPtr cloneEmpty() const override { /// Keep current dynamic structure - return Base::create(variant_column->cloneEmpty(), variant_info, max_dynamic_types, statistics); + return Base::create(variant_column->cloneEmpty(), variant_info, max_dynamic_types, global_max_dynamic_types, statistics); } MutableColumnPtr cloneResized(size_t size) const override { - return Base::create(variant_column->cloneResized(size), variant_info, max_dynamic_types, statistics); + return Base::create(variant_column->cloneResized(size), variant_info, max_dynamic_types, global_max_dynamic_types, statistics); } size_t size() const override @@ -111,15 +135,9 @@ public: return variant_column->size(); } - Field operator[](size_t n) const override - { - return (*variant_column)[n]; - } + Field operator[](size_t n) const override; - void get(size_t n, Field & res) const override - { - variant_column->get(n, res); - } + void get(size_t n, Field & res) const override; bool isDefaultAt(size_t n) const override { @@ -187,7 +205,7 @@ public: ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - return create(variant_column->filter(filt, result_size_hint), variant_info, max_dynamic_types); + return create(variant_column->filter(filt, result_size_hint), variant_info, max_dynamic_types, global_max_dynamic_types); } void expand(const Filter & mask, bool inverted) override @@ -197,17 +215,17 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override { - return create(variant_column->permute(perm, limit), variant_info, max_dynamic_types); + return create(variant_column->permute(perm, limit), variant_info, max_dynamic_types, global_max_dynamic_types); } ColumnPtr index(const IColumn & indexes, size_t limit) const override { - return create(variant_column->index(indexes, limit), variant_info, max_dynamic_types); + return create(variant_column->index(indexes, limit), variant_info, max_dynamic_types, global_max_dynamic_types); } ColumnPtr replicate(const Offsets & replicate_offsets) const override { - return create(variant_column->replicate(replicate_offsets), variant_info, max_dynamic_types); + return create(variant_column->replicate(replicate_offsets), variant_info, max_dynamic_types, global_max_dynamic_types); } MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override @@ -216,7 +234,7 @@ public: MutableColumns scattered_columns; scattered_columns.reserve(num_columns); for (auto & scattered_variant_column : scattered_variant_columns) - scattered_columns.emplace_back(create(std::move(scattered_variant_column), variant_info, max_dynamic_types)); + scattered_columns.emplace_back(create(std::move(scattered_variant_column), variant_info, max_dynamic_types, global_max_dynamic_types)); return scattered_columns; } @@ -238,16 +256,10 @@ public: } void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override - { - variant_column->getPermutation(direction, stability, limit, nan_direction_hint, res); - } + size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override - { - variant_column->updatePermutation(direction, stability, limit, nan_direction_hint, res, equal_ranges); - } + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; void reserve(size_t n) override { @@ -293,7 +305,7 @@ public: bool structureEquals(const IColumn & rhs) const override { if (const auto * rhs_concrete = typeid_cast(&rhs)) - return max_dynamic_types == rhs_concrete->max_dynamic_types; + return global_max_dynamic_types == rhs_concrete->global_max_dynamic_types; return false; } @@ -336,17 +348,67 @@ public: const ColumnVariant & getVariantColumn() const { return assert_cast(*variant_column); } ColumnVariant & getVariantColumn() { return assert_cast(*variant_column); } - bool addNewVariant(const DataTypePtr & new_variant); - void addStringVariant(); + bool addNewVariant(const DataTypePtr & new_variant, const String & new_variant_name); + bool addNewVariant(const DataTypePtr & new_variant) { return addNewVariant(new_variant, new_variant->getName()); } bool hasDynamicStructure() const override { return true; } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; - const Statistics & getStatistics() const { return statistics; } + const StatisticsPtr & getStatistics() const { return statistics; } + void setStatistics(const StatisticsPtr & statistics_) { statistics = statistics_; } size_t getMaxDynamicTypes() const { return max_dynamic_types; } + void setVariantType(const DataTypePtr & variant_type); + void setMaxDynamicPaths(size_t max_dynamic_type_); + + static const String & getSharedVariantTypeName() + { + static const String name = SHARED_VARIANT_TYPE_NAME; + return name; + } + + static DataTypePtr getSharedVariantDataType(); + + ColumnVariant::Discriminator getSharedVariantDiscriminator() const + { + return variant_info.variant_name_to_discriminator.at(getSharedVariantTypeName()); + } + + ColumnString & getSharedVariant() + { + return assert_cast(getVariantColumn().getVariantByGlobalDiscriminator(getSharedVariantDiscriminator())); + } + + const ColumnString & getSharedVariant() const + { + return assert_cast(getVariantColumn().getVariantByGlobalDiscriminator(getSharedVariantDiscriminator())); + } + + /// Serializes type and value in binary format into provided shared variant. Doesn't update Variant discriminators and offsets. + static void serializeValueIntoSharedVariant(ColumnString & shared_variant, const IColumn & src, const DataTypePtr & type, const SerializationPtr & serialization, size_t n); + + /// Insert value into shared variant. Also updates Variant discriminators and offsets. + void insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n); + + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) const + { + /// Get serialization for provided data type. + /// To avoid calling type->getDefaultSerialization() every time we use simple cache with max size. + /// When max size is reached, just clear the cache. + if (serialization_cache.size() == SERIALIZATION_CACHE_MAX_SIZE) + serialization_cache.clear(); + + if (auto it = serialization_cache.find(variant_name); it != serialization_cache.end()) + return it->second; + + return serialization_cache.emplace(variant_name, variant_type->getDefaultSerialization()).first->second; + } + + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } private: + void createVariantInfo(const DataTypePtr & variant_type); + /// Combine current variant with the other variant and return global discriminators mapping /// from other variant to the combined one. It's used for inserting from /// different variants. @@ -359,12 +421,19 @@ private: /// Store the type of current variant with some additional information. VariantInfo variant_info; /// The maximum number of different types that can be stored in this Dynamic column. - /// If exceeded, all new variants will be converted to String. + /// If exceeded, all new variants will be added to a special shared variant with type String + /// in binary format. This limit can be different for different instances of Dynamic column. + /// When max_dynamic_types = 0, we will have only shared variant and insert all values into it. size_t max_dynamic_types; + /// The types limit specified in the data type by the user Dynamic(max_types=N). + /// max_dynamic_types in all column instances of this Dynamic type can be only smaller + /// (for example, max_dynamic_types can be reduced in takeDynamicStructureFromSourceColumns + /// before merge of different Dynamic columns). + size_t global_max_dynamic_types; /// Size statistics of each variants from MergeTree data part. /// Used in takeDynamicStructureFromSourceColumns and set during deserialization. - Statistics statistics; + StatisticsPtr statistics; /// Cache (Variant name) -> (global discriminators mapping from this variant to current variant in Dynamic column). /// Used to avoid mappings recalculation in combineVariants for the same Variant types. @@ -372,6 +441,17 @@ private: /// Cache of Variant types that couldn't be combined with current variant in Dynamic column. /// Used to avoid checking if combination is possible for the same Variant types. std::unordered_set variants_with_failed_combination; + + /// We can use serializations of different data types to serialize values into shared variant. + /// To avoid creating the same serialization multiple times, use simple cache. + static const size_t SERIALIZATION_CACHE_MAX_SIZE = 256; + mutable std::unordered_map serialization_cache; }; +void extendVariantColumn( + IColumn & variant_column, + const DataTypePtr & old_variant_type, + const DataTypePtr & new_variant_type, + std::unordered_map old_variant_name_to_discriminator); + } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index de7efb41d19..7531e976926 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -476,7 +476,7 @@ void ColumnVariant::insertFromImpl(const DB::IColumn & src_, size_t n, const std } } -void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping) +void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator) { const size_t num_variants = variants.size(); const auto & src = assert_cast(src_); @@ -557,9 +557,12 @@ void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, Discriminator global_discr = src_global_discr; if (global_discriminators_mapping && src_global_discr != NULL_DISCRIMINATOR) global_discr = (*global_discriminators_mapping)[src_global_discr]; - Discriminator local_discr = localDiscriminatorByGlobal(global_discr); - if (nested_length) - variants[local_discr]->insertRangeFrom(*src.variants[src_local_discr], nested_start, nested_length); + if (!skip_discriminator || global_discr != *skip_discriminator) + { + Discriminator local_discr = localDiscriminatorByGlobal(global_discr); + if (nested_length) + variants[local_discr]->insertRangeFrom(*src.variants[src_local_discr], nested_start, nested_length); + } } } @@ -610,7 +613,7 @@ void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t l void ColumnVariant::doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) #endif { - insertRangeFromImpl(src_, start, length, nullptr); + insertRangeFromImpl(src_, start, length, nullptr, nullptr); } #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -627,9 +630,9 @@ void ColumnVariant::insertFrom(const DB::IColumn & src_, size_t n, const std::ve insertFromImpl(src_, n, &global_discriminators_mapping); } -void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping) +void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping, Discriminator skip_discriminator) { - insertRangeFromImpl(src_, start, length, &global_discriminators_mapping); + insertRangeFromImpl(src_, start, length, &global_discriminators_mapping, &skip_discriminator); } void ColumnVariant::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length, const std::vector & global_discriminators_mapping) @@ -673,6 +676,14 @@ void ColumnVariant::insertManyIntoVariantFrom(DB::ColumnVariant::Discriminator g variants[local_discr]->insertManyFrom(src_, position, length); } +void ColumnVariant::deserializeBinaryIntoVariant(ColumnVariant::Discriminator global_discr, const SerializationPtr & serialization, ReadBuffer & buf, const FormatSettings & format_settings) +{ + auto local_discr = localDiscriminatorByGlobal(global_discr); + serialization->deserializeBinary(*variants[local_discr], buf, format_settings); + getLocalDiscriminators().push_back(local_discr); + getOffsets().push_back(variants[local_discr]->size() - 1); +} + void ColumnVariant::insertDefault() { getLocalDiscriminators().push_back(NULL_DISCRIMINATOR); @@ -1213,9 +1224,7 @@ struct ColumnVariant::ComparatorBase ALWAYS_INLINE int compare(size_t lhs, size_t rhs) const { - int res = parent.compareAt(lhs, rhs, parent, nan_direction_hint); - - return res; + return parent.compareAt(lhs, rhs, parent, nan_direction_hint); } }; diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 34c24b5428d..571a843d113 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -2,6 +2,8 @@ #include #include +#include +#include namespace DB @@ -196,13 +198,15 @@ public: /// Methods for insertion from another Variant but with known mapping between global discriminators. void insertFrom(const IColumn & src_, size_t n, const std::vector & global_discriminators_mapping); - void insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping); + /// Don't insert data into variant with skip_discriminator global discriminator, it will be processed separately. + void insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping, Discriminator skip_discriminator); void insertManyFrom(const IColumn & src_, size_t position, size_t length, const std::vector & global_discriminators_mapping); /// Methods for insertion into a specific variant. void insertIntoVariantFrom(Discriminator global_discr, const IColumn & src_, size_t n); void insertRangeIntoVariantFrom(Discriminator global_discr, const IColumn & src_, size_t start, size_t length); void insertManyIntoVariantFrom(Discriminator global_discr, const IColumn & src_, size_t position, size_t length); + void deserializeBinaryIntoVariant(Discriminator global_discr, const SerializationPtr & serialization, ReadBuffer & buf, const FormatSettings & format_settings); void insertDefault() override; void insertManyDefaults(size_t length) override; @@ -263,6 +267,7 @@ public: ColumnPtr & getVariantPtrByGlobalDiscriminator(size_t discr) { return variants[global_to_local_discriminators.at(discr)]; } const NestedColumns & getVariants() const { return variants; } + NestedColumns & getVariants() { return variants; } const IColumn & getLocalDiscriminatorsColumn() const { return *local_discriminators; } IColumn & getLocalDiscriminatorsColumn() { return *local_discriminators; } @@ -302,6 +307,8 @@ public: return true; } + std::vector getLocalToGlobalDiscriminatorsMapping() const { return local_to_global_discriminators; } + /// Check if we have only 1 non-empty variant and no NULL values, /// and if so, return the discriminator of this non-empty column. std::optional getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls() const; @@ -322,7 +329,7 @@ public: private: void insertFromImpl(const IColumn & src_, size_t n, const std::vector * global_discriminators_mapping); - void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping); + void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator); void insertManyFromImpl(const IColumn & src_, size_t position, size_t length, const std::vector * global_discriminators_mapping); void initIdentityGlobalToLocalDiscriminatorsMapping(); diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index a2862b09de1..5445bd525d9 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -9,9 +9,12 @@ TEST(ColumnDynamic, CreateEmpty) { auto column = ColumnDynamic::create(255); ASSERT_TRUE(column->empty()); - ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant()"); - ASSERT_TRUE(column->getVariantInfo().variant_names.empty()); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.empty()); + ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_names[0], "SharedVariant"); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.at("SharedVariant"), 0); + ASSERT_TRUE(column->getVariantColumn().getVariantByGlobalDiscriminator(0).empty()); } TEST(ColumnDynamic, InsertDefault) @@ -19,9 +22,12 @@ TEST(ColumnDynamic, InsertDefault) auto column = ColumnDynamic::create(255); column->insertDefault(); ASSERT_TRUE(column->size() == 1); - ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant()"); - ASSERT_TRUE(column->getVariantInfo().variant_names.empty()); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.empty()); + ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_names[0], "SharedVariant"); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.size(), 1); + ASSERT_EQ(column->getVariantInfo().variant_name_to_discriminator.at("SharedVariant"), 0); + ASSERT_TRUE(column->getVariantColumn().getVariantByGlobalDiscriminator(0).empty()); ASSERT_TRUE(column->isNullAt(0)); ASSERT_EQ((*column)[0], Field(Null())); } @@ -41,10 +47,10 @@ TEST(ColumnDynamic, InsertFields) column->insert(Field(43.43)); ASSERT_TRUE(column->size() == 10); - ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, String)"); - std::vector expected_names = {"Float64", "Int8", "String"}; + ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, SharedVariant, String)"); + std::vector expected_names = {"Float64", "Int8", "SharedVariant", "String"}; ASSERT_EQ(column->getVariantInfo().variant_names, expected_names); - std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"String", 2}}; + std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}; ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator == expected_variant_name_to_discriminator); } @@ -66,56 +72,66 @@ TEST(ColumnDynamic, InsertFieldsOverflow1) { auto column = getDynamicWithManyVariants(253); - ASSERT_EQ(column->getVariantInfo().variant_names.size(), 253); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 254); column->insert(Field(42.42)); - ASSERT_EQ(column->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column->size(), 254); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("Float64")); column->insert(Field(42)); + ASSERT_EQ(column->size(), 255); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 1); Field field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "42"); + ASSERT_EQ(field, 42); column->insert(Field(43)); + ASSERT_EQ(column->size(), 256); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 2); field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "43"); + ASSERT_EQ(field, 43); column->insert(Field("str1")); + ASSERT_EQ(column->size(), 257); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 3); field = (*column)[column->size() - 1]; ASSERT_EQ(field, "str1"); column->insert(Field(Array({Field(42), Field(43)}))); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 4); field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "[42, 43]"); + ASSERT_EQ(field, Field(Array({Field(42), Field(43)}))); } TEST(ColumnDynamic, InsertFieldsOverflow2) { auto column = getDynamicWithManyVariants(254); - ASSERT_EQ(column->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); column->insert(Field("str1")); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 1); + Field field = (*column)[column->size() - 1]; + ASSERT_EQ(field, "str1"); column->insert(Field(42)); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); - Field field = (*column)[column->size() - 1]; - ASSERT_EQ(field, "42"); + ASSERT_FALSE(column->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column->getSharedVariant().size(), 2); + field = (*column)[column->size() - 1]; + ASSERT_EQ(field, 42); } ColumnDynamic::MutablePtr getInsertFromColumn(size_t num = 1) @@ -155,7 +171,7 @@ void checkInsertFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynami TEST(ColumnDynamic, InsertFrom1) { auto column_to = ColumnDynamic::create(255); - checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertFrom2) @@ -165,7 +181,7 @@ TEST(ColumnDynamic, InsertFrom2) column_to->insert(Field(42.42)); column_to->insert(Field("str")); - checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertFrom3) @@ -176,7 +192,7 @@ TEST(ColumnDynamic, InsertFrom3) column_to->insert(Field("str")); column_to->insert(Array({Field(42)})); - checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, String)", {"Array(Int8)", "Float64", "Int8", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"String", 3}}); + checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, SharedVariant, String)", {"Array(Int8)", "Float64", "Int8", "SharedVariant", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"SharedVariant", 3}, {"String", 4}}); } TEST(ColumnDynamic, InsertFromOverflow1) @@ -188,7 +204,7 @@ TEST(ColumnDynamic, InsertFromOverflow1) auto column_to = getDynamicWithManyVariants(253); column_to->insertFrom(*column_from, 0); - ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); auto field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, 42); @@ -196,13 +212,15 @@ TEST(ColumnDynamic, InsertFromOverflow1) column_to->insertFrom(*column_from, 1); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); column_to->insertFrom(*column_from, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, "str"); } @@ -221,9 +239,32 @@ TEST(ColumnDynamic, InsertFromOverflow2) column_to->insertFrom(*column_from, 1); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); +} + +TEST(ColumnDynamic, InsertFromOverflow3) +{ + auto column_from = ColumnDynamic::create(1); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + + auto column_to = ColumnDynamic::create(255); + column_to->insert(Field(41)); + + column_to->insertFrom(*column_from, 0); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); + auto field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42); + + column_to->insertFrom(*column_from, 1); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42.42); } void checkInsertManyFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynamic::MutablePtr & column_to, const std::string & expected_variant, const std::vector & expected_names, const std::unordered_map & expected_variant_name_to_discriminator) @@ -257,7 +298,7 @@ void checkInsertManyFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDy TEST(ColumnDynamic, InsertManyFrom1) { auto column_to = ColumnDynamic::create(255); - checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertManyFrom2) @@ -267,7 +308,7 @@ TEST(ColumnDynamic, InsertManyFrom2) column_to->insert(Field(42.42)); column_to->insert(Field("str")); - checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertManyFrom3) @@ -278,7 +319,7 @@ TEST(ColumnDynamic, InsertManyFrom3) column_to->insert(Field("str")); column_to->insert(Array({Field(42)})); - checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, String)", {"Array(Int8)", "Float64", "Int8", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"String", 3}}); + checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Array(Int8), Float64, Int8, SharedVariant, String)", {"Array(Int8)", "Float64", "Int8", "SharedVariant", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"SharedVariant", 3}, {"String", 4}}); } TEST(ColumnDynamic, InsertManyFromOverflow1) @@ -290,8 +331,9 @@ TEST(ColumnDynamic, InsertManyFromOverflow1) auto column_to = getDynamicWithManyVariants(253); column_to->insertManyFrom(*column_from, 0, 2); - ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); auto field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, 42); field = (*column_to)[column_to->size() - 1]; @@ -300,15 +342,17 @@ TEST(ColumnDynamic, InsertManyFromOverflow1) column_to->insertManyFrom(*column_from, 1, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); column_to->insertManyFrom(*column_from, 2, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, "str"); field = (*column_to)[column_to->size() - 2]; @@ -323,8 +367,9 @@ TEST(ColumnDynamic, InsertManyFromOverflow2) auto column_to = getDynamicWithManyVariants(253); column_to->insertManyFrom(*column_from, 0, 2); - ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 254); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); auto field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, 42); field = (*column_to)[column_to->size() - 1]; @@ -333,11 +378,39 @@ TEST(ColumnDynamic, InsertManyFromOverflow2) column_to->insertManyFrom(*column_from, 1, 2); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, "42.42"); + ASSERT_EQ(field, 42.42); +} + + +TEST(ColumnDynamic, InsertManyFromOverflow3) +{ + auto column_from = ColumnDynamic::create(1); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + + auto column_to = ColumnDynamic::create(255); + column_to->insert(Field(41)); + + column_to->insertManyFrom(*column_from, 0, 2); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_EQ(column_to->getSharedVariant().size(), 0); + auto field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, 42); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42); + + column_to->insertManyFrom(*column_from, 1, 2); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, 42.42); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, 42.42); } void checkInsertRangeFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynamic::MutablePtr & column_to, const std::string & expected_variant, const std::vector & expected_names, const std::unordered_map & expected_variant_name_to_discriminator) @@ -369,7 +442,7 @@ void checkInsertRangeFrom(const ColumnDynamic::MutablePtr & column_from, ColumnD TEST(ColumnDynamic, InsertRangeFrom1) { auto column_to = ColumnDynamic::create(255); - checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertRangeFrom2) @@ -379,7 +452,7 @@ TEST(ColumnDynamic, InsertRangeFrom2) column_to->insert(Field(42.42)); column_to->insert(Field("str1")); - checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, String)", {"Float64", "Int8", "String"}, {{"Float64", 0}, {"Int8", 1}, {"String", 2}}); + checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertRangeFrom3) @@ -390,7 +463,7 @@ TEST(ColumnDynamic, InsertRangeFrom3) column_to->insert(Field("str1")); column_to->insert(Array({Field(42)})); - checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Array(Int8), Float64, Int8, String)", {"Array(Int8)", "Float64", "Int8", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"String", 3}}); + checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Array(Int8), Float64, Int8, SharedVariant, String)", {"Array(Int8)", "Float64", "Int8", "SharedVariant", "String"}, {{"Array(Int8)", 0}, {"Float64", 1}, {"Int8", 2}, {"SharedVariant", 3}, {"String", 4}}); } TEST(ColumnDynamic, InsertRangeFromOverflow1) @@ -403,16 +476,18 @@ TEST(ColumnDynamic, InsertRangeFromOverflow1) auto column_to = getDynamicWithManyVariants(253); column_to->insertRangeFrom(*column_from, 0, 4); + ASSERT_EQ(column_to->size(), 257); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); auto field = (*column_to)[column_to->size() - 4]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, Field("str")); } @@ -428,14 +503,15 @@ TEST(ColumnDynamic, InsertRangeFromOverflow2) column_to->insertRangeFrom(*column_from, 0, 3); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 1); auto field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); } TEST(ColumnDynamic, InsertRangeFromOverflow3) @@ -449,15 +525,16 @@ TEST(ColumnDynamic, InsertRangeFromOverflow3) column_to->insert(Field("Str")); column_to->insertRangeFrom(*column_from, 0, 3); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 3); auto field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); } TEST(ColumnDynamic, InsertRangeFromOverflow4) @@ -471,12 +548,13 @@ TEST(ColumnDynamic, InsertRangeFromOverflow4) column_to->insertRangeFrom(*column_from, 0, 3); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 3); auto field = (*column_to)[column_to->size() - 3]; - ASSERT_EQ(field, Field("42")); + ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, Field("str")); } @@ -493,15 +571,16 @@ TEST(ColumnDynamic, InsertRangeFromOverflow5) column_to->insert(Field("str")); column_to->insertRangeFrom(*column_from, 0, 4); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_EQ(column_to->getSharedVariant().size(), 3); auto field = (*column_to)[column_to->size() - 4]; ASSERT_EQ(field, Field(42)); field = (*column_to)[column_to->size() - 3]; ASSERT_EQ(field, Field(43)); field = (*column_to)[column_to->size() - 2]; - ASSERT_EQ(field, Field("42.42")); + ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 1]; ASSERT_EQ(field, Field("str")); } @@ -520,13 +599,14 @@ TEST(ColumnDynamic, InsertRangeFromOverflow6) auto column_to = getDynamicWithManyVariants(253); column_to->insertRangeFrom(*column_from, 2, 5); ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 255); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); - ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); auto field = (*column_to)[column_to->size() - 5]; - ASSERT_EQ(field, Field("44")); + ASSERT_EQ(field, Field(44)); field = (*column_to)[column_to->size() - 4]; ASSERT_EQ(field, Field(42.42)); field = (*column_to)[column_to->size() - 3]; @@ -534,7 +614,131 @@ TEST(ColumnDynamic, InsertRangeFromOverflow6) field = (*column_to)[column_to->size() - 2]; ASSERT_EQ(field, Field("str")); field = (*column_to)[column_to->size() - 1]; - ASSERT_EQ(field, Field("[42]")); + ASSERT_EQ(field, Field(Array({Field(42)}))); +} + +TEST(ColumnDynamic, InsertRangeFromOverflow7) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field(42.42)); + column_from->insert(Field("str1")); + column_from->insert(Field(42)); + column_from->insert(Field(43.43)); + column_from->insert(Field(Array({Field(41)}))); + column_from->insert(Field(43)); + column_from->insert(Field("str2")); + column_from->insert(Field(Array({Field(42)}))); + + auto column_to = ColumnDynamic::create(255); + column_to->insert(Field(42)); + + column_to->insertRangeFrom(*column_from, 0, 8); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 4); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); + auto field = (*column_to)[column_to->size() - 8]; + ASSERT_EQ(field, Field(42.42)); + field = (*column_to)[column_to->size() - 7]; + ASSERT_EQ(field, Field("str1")); + field = (*column_to)[column_to->size() - 6]; + ASSERT_EQ(field, Field(42)); + field = (*column_to)[column_to->size() - 5]; + ASSERT_EQ(field, Field(43.43)); + field = (*column_to)[column_to->size() - 4]; + ASSERT_EQ(field, Field(Array({Field(41)}))); + field = (*column_to)[column_to->size() - 3]; + ASSERT_EQ(field, Field(43)); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, Field(Array({Field(42)}))); +} + +TEST(ColumnDynamic, InsertRangeFromOverflow8) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field(42.42)); + column_from->insert(Field("str1")); + column_from->insert(Field(42)); + column_from->insert(Field(43.43)); + column_from->insert(Field(Array({Field(41)}))); + column_from->insert(Field(43)); + column_from->insert(Field("str2")); + column_from->insert(Field(Array({Field(42)}))); + + auto column_to = ColumnDynamic::create(3); + column_to->insert(Field(42)); + column_from->insert(Field("str1")); + + column_to->insertRangeFrom(*column_from, 0, 8); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 3); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); + auto field = (*column_to)[column_to->size() - 8]; + ASSERT_EQ(field, Field(42.42)); + field = (*column_to)[column_to->size() - 7]; + ASSERT_EQ(field, Field("str1")); + field = (*column_to)[column_to->size() - 6]; + ASSERT_EQ(field, Field(42)); + field = (*column_to)[column_to->size() - 5]; + ASSERT_EQ(field, Field(43.43)); + field = (*column_to)[column_to->size() - 4]; + ASSERT_EQ(field, Field(Array({Field(41)}))); + field = (*column_to)[column_to->size() - 3]; + ASSERT_EQ(field, Field(43)); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, Field(Array({Field(42)}))); +} + +TEST(ColumnDynamic, InsertRangeFromOverflow9) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field("str1")); + column_from->insert(Field(42.42)); + column_from->insert(Field("str2")); + column_from->insert(Field(42)); + column_from->insert(Field(43.43)); + column_from->insert(Field(Array({Field(41)}))); + column_from->insert(Field(43)); + column_from->insert(Field("str2")); + column_from->insert(Field(Array({Field(42)}))); + + auto column_to = ColumnDynamic::create(3); + column_to->insert(Field(42)); + + column_to->insertRangeFrom(*column_from, 0, 9); + ASSERT_EQ(column_to->getVariantInfo().variant_names.size(), 3); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 4); + auto field = (*column_to)[column_to->size() - 9]; + ASSERT_EQ(field, Field("str1")); + field = (*column_to)[column_to->size() - 8]; + ASSERT_EQ(field, Field(42.42)); + field = (*column_to)[column_to->size() - 7]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 6]; + ASSERT_EQ(field, Field(42)); + field = (*column_to)[column_to->size() - 5]; + ASSERT_EQ(field, Field(43.43)); + field = (*column_to)[column_to->size() - 4]; + ASSERT_EQ(field, Field(Array({Field(41)}))); + field = (*column_to)[column_to->size() - 3]; + ASSERT_EQ(field, Field(43)); + field = (*column_to)[column_to->size() - 2]; + ASSERT_EQ(field, Field("str2")); + field = (*column_to)[column_to->size() - 1]; + ASSERT_EQ(field, Field(Array({Field(42)}))); } TEST(ColumnDynamic, SerializeDeserializeFromArena1) @@ -583,18 +787,18 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) pos = column_to->deserializeAndInsertFromArena(pos); column_to->deserializeAndInsertFromArena(pos); - ASSERT_EQ((*column_from)[column_from->size() - 4], 42); - ASSERT_EQ((*column_from)[column_from->size() - 3], 42.42); - ASSERT_EQ((*column_from)[column_from->size() - 2], "str"); - ASSERT_EQ((*column_from)[column_from->size() - 1], Null()); - ASSERT_EQ(column_to->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, String)"); - std::vector expected_names = {"Float64", "Int8", "String"}; + ASSERT_EQ((*column_to)[column_to->size() - 4], 42); + ASSERT_EQ((*column_to)[column_to->size() - 3], 42.42); + ASSERT_EQ((*column_to)[column_to->size() - 2], "str"); + ASSERT_EQ((*column_to)[column_to->size() - 1], Null()); + ASSERT_EQ(column_to->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, SharedVariant, String)"); + std::vector expected_names = {"Float64", "Int8", "SharedVariant", "String"}; ASSERT_EQ(column_to->getVariantInfo().variant_names, expected_names); - std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"String", 2}}; + std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}; ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator == expected_variant_name_to_discriminator); } -TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow) +TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow1) { auto column_from = ColumnDynamic::create(255); column_from->insert(Field(42)); @@ -615,18 +819,56 @@ TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow) pos = column_to->deserializeAndInsertFromArena(pos); column_to->deserializeAndInsertFromArena(pos); - ASSERT_EQ((*column_from)[column_from->size() - 4], 42); - ASSERT_EQ((*column_from)[column_from->size() - 3], 42.42); - ASSERT_EQ((*column_from)[column_from->size() - 2], "str"); - ASSERT_EQ((*column_from)[column_from->size() - 1], Null()); + ASSERT_EQ((*column_to)[column_to->size() - 4], 42); + ASSERT_EQ((*column_to)[column_to->size() - 3], 42.42); + ASSERT_EQ((*column_to)[column_to->size() - 2], "str"); + ASSERT_EQ((*column_to)[column_to->size() - 1], Null()); ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); +} + +TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow2) +{ + auto column_from = ColumnDynamic::create(2); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + column_from->insert(Field("str")); + column_from->insert(Field(Null())); + column_from->insert(Field(Array({Field(42)}))); + + Arena arena; + const char * pos = nullptr; + auto ref1 = column_from->serializeValueIntoArena(0, arena, pos); + column_from->serializeValueIntoArena(1, arena, pos); + column_from->serializeValueIntoArena(2, arena, pos); + column_from->serializeValueIntoArena(3, arena, pos); + column_from->serializeValueIntoArena(4, arena, pos); + + auto column_to = ColumnDynamic::create(3); + column_to->insert(Field(42.42)); + pos = column_to->deserializeAndInsertFromArena(ref1.data); + pos = column_to->deserializeAndInsertFromArena(pos); + pos = column_to->deserializeAndInsertFromArena(pos); + pos = column_to->deserializeAndInsertFromArena(pos); + column_to->deserializeAndInsertFromArena(pos); + + ASSERT_EQ((*column_to)[column_to->size() - 5], 42); + ASSERT_EQ((*column_to)[column_to->size() - 4], 42.42); + ASSERT_EQ((*column_to)[column_to->size() - 3], "str"); + ASSERT_EQ((*column_to)[column_to->size() - 2], Null()); + ASSERT_EQ((*column_to)[column_to->size() - 1], Field(Array({Field(42)}))); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Int8")); + ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Float64")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("String")); + ASSERT_FALSE(column_to->getVariantInfo().variant_name_to_discriminator.contains("Array(Int8)")); + ASSERT_EQ(column_to->getSharedVariant().size(), 2); } TEST(ColumnDynamic, skipSerializedInArena) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(3); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -647,6 +889,34 @@ TEST(ColumnDynamic, skipSerializedInArena) pos = column_to->skipSerializedInArena(pos); ASSERT_EQ(pos, end); - ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator.empty()); - ASSERT_TRUE(column_to->getVariantInfo().variant_names.empty()); + ASSERT_EQ(column_to->getVariantInfo().variant_name_to_discriminator.at("SharedVariant"), 0); + ASSERT_EQ(column_to->getVariantInfo().variant_names, Names{"SharedVariant"}); +} + +TEST(ColumnDynamic, compare) +{ + auto column_from = ColumnDynamic::create(3); + column_from->insert(Field(42)); + column_from->insert(Field(42.42)); + column_from->insert(Field("str")); + column_from->insert(Field(Null())); + column_from->insert(Field(Array({Field(42)}))); + + ASSERT_EQ(column_from->compareAt(0, 0, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(0, 1, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(1, 1, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(0, 2, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(2, 0, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(2, 4, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(4, 2, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(4, 4, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(0, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(1, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(2, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(3, 3, *column_from, -1), 0); + ASSERT_EQ(column_from->compareAt(4, 3, *column_from, -1), 1); + ASSERT_EQ(column_from->compareAt(3, 0, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(3, 1, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(3, 2, *column_from, -1), -1); + ASSERT_EQ(column_from->compareAt(3, 4, *column_from, -1), -1); } diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index a1b1f8325f0..e00638a50ab 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include namespace DB { @@ -71,8 +73,8 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() == 0 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255"); + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS - 1) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 0 and 254"); return std::make_shared(literal->value.get()); } @@ -84,30 +86,72 @@ void registerDataTypeDynamic(DataTypeFactory & factory) std::unique_ptr DataTypeDynamic::getDynamicSubcolumnData(std::string_view subcolumn_name, const DB::IDataType::SubstreamData & data, bool throw_if_null) const { - auto [subcolumn_type_name, subcolumn_nested_name] = Nested::splitName(subcolumn_name); + auto [type_subcolumn_name, subcolumn_nested_name] = Nested::splitName(subcolumn_name); /// Check if requested subcolumn is a valid data type. - auto subcolumn_type = DataTypeFactory::instance().tryGet(String(subcolumn_type_name)); + auto subcolumn_type = DataTypeFactory::instance().tryGet(String(type_subcolumn_name)); if (!subcolumn_type) { if (throw_if_null) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Dynamic type doesn't have subcolumn '{}'", subcolumn_type_name); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Dynamic type doesn't have subcolumn '{}'", type_subcolumn_name); return nullptr; } std::unique_ptr res = std::make_unique(subcolumn_type->getDefaultSerialization()); res->type = subcolumn_type; std::optional discriminator; + ColumnPtr null_map_for_variant_from_shared_variant; if (data.column) { /// If column was provided, we should extract subcolumn from Dynamic column. const auto & dynamic_column = assert_cast(*data.column); const auto & variant_info = dynamic_column.getVariantInfo(); + const auto & variant_column = dynamic_column.getVariantColumn(); + const auto & shared_variant = dynamic_column.getSharedVariant(); /// Check if provided Dynamic column has subcolumn of this type. - auto it = variant_info.variant_name_to_discriminator.find(subcolumn_type->getName()); + String subcolumn_type_name = subcolumn_type->getName(); + auto it = variant_info.variant_name_to_discriminator.find(subcolumn_type_name); if (it != variant_info.variant_name_to_discriminator.end()) { discriminator = it->second; - res->column = dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(*discriminator); + res->column = variant_column.getVariantPtrByGlobalDiscriminator(*discriminator); + } + /// Otherwise if there is data in shared variant try to find requested type there. + else if (!shared_variant.empty()) + { + /// Create null map for resulting subcolumn to make it Nullable. + auto null_map_column = ColumnUInt8::create(); + NullMap & null_map = assert_cast(*null_map_column).getData(); + null_map.reserve(variant_column.size()); + auto subcolumn = subcolumn_type->createColumn(); + auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(dynamic_column.getSharedVariantDiscriminator()); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + const auto & offsets = variant_column.getOffsets(); + const FormatSettings format_settings; + for (size_t i = 0; i != local_discriminators.size(); ++i) + { + if (local_discriminators[i] == shared_variant_local_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + if (type->getName() == subcolumn_type_name) + { + dynamic_column.getVariantSerialization(subcolumn_type, subcolumn_type_name)->deserializeBinary(*subcolumn, buf, format_settings); + null_map.push_back(0); + } + else + { + null_map.push_back(1); + } + } + else + { + null_map.push_back(1); + } + } + + res->column = std::move(subcolumn); + null_map_for_variant_from_shared_variant = std::move(null_map_column); } } @@ -125,7 +169,7 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa return nullptr; } - res->serialization = std::make_shared(res->serialization, subcolumn_type->getName(), is_null_map_subcolumn); + res->serialization = std::make_shared(res->serialization, subcolumn_type->getName(), String(subcolumn_nested_name), is_null_map_subcolumn); /// Make resulting subcolumn Nullable only if type subcolumn can be inside Nullable or can be LowCardinality(Nullable()). bool make_subcolumn_nullable = subcolumn_type->canBeInsideNullable() || subcolumn_type->lowCardinality(); if (!is_null_map_subcolumn && make_subcolumn_nullable) @@ -133,10 +177,10 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa if (data.column) { + /// Check if provided Dynamic column has subcolumn of this type. In this case we should use VariantSubcolumnCreator/VariantNullMapSubcolumnCreator to + /// create full subcolumn from variant according to discriminators. if (discriminator) { - /// Provided Dynamic column has subcolumn of this type, we should use VariantSubcolumnCreator/VariantNullMapSubcolumnCreator to - /// create full subcolumn from variant according to discriminators. const auto & variant_column = assert_cast(*data.column).getVariantColumn(); std::unique_ptr creator; if (is_null_map_subcolumn) @@ -154,6 +198,21 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa make_subcolumn_nullable); res->column = creator->create(res->column); } + /// Check if requested type was extracted from shared variant. In this case we should use + /// VariantSubcolumnCreator to create full subcolumn from variant according to created null map. + else if (null_map_for_variant_from_shared_variant) + { + if (is_null_map_subcolumn) + { + res->column = null_map_for_variant_from_shared_variant; + } + else + { + SerializationVariantElement::VariantSubcolumnCreator creator( + null_map_for_variant_from_shared_variant, "", 0, 0, make_subcolumn_nullable, null_map_for_variant_from_shared_variant); + res->column = creator.create(res->column); + } + } /// Provided Dynamic column doesn't have subcolumn of this type, just create column filled with default values. else if (is_null_map_subcolumn) { diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 6f7dcd65b83..ca2ebdfbdbb 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -150,6 +150,12 @@ DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) cons return type; } +DataTypePtr DataTypeFactory::getCustom(const String & base_name, DataTypeCustomDescPtr customization) const +{ + auto type = get(base_name); + type->setCustomization(std::move(customization)); + return type; +} void DataTypeFactory::registerDataType(const String & family_name, Value creator, Case case_sensitiveness) { diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index edba9886d1c..a8324341691 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -34,6 +34,7 @@ public: DataTypePtr get(const String & family_name, const ASTPtr & parameters) const; DataTypePtr get(const ASTPtr & ast) const; DataTypePtr getCustom(DataTypeCustomDescPtr customization) const; + DataTypePtr getCustom(const String & base_name, DataTypeCustomDescPtr customization) const; /// Return nullptr in case of error. DataTypePtr tryGet(const String & full_name) const; diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index e96937d522d..67b4a0a5e31 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -192,17 +192,12 @@ MutableColumnPtr DataTypeTuple::createColumn() const MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serialization) const { - /// If we read Tuple as Variant subcolumn, it may be wrapped to SerializationVariantElement. - /// Here we don't need it, so we drop this wrapper. - const auto * current_serialization = &serialization; - while (const auto * serialization_variant_element = typeid_cast(current_serialization)) - current_serialization = serialization_variant_element->getNested().get(); - - /// If we read subcolumn of nested Tuple, it may be wrapped to SerializationNamed + /// If we read subcolumn of nested Tuple or this Tuple is a subcolumn, it may be wrapped to SerializationWrapper /// several times to allow to reconstruct the substream path name. /// Here we don't need substream path name, so we drop first several wrapper serializations. - while (const auto * serialization_named = typeid_cast(current_serialization)) - current_serialization = serialization_named->getNested().get(); + const auto * current_serialization = &serialization; + while (const auto * serialization_wrapper = dynamic_cast(current_serialization)) + current_serialization = serialization_wrapper->getNested().get(); const auto * serialization_tuple = typeid_cast(current_serialization); if (!serialization_tuple) diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index bd994e313ba..610f246265e 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -444,7 +444,7 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) case BinaryTypeIndex::Dynamic: { const auto & dynamic_type = assert_cast(*type); - /// Maximum number of dynamic types is 255, we can write it as 1 byte. + /// Maximum number of dynamic types is 254, we can write it as 1 byte. writeBinary(UInt8(dynamic_type.getMaxDynamicTypes()), buf); break; } diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 7609ffc91ca..67b29750948 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -27,15 +27,21 @@ namespace ErrorCodes struct SerializeBinaryBulkStateDynamic : public ISerialization::SerializeBinaryBulkState { SerializationDynamic::DynamicStructureSerializationVersion structure_version; + size_t max_dynamic_types; DataTypePtr variant_type; Names variant_names; SerializationPtr variant_serialization; ISerialization::SerializeBinaryBulkStatePtr variant_state; - /// Variants statistics. Map (Variant name) -> (Variant size). - ColumnDynamic::Statistics statistics = { .source = ColumnDynamic::Statistics::Source::READ, .data = {} }; + /// Variants statistics. + ColumnDynamic::Statistics statistics; + /// If true, statistics will be recalculated during serialization. + bool recalculate_statistics = false; - explicit SerializeBinaryBulkStateDynamic(UInt64 structure_version_) : structure_version(structure_version_) {} + explicit SerializeBinaryBulkStateDynamic(UInt64 structure_version_) + : structure_version(structure_version_), statistics(ColumnDynamic::Statistics::Source::READ) + { + } }; struct DeserializeBinaryBulkStateDynamic : public ISerialization::DeserializeBinaryBulkState @@ -106,20 +112,41 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( writeBinaryLittleEndian(structure_version, *stream); auto dynamic_state = std::make_shared(structure_version); + dynamic_state->max_dynamic_types = column_dynamic.getMaxDynamicTypes(); + /// Write max_dynamic_types parameter, because it can differ from the max_dynamic_types + /// that is specified in the Dynamic type (we could decrease it before merge). + writeBinaryLittleEndian(dynamic_state->max_dynamic_types, *stream); + dynamic_state->variant_type = variant_info.variant_type; dynamic_state->variant_names = variant_info.variant_names; const auto & variant_column = column_dynamic.getVariantColumn(); - /// Write internal Variant type name. + /// Write information about variants. + size_t num_variants = dynamic_state->variant_names.size() - 1; /// Don't write shared variant, Dynamic column should always have it. + writeBinaryLittleEndian(num_variants, *stream); if (settings.data_types_binary_encoding) - encodeDataType(dynamic_state->variant_type, *stream); + { + const auto & variants = assert_cast(*dynamic_state->variant_type).getVariants(); + for (const auto & variant: variants) + { + if (variant->getName() != ColumnDynamic::getSharedVariantTypeName()) + encodeDataType(dynamic_state->variant_type, *stream); + } + } else - writeStringBinary(dynamic_state->variant_type->getName(), *stream); + { + for (const auto & name : dynamic_state->variant_names) + { + if (name != ColumnDynamic::getSharedVariantTypeName()) + writeStringBinary(name, *stream); + } + } /// Write statistics in prefix if needed. if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::PREFIX) { const auto & statistics = column_dynamic.getStatistics(); + /// First, write statistics for usual variants. for (size_t i = 0; i != variant_info.variant_names.size(); ++i) { size_t size = 0; @@ -129,13 +156,55 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( /// - statistics read from the data part during deserialization of Dynamic column (Statistics::Source::READ). /// We can rely only on statistics calculated during the merge, because column with statistics that was read /// during deserialization from some data part could be filtered/limited/transformed/etc and so the statistics can be outdated. - if (!statistics.data.empty() && statistics.source == ColumnDynamic::Statistics::Source::MERGE) - size = statistics.data.at(variant_info.variant_names[i]); + if (statistics && statistics->source == ColumnDynamic::Statistics::Source::MERGE) + size = statistics->variants_statistics.at(variant_info.variant_names[i]); /// Otherwise we can use only variant sizes from current column. else size = variant_column.getVariantByGlobalDiscriminator(i).size(); writeVarUInt(size, *stream); } + + /// Second, write statistics for variants in shared variant. + /// Check if we have statistics calculated during merge of some data parts (Statistics::Source::MERGE). + if (statistics && statistics->source == ColumnDynamic::Statistics::Source::MERGE) + { + writeVarUInt(statistics->shared_variants_statistics.size(), *stream); + for (const auto & [variant_name, size] : statistics->shared_variants_statistics) + { + writeStringBinary(variant_name, *stream); + writeVarUInt(size, *stream); + } + } + /// If we don't have statistics for shared variants from merge, calculate it from the column. + else + { + std::unordered_map shared_variants_statistics; + const auto & shared_variant = column_dynamic.getSharedVariant(); + for (size_t i = 0; i != shared_variant.size(); ++i) + { + auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + if (auto it = shared_variants_statistics.find(type_name); it != shared_variants_statistics.end()) + ++it->second; + else if (shared_variants_statistics.size() < ColumnDynamic::Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) + shared_variants_statistics.emplace(type_name, 1); + } + + writeVarUInt(shared_variants_statistics.size(), *stream); + for (const auto & [variant_name, size] : shared_variants_statistics) + { + writeStringBinary(variant_name, *stream); + writeVarUInt(size, *stream); + } + } + } + /// Otherwise statistics will be written in the suffix, in this case we will recalculate + /// statistics during serialization to make it more precise. + else + { + dynamic_state->recalculate_statistics = true; } dynamic_state->variant_serialization = dynamic_state->variant_type->getDefaultSerialization(); @@ -182,33 +251,58 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD UInt64 structure_version; readBinaryLittleEndian(structure_version, *structure_stream); auto structure_state = std::make_shared(structure_version); - /// Read internal Variant type name. + /// Read max_dynamic_types parameter. + readBinaryLittleEndian(structure_state->max_dynamic_types, *structure_stream); + /// Read information about variants. + DataTypes variants; + size_t num_variants; + readBinaryLittleEndian(num_variants, *structure_stream); + variants.reserve(num_variants + 1); /// +1 for shared variant. if (settings.data_types_binary_encoding) { - structure_state->variant_type = decodeDataType(*structure_stream); + for (size_t i = 0; i != num_variants; ++i) + variants.push_back(decodeDataType(*structure_stream)); } else { String data_type_name; - readStringBinary(data_type_name, *structure_stream); - structure_state->variant_type = DataTypeFactory::instance().get(data_type_name); + for (size_t i = 0; i != num_variants; ++i) + { + readStringBinary(data_type_name, *structure_stream); + variants.push_back(DataTypeFactory::instance().get(data_type_name)); + } } - const auto * variant_type = typeid_cast(structure_state->variant_type.get()); - if (!variant_type) - throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect type of Dynamic nested column, expected Variant, got {}", structure_state->variant_type->getName()); + /// Add shared variant, Dynamic column should always have it. + variants.push_back(ColumnDynamic::getSharedVariantDataType()); + auto variant_type = std::make_shared(variants); /// Read statistics. if (settings.dynamic_read_statistics) { - const auto & variants = variant_type->getVariants(); + ColumnDynamic::Statistics statistics(ColumnDynamic::Statistics::Source::READ); + /// First, read statistics for usual variants. size_t variant_size; - for (const auto & variant : variants) + for (const auto & variant : variant_type->getVariants()) { readVarUInt(variant_size, *structure_stream); - structure_state->statistics.data[variant->getName()] = variant_size; + statistics.variants_statistics[variant->getName()] = variant_size; } + + /// Second, rend statistics for shared variants. + size_t statistics_size; + readVarUInt(statistics_size, *structure_stream); + String variant_name; + for (size_t i = 0; i != statistics_size; ++i) + { + readStringBinary(variant_name, *structure_stream); + readVarUInt(variant_size, *structure_stream); + statistics.shared_variants_statistics[variant_name] = variant_size; + } + + structure_state->statistics = std::make_shared(std::move(statistics)); } + structure_state->variant_type = std::move(variant_type); state = structure_state; addToSubstreamsDeserializeStatesCache(cache, settings.path, state); } @@ -231,8 +325,16 @@ void SerializationDynamic::serializeBinaryBulkStateSuffix( /// Write statistics in suffix if needed. if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::SUFFIX) { + /// First, write statistics for usual variants. for (const auto & variant_name : dynamic_state->variant_names) - writeVarUInt(dynamic_state->statistics.data[variant_name], *stream); + writeVarUInt(dynamic_state->statistics.variants_statistics[variant_name], *stream); + /// Second, write statistics for shared variants. + writeVarUInt(dynamic_state->statistics.shared_variants_statistics.size(), *stream); + for (const auto & [variant_name, size] : dynamic_state->statistics.shared_variants_statistics) + { + writeStringBinary(variant_name, *stream); + writeVarUInt(size, *stream); + } } settings.path.push_back(Substream::DynamicData); @@ -255,9 +357,42 @@ void SerializationDynamic::serializeBinaryBulkWithMultipleStreams( if (!variant_info.variant_type->equals(*dynamic_state->variant_type)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of internal columns of Dynamic. Expected: {}, Got: {}", dynamic_state->variant_type->getName(), variant_info.variant_type->getName()); + if (column_dynamic.getMaxDynamicTypes() != dynamic_state->max_dynamic_types) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of max_dynamic_types parameter of Dynamic. Expected: {}, Got: {}", dynamic_state->max_dynamic_types, column_dynamic.getMaxDynamicTypes()); + settings.path.push_back(Substream::DynamicData); - assert_cast(*dynamic_state->variant_serialization) - .serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(*variant_column, offset, limit, settings, dynamic_state->variant_state, dynamic_state->statistics.data); + if (dynamic_state->recalculate_statistics) + { + assert_cast(*dynamic_state->variant_serialization) + .serializeBinaryBulkWithMultipleStreamsAndUpdateVariantStatistics(*variant_column, offset, limit, settings, dynamic_state->variant_state, dynamic_state->statistics.variants_statistics); + /// Calculate statistics for shared variants. + const auto & shared_variant = column_dynamic.getSharedVariant(); + if (!shared_variant.empty()) + { + const auto & local_discriminators = variant_column->getLocalDiscriminators(); + const auto & offsets = variant_column->getOffsets(); + const auto shared_variant_discr = variant_column->localDiscriminatorByGlobal(column_dynamic.getSharedVariantDiscriminator()); + size_t end = limit == 0 || offset + limit > local_discriminators.size() ? local_discriminators.size() : offset + limit; + for (size_t i = offset; i != end; ++i) + { + if (local_discriminators[i] == shared_variant_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + if (auto it = dynamic_state->statistics.shared_variants_statistics.find(type_name); it != dynamic_state->statistics.shared_variants_statistics.end()) + ++it->second; + else if (dynamic_state->statistics.shared_variants_statistics.size() < ColumnDynamic::Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) + dynamic_state->statistics.shared_variants_statistics.emplace(type_name, 1); + } + } + } + } + else + { + assert_cast(*dynamic_state->variant_serialization).serializeBinaryBulkWithMultipleStreams(*variant_column, offset, limit, settings, dynamic_state->variant_state); + } settings.path.pop_back(); } @@ -272,13 +407,17 @@ void SerializationDynamic::deserializeBinaryBulkWithMultipleStreams( return; auto mutable_column = column->assumeMutable(); + auto & column_dynamic = assert_cast(*mutable_column); auto * dynamic_state = checkAndGetState(state); auto * structure_state = checkAndGetState(dynamic_state->structure_state); if (mutable_column->empty()) - mutable_column = ColumnDynamic::create(structure_state->variant_type->createColumn(), structure_state->variant_type, max_dynamic_types, structure_state->statistics); + { + column_dynamic.setMaxDynamicPaths(structure_state->max_dynamic_types); + column_dynamic.setVariantType(structure_state->variant_type); + column_dynamic.setStatistics(structure_state->statistics); + } - auto & column_dynamic = assert_cast(*mutable_column); const auto & variant_info = column_dynamic.getVariantInfo(); if (!variant_info.variant_type->equals(*structure_state->variant_type)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of internal columns of Dynamic. Expected: {}, Got: {}", structure_state->variant_type->getName(), variant_info.variant_type->getName()); @@ -329,24 +468,42 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu encodeDataType(std::make_shared(), ostr); return; } + /// Check if this value is in shared variant. In this case it's already + /// in desired binary format. + else if (global_discr == dynamic_column.getSharedVariantDiscriminator()) + { + auto value = dynamic_column.getSharedVariant().getDataAt(variant_column.offsetAt(row_num)); + ostr.write(value.data, value.size); + return; + } const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(global_discr); + const auto & variant_type_name = variant_info.variant_names[global_discr]; encodeDataType(variant_type, ostr); - variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); + dynamic_column.getVariantSerialization(variant_type, variant_type_name)->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); } -template -static void deserializeVariant( +template +static ReturnType deserializeVariant( ColumnVariant & variant_column, - const DataTypePtr & variant_type, + const SerializationPtr & variant_serialization, ColumnVariant::Discriminator global_discr, ReadBuffer & istr, DeserializeFunc deserialize) { auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discr); - deserialize(*variant_type->getDefaultSerialization(), variant, istr); + if constexpr (std::is_same_v) + { + if (!deserialize(*variant_serialization, variant, istr)) + return ReturnType(false); + } + else + { + deserialize(*variant_serialization, variant, istr); + } variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(global_discr)); variant_column.getOffsets().push_back(variant.size() - 1); + return ReturnType(true); } void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -360,11 +517,12 @@ void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr } auto variant_type_name = variant_type->getName(); + const auto & variant_serialization = dynamic_column.getVariantSerialization(variant_type, variant_type_name); const auto & variant_info = dynamic_column.getVariantInfo(); auto it = variant_info.variant_name_to_discriminator.find(variant_type_name); if (it != variant_info.variant_name_to_discriminator.end()) { - deserializeVariant(dynamic_column.getVariantColumn(), variant_type, it->second, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); + deserializeVariant(dynamic_column.getVariantColumn(), variant_serialization, it->second, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); return; } @@ -372,25 +530,15 @@ void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr if (dynamic_column.addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator.at(variant_type_name); - deserializeVariant(dynamic_column.getVariantColumn(), variant_type, discr, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); + deserializeVariant(dynamic_column.getVariantColumn(), variant_serialization, discr, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); return; } /// We reached maximum number of variants and couldn't add new variant. - /// This case should be really rare in real use cases. - /// We should always be able to add String variant and insert value as String. - dynamic_column.addStringVariant(); + /// In this case we insert this value into shared variant in binary form. auto tmp_variant_column = variant_type->createColumn(); - variant_type->getDefaultSerialization()->deserializeBinary(*tmp_variant_column, istr, settings); - auto string_column = castColumn(ColumnWithTypeAndName(tmp_variant_column->getPtr(), variant_type, ""), std::make_shared()); - auto & variant_column = dynamic_column.getVariantColumn(); - variant_column.insertIntoVariantFrom(variant_info.variant_name_to_discriminator.at("String"), *string_column, 0); -} - -void SerializationDynamic::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextCSV(dynamic_column.getVariantColumn(), row_num, ostr, settings); + variant_serialization->deserializeBinary(*tmp_variant_column, istr, settings); + dynamic_column.insertValueIntoSharedVariant(*tmp_variant_column, variant_type, variant_type_name, 0); } template @@ -406,6 +554,7 @@ static void deserializeTextImpl( auto & dynamic_column = assert_cast(column); auto & variant_column = dynamic_column.getVariantColumn(); const auto & variant_info = dynamic_column.getVariantInfo(); + const auto & variant_types = assert_cast(*variant_info.variant_type).getVariants(); String field = read_field(istr); auto field_buf = std::make_unique(field); JSONInferenceInfo json_info; @@ -413,27 +562,81 @@ static void deserializeTextImpl( if (escaping_rule == FormatSettings::EscapingRule::JSON) transformFinalInferredJSONTypeIfNeeded(variant_type, settings, &json_info); - if (checkIfTypeIsComplete(variant_type) && dynamic_column.addNewVariant(variant_type)) + /// If inferred type is not complete, we cannot add it as a new variant. + /// Let's try to deserialize this field into existing variants. + /// If failed, insert this value as String. + if (!checkIfTypeIsComplete(variant_type)) + { + size_t shared_variant_discr = dynamic_column.getSharedVariantDiscriminator(); + for (size_t i = 0; i != variant_types.size(); ++i) + { + field_buf = std::make_unique(field); + if (i != shared_variant_discr + && deserializeVariant( + variant_column, + dynamic_column.getVariantSerialization(variant_types[i], variant_info.variant_names[i]), + i, + *field_buf, + try_deserialize_variant)) + return; + } + + variant_type = std::make_shared(); + /// To be able to deserialize field as String with Quoted escaping rule, it should be quoted. + if (escaping_rule == FormatSettings::EscapingRule::Quoted && (field.size() < 2 || field.front() != '\'' || field.back() != '\'')) + field = "'" + field + "'"; + } + else if (dynamic_column.addNewVariant(variant_type, variant_type->getName())) { auto discr = variant_info.variant_name_to_discriminator.at(variant_type->getName()); - deserializeVariant(dynamic_column.getVariantColumn(), variant_type, discr, *field_buf, deserialize_variant); + deserializeVariant(dynamic_column.getVariantColumn(), dynamic_column.getVariantSerialization(variant_type), discr, *field_buf, deserialize_variant); return; } - /// We couldn't infer type or add new variant. Try to insert field into current variants. + /// We couldn't infer type or add new variant. Insert it into shared variant. + auto tmp_variant_column = variant_type->createColumn(); field_buf = std::make_unique(field); - if (try_deserialize_variant(*variant_info.variant_type->getDefaultSerialization(), variant_column, *field_buf)) - return; + auto variant_type_name = variant_type->getName(); + deserialize_variant(*dynamic_column.getVariantSerialization(variant_type, variant_type_name), *tmp_variant_column, *field_buf); + dynamic_column.insertValueIntoSharedVariant(*tmp_variant_column, variant_type, variant_type_name, 0); +} - /// We couldn't insert field into any existing variant, add String variant and read value as String. - dynamic_column.addStringVariant(); +template +static void serializeTextImpl( + const IColumn & column, + size_t row_num, + WriteBuffer & ostr, + const FormatSettings & settings, + NestedSerialize nested_serialize) +{ + const auto & dynamic_column = assert_cast(column); + const auto & variant_column = dynamic_column.getVariantColumn(); + /// Check if this row has value in shared variant. In this case we should first deserialize it from binary format. + if (variant_column.globalDiscriminatorAt(row_num) == dynamic_column.getSharedVariantDiscriminator()) + { + auto value = dynamic_column.getSharedVariant().getDataAt(variant_column.offsetAt(row_num)); + ReadBufferFromMemory buf(value.data, value.size); + auto variant_type = decodeDataType(buf); + auto tmp_variant_column = variant_type->createColumn(); + auto variant_serialization = dynamic_column.getVariantSerialization(variant_type); + variant_serialization->deserializeBinary(*tmp_variant_column, buf, settings); + nested_serialize(*variant_serialization, *tmp_variant_column, 0, ostr); + } + /// Otherwise just use serialization for Variant. + else + { + nested_serialize(*dynamic_column.getVariantInfo().variant_type->getDefaultSerialization(), variant_column, row_num, ostr); + } +} - if (escaping_rule == FormatSettings::EscapingRule::Quoted && (field.size() < 2 || field.front() != '\'' || field.back() != '\'')) - field = "'" + field + "'"; +void SerializationDynamic::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextCSV(col, row, buf, settings); + }; - field_buf = std::make_unique(field); - auto string_discr = variant_info.variant_name_to_discriminator.at("String"); - deserializeVariant(dynamic_column.getVariantColumn(), std::make_shared(), string_discr, *field_buf, deserialize_variant); + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -466,8 +669,12 @@ bool SerializationDynamic::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadB void SerializationDynamic::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextEscaped(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextEscaped(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -500,8 +707,12 @@ bool SerializationDynamic::tryDeserializeTextEscaped(DB::IColumn & column, DB::R void SerializationDynamic::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextQuoted(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextQuoted(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -534,8 +745,12 @@ bool SerializationDynamic::tryDeserializeTextQuoted(DB::IColumn & column, DB::Re void SerializationDynamic::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextJSON(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextJSON(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -568,8 +783,12 @@ bool SerializationDynamic::tryDeserializeTextJSON(DB::IColumn & column, DB::Read void SerializationDynamic::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextRaw(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextRaw(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -602,8 +821,12 @@ bool SerializationDynamic::tryDeserializeTextRaw(DB::IColumn & column, DB::ReadB void SerializationDynamic::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeText(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeText(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -636,8 +859,12 @@ bool SerializationDynamic::tryDeserializeWholeText(DB::IColumn & column, DB::Rea void SerializationDynamic::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextXML(dynamic_column.getVariantColumn(), row_num, ostr, settings); + auto nested_serialize = [&settings](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextXML(col, row, buf, settings); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } } diff --git a/src/DataTypes/Serializations/SerializationDynamic.h b/src/DataTypes/Serializations/SerializationDynamic.h index 001a3cf87ce..3dbf311fb6c 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.h +++ b/src/DataTypes/Serializations/SerializationDynamic.h @@ -105,9 +105,13 @@ private: { DynamicStructureSerializationVersion structure_version; DataTypePtr variant_type; - ColumnDynamic::Statistics statistics = {.source = ColumnDynamic::Statistics::Source::READ, .data = {}}; + size_t max_dynamic_types; + ColumnDynamic::StatisticsPtr statistics; - explicit DeserializeBinaryBulkStateDynamicStructure(UInt64 structure_version_) : structure_version(structure_version_) {} + explicit DeserializeBinaryBulkStateDynamicStructure(UInt64 structure_version_) + : structure_version(structure_version_) + { + } }; size_t max_dynamic_types; diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.cpp b/src/DataTypes/Serializations/SerializationDynamicElement.cpp index 211f0ac9377..cffca14bca5 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.cpp +++ b/src/DataTypes/Serializations/SerializationDynamicElement.cpp @@ -4,7 +4,10 @@ #include #include #include +#include #include +#include +#include #include namespace DB @@ -21,6 +24,8 @@ struct DeserializeBinaryBulkStateDynamicElement : public ISerialization::Deseria ISerialization::DeserializeBinaryBulkStatePtr structure_state; SerializationPtr variant_serialization; ISerialization::DeserializeBinaryBulkStatePtr variant_element_state; + bool read_from_shared_variant; + ColumnPtr shared_variant; }; void SerializationDynamicElement::enumerateStreams( @@ -73,9 +78,10 @@ void SerializationDynamicElement::deserializeBinaryBulkStatePrefix( auto dynamic_element_state = std::make_shared(); dynamic_element_state->structure_state = std::move(structure_state); - const auto & variant_type = checkAndGetState(dynamic_element_state->structure_state)->variant_type; + const auto & variant_type = assert_cast( + *checkAndGetState(dynamic_element_state->structure_state)->variant_type); /// Check if we actually have required element in the Variant. - if (auto global_discr = assert_cast(*variant_type).tryGetVariantDiscriminator(dynamic_element_name)) + if (auto global_discr = variant_type.tryGetVariantDiscriminator(dynamic_element_name)) { settings.path.push_back(Substream::DynamicData); if (is_null_map_subcolumn) @@ -83,6 +89,21 @@ void SerializationDynamicElement::deserializeBinaryBulkStatePrefix( else dynamic_element_state->variant_serialization = std::make_shared(nested_serialization, dynamic_element_name, *global_discr); dynamic_element_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_element_state->variant_element_state, cache); + dynamic_element_state->read_from_shared_variant = false; + settings.path.pop_back(); + } + /// If we don't have this element in the Variant, we will read shared variant and try to find it there. + else + { + auto shared_variant_global_discr = variant_type.tryGetVariantDiscriminator(ColumnDynamic::getSharedVariantTypeName()); + chassert(shared_variant_global_discr.has_value()); + settings.path.push_back(Substream::DynamicData); + dynamic_element_state->variant_serialization = std::make_shared( + ColumnDynamic::getSharedVariantDataType()->getDefaultSerialization(), + ColumnDynamic::getSharedVariantTypeName(), + *shared_variant_global_discr); + dynamic_element_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_element_state->variant_element_state, cache); + dynamic_element_state->read_from_shared_variant = true; settings.path.pop_back(); } @@ -115,23 +136,103 @@ void SerializationDynamicElement::deserializeBinaryBulkWithMultipleStreams( auto * dynamic_element_state = checkAndGetState(state); - if (dynamic_element_state->variant_serialization) + /// Check if this subcolumn should not be read from shared variant. + /// In this case just read data from the corresponding variant. + if (!dynamic_element_state->read_from_shared_variant) { settings.path.push_back(Substream::DynamicData); - dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams(result_column, limit, settings, dynamic_element_state->variant_element_state, cache); + dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams( + result_column, limit, settings, dynamic_element_state->variant_element_state, cache); settings.path.pop_back(); } - else if (is_null_map_subcolumn) - { - auto mutable_column = result_column->assumeMutable(); - auto & data = assert_cast(*mutable_column).getData(); - data.resize_fill(data.size() + limit, 1); - } + /// Otherwise, read the shared variant column and extract requested type from it. else { - auto mutable_column = result_column->assumeMutable(); - mutable_column->insertManyDefaults(limit); - result_column = std::move(mutable_column); + settings.path.push_back(Substream::DynamicData); + /// Initialize shared_variant column if needed. + if (result_column->empty()) + dynamic_element_state->shared_variant = makeNullable(ColumnDynamic::getSharedVariantDataType()->createColumn()); + size_t prev_size = result_column->size(); + dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams( + dynamic_element_state->shared_variant, limit, settings, dynamic_element_state->variant_element_state, cache); + settings.path.pop_back(); + + /// If we need to read a subcolumn from variant column, create an empty variant column, fill it and extract subcolumn. + auto variant_type = DataTypeFactory::instance().get(dynamic_element_name); + auto result_type = makeNullableOrLowCardinalityNullableSafe(variant_type); + MutableColumnPtr variant_column = nested_subcolumn.empty() || is_null_map_subcolumn ? result_column->assumeMutable() : result_type->createColumn(); + variant_column->reserve(variant_column->size() + limit); + MutableColumnPtr non_nullable_variant_column = variant_column->assumeMutable(); + NullMap * null_map = nullptr; + bool is_low_cardinality_nullable = isColumnLowCardinalityNullable(*variant_column); + /// Resulting subolumn can be Nullable, but value is serialized in shared variant as non-Nullable. + /// Extract non-nullable column and remember the null map to fill it during deserialization. + if (isColumnNullable(*variant_column)) + { + auto & nullable_variant_column = assert_cast(*variant_column); + non_nullable_variant_column = nullable_variant_column.getNestedColumnPtr()->assumeMutable(); + null_map = &nullable_variant_column.getNullMapData(); + } + else if (is_null_map_subcolumn) + { + null_map = &assert_cast(*variant_column).getData(); + } + + auto variant_serialization = variant_type->getDefaultSerialization(); + + const auto & nullable_shared_variant = assert_cast(*dynamic_element_state->shared_variant); + const auto & shared_null_map = nullable_shared_variant.getNullMapData(); + const auto & shared_variant = assert_cast(nullable_shared_variant.getNestedColumn()); + const FormatSettings format_settings; + for (size_t i = prev_size; i != shared_variant.size(); ++i) + { + if (!shared_null_map[i]) + { + auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + if (type->getName() == dynamic_element_name) + { + /// When requested type is LowCardinality the subcolumn type name will be LowCardinality(Nullable). + /// Value in shared variant is serialized as LowCardinality and we cannot simply deserialize it + /// inside LowCardinality(Nullable) column (it will try to deserialize null bit). In this case we + /// have to create temporary LowCardinality column, deserialize value into it and insert it into + /// resulting LowCardinality(Nullable) (insertion from LowCardinality column to LowCardinality(Nullable) + /// column is allowed). + if (is_low_cardinality_nullable) + { + auto tmp_column = variant_type->createColumn(); + variant_serialization->deserializeBinary(*tmp_column, buf, format_settings); + non_nullable_variant_column->insertFrom(*tmp_column, 0); + } + else if (is_null_map_subcolumn) + { + null_map->push_back(0); + } + else + { + variant_serialization->deserializeBinary(*non_nullable_variant_column, buf, format_settings); + if (null_map) + null_map->push_back(0); + } + } + else + { + variant_column->insertDefault(); + } + } + else + { + variant_column->insertDefault(); + } + } + + /// Extract nested subcolumn if needed. + if (!nested_subcolumn.empty() && !is_null_map_subcolumn) + { + auto subcolumn = result_type->getSubcolumn(nested_subcolumn, variant_column->getPtr()); + result_column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size()); + } } } diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.h b/src/DataTypes/Serializations/SerializationDynamicElement.h index 127d14a55e0..c674cf479ae 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.h +++ b/src/DataTypes/Serializations/SerializationDynamicElement.h @@ -13,11 +13,15 @@ private: /// To be able to deserialize Dynamic element as a subcolumn /// we need its type name and global discriminator. String dynamic_element_name; + /// Nested subcolumn of a type dynamic type. For example, for `Tuple(a UInt32)`.a + /// subcolumn dynamic_element_name = 'Tuple(a UInt32)' and nested_subcolumn = 'a'. + /// Needed to extract nested subcolumn from values in shared variant. + String nested_subcolumn; bool is_null_map_subcolumn; public: - SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_, bool is_null_map_subcolumn_ = false) - : SerializationWrapper(nested_), dynamic_element_name(dynamic_element_name_), is_null_map_subcolumn(is_null_map_subcolumn_) + SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_, const String & nested_subcolumn_, bool is_null_map_subcolumn_ = false) + : SerializationWrapper(nested_), dynamic_element_name(dynamic_element_name_), nested_subcolumn(nested_subcolumn_), is_null_map_subcolumn(is_null_map_subcolumn_) { } diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 03b5d9584e0..36dc85f60ee 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -305,8 +305,10 @@ SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( const String & variant_element_name_, ColumnVariant::Discriminator global_variant_discriminator_, ColumnVariant::Discriminator local_variant_discriminator_, - bool make_nullable_) + bool make_nullable_, + const ColumnPtr & null_map_) : local_discriminators(local_discriminators_) + , null_map(null_map_) , variant_element_name(variant_element_name_) , global_variant_discriminator(global_variant_discriminator_) , local_variant_discriminator(local_variant_discriminator_) @@ -314,12 +316,13 @@ SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( { } -DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::DataTypePtr & prev) const + +DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DataTypePtr & prev) const { return make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev) : prev; } -SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::SerializationPtr & prev) const +SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const SerializationPtr & prev) const { return std::make_shared(prev, variant_element_name, global_variant_discriminator); } @@ -339,12 +342,16 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: return res; } - /// In general case we should iterate through discriminators and create null-map for our variant. - NullMap null_map; - null_map.reserve(local_discriminators->size()); - const auto & local_discriminators_data = assert_cast(*local_discriminators).getData(); - for (auto local_discr : local_discriminators_data) - null_map.push_back(local_discr != local_variant_discriminator); + /// In general case we should iterate through discriminators and create null-map for our variant if we don't already have it. + std::optional null_map_from_discriminators; + if (!null_map) + { + null_map_from_discriminators = NullMap(); + null_map_from_discriminators->reserve(local_discriminators->size()); + const auto & local_discriminators_data = assert_cast(*local_discriminators).getData(); + for (auto local_discr : local_discriminators_data) + null_map_from_discriminators->push_back(local_discr != local_variant_discriminator); + } /// Now we can create new column from null-map and variant column using IColumn::expand. auto res_column = IColumn::mutate(prev); @@ -356,13 +363,21 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: if (make_nullable && prev->lowCardinality()) res_column = assert_cast(*res_column).cloneNullable(); - res_column->expand(null_map, /*inverted = */ true); + if (null_map_from_discriminators) + res_column->expand(*null_map_from_discriminators, /*inverted = */ true); + else + res_column->expand(assert_cast(*null_map).getData(), /*inverted = */ true); if (make_nullable && prev->canBeInsideNullable()) { - auto null_map_col = ColumnUInt8::create(); - null_map_col->getData() = std::move(null_map); - return ColumnNullable::create(std::move(res_column), std::move(null_map_col)); + if (null_map_from_discriminators) + { + auto null_map_col = ColumnUInt8::create(); + null_map_col->getData() = std::move(*null_map_from_discriminators); + return ColumnNullable::create(std::move(res_column), std::move(null_map_col)); + } + + return ColumnNullable::create(std::move(res_column), null_map->assumeMutable()); } return res_column; diff --git a/src/DataTypes/Serializations/SerializationVariantElement.h b/src/DataTypes/Serializations/SerializationVariantElement.h index 69101aea0f5..64f86eb2190 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.h +++ b/src/DataTypes/Serializations/SerializationVariantElement.h @@ -63,18 +63,22 @@ public: struct VariantSubcolumnCreator : public ISubcolumnCreator { + private: const ColumnPtr local_discriminators; + const ColumnPtr null_map; /// optional const String variant_element_name; const ColumnVariant::Discriminator global_variant_discriminator; const ColumnVariant::Discriminator local_variant_discriminator; bool make_nullable; + public: VariantSubcolumnCreator( const ColumnPtr & local_discriminators_, const String & variant_element_name_, ColumnVariant::Discriminator global_variant_discriminator_, ColumnVariant::Discriminator local_variant_discriminator_, - bool make_nullable_); + bool make_nullable_, + const ColumnPtr & null_map_ = nullptr); DataTypePtr create(const DataTypePtr & prev) const override; ColumnPtr create(const ColumnPtr & prev) const override; diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 242d2dc9f80..86fde3852b8 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1362,13 +1362,14 @@ public: } auto & variant_column = column_dynamic.getVariantColumn(); - auto variant_info = column_dynamic.getVariantInfo(); + const auto & variant_info = column_dynamic.getVariantInfo(); /// Second, infer ClickHouse type for this element and add it as a new variant. auto element_type = elementToDataType(element, format_settings); - if (column_dynamic.addNewVariant(element_type)) + auto element_type_name = element_type->getName(); + if (column_dynamic.addNewVariant(element_type, element_type_name)) { auto node = buildJSONExtractTree(element_type, "Dynamic inference"); - auto global_discriminator = variant_info.variant_name_to_discriminator[element_type->getName()]; + auto global_discriminator = variant_info.variant_name_to_discriminator.at(element_type_name); auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discriminator); if (!node->insertResultToColumn(variant, element, insert_settings, format_settings, error)) return false; @@ -1377,29 +1378,15 @@ public: return true; } - /// We couldn't add new variant. Try to insert element into current variants. - auto variant_node = buildJSONExtractTree(variant_info.variant_type, "Dynamic inference"); - if (variant_node->insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) - return true; - - /// We couldn't insert element into any existing variant, add String variant and read value as String. - column_dynamic.addStringVariant(); - auto string_global_discriminator = variant_info.variant_name_to_discriminator["String"]; - auto & string_column = variant_column.getVariantByGlobalDiscriminator(string_global_discriminator); - if (!getStringNode()->insertResultToColumn(string_column, element, insert_settings, format_settings, error)) + /// We couldn't add this variant, insert it into shared variant. + auto tmp_variant_column = element_type->createColumn(); + auto node = buildJSONExtractTree(element_type, "Dynamic inference"); + if (!node->insertResultToColumn(*tmp_variant_column, element, insert_settings, format_settings, error)) return false; - variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(string_global_discriminator)); - variant_column.getOffsets().push_back(string_column.size() - 1); + column_dynamic.insertValueIntoSharedVariant(*tmp_variant_column, element_type, element_type_name, 0); return true; } - static const std::unique_ptr> & getStringNode() - { - static const std::unique_ptr> string_node - = buildJSONExtractTree(std::make_shared(), "Dynamic inference"); - return string_node; - } - static DataTypePtr elementToDataType(const typename JSONParser::Element & element, const FormatSettings & format_settings) { JSONInferenceInfo json_inference_info; diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..21b98cf505c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include #include @@ -4287,13 +4288,98 @@ private: WrapperType createDynamicToColumnWrapper(const DataTypePtr &) const { return [this] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr { + /// When casting Dynamic to regular column we should cast all variants from current Dynamic column + /// and construct the result based on discriminators. const auto & column_dynamic = assert_cast(*arguments.front().column.get()); + const auto & variant_column = column_dynamic.getVariantColumn(); const auto & variant_info = column_dynamic.getVariantInfo(); - auto variant_wrapper = createVariantToColumnWrapper(assert_cast(*variant_info.variant_type), result_type); - ColumnsWithTypeAndName args = {ColumnWithTypeAndName(column_dynamic.getVariantColumnPtr(), variant_info.variant_type, "")}; - return variant_wrapper(args, result_type, col_nullable, input_rows_count); + + /// First, cast usual variants to result type. + const auto & variant_types = assert_cast(*variant_info.variant_type).getVariants(); + std::vector casted_variant_columns; + casted_variant_columns.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i); + ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}}; + auto variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); + casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + } + + /// Second, collect all variants stored in shared variant and cast them to result type. + std::vector variant_columns_from_shared_variant; + DataTypes variant_types_from_shared_variant; + /// We will need to know what variant to use when we see discriminator of a shared variant. + /// To do it, we remember what variant was extracted from each row and what was it's offset. + PaddedPODArray shared_variant_indexes; + PaddedPODArray shared_variant_offsets; + std::unordered_map shared_variant_to_index; + const auto & shared_variant = column_dynamic.getSharedVariant(); + const auto shared_variant_discr = column_dynamic.getSharedVariantDiscriminator(); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + const auto & offsets = variant_column.getOffsets(); + if (!shared_variant.empty()) + { + shared_variant_indexes.reserve(input_rows_count); + shared_variant_offsets.reserve(input_rows_count); + FormatSettings format_settings; + const auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(shared_variant_discr); + for (size_t i = 0; i != input_rows_count; ++i) + { + if (local_discriminators[i] == shared_variant_local_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + auto it = shared_variant_to_index.find(type_name); + /// Check if didn't created column for this variant yet. + if (it == shared_variant_to_index.end()) + { + it = shared_variant_to_index.emplace(type_name, variant_columns_from_shared_variant.size()).first; + variant_columns_from_shared_variant.push_back(type->createColumn()); + variant_types_from_shared_variant.push_back(type); + } + + shared_variant_indexes.push_back(it->second); + shared_variant_offsets.push_back(variant_columns_from_shared_variant[it->second]->size()); + type->getDefaultSerialization()->deserializeBinary(*variant_columns_from_shared_variant[it->second], buf, format_settings); + } + else + { + shared_variant_indexes.emplace_back(); + shared_variant_offsets.emplace_back(); + } + } + } + + /// Cast all extracted variants into result type. + std::vector casted_shared_variant_columns; + casted_shared_variant_columns.reserve(variant_types_from_shared_variant.size()); + for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i) + { + ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}}; + auto variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); + casted_shared_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size())); + } + + /// Construct result column from all casted variants. + auto res = result_type->createColumn(); + res->reserve(input_rows_count); + for (size_t i = 0; i != input_rows_count; ++i) + { + auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + res->insertDefault(); + else if (global_discr == shared_variant_discr) + res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + else + res->insertFrom(*casted_variant_columns[global_discr], offsets[i]); + } + + return res; }; } @@ -4320,200 +4406,51 @@ private: }; } - std::pair getReducedVariant( - const ColumnVariant & variant_column, - const DataTypePtr & variant_type, - const std::unordered_map & variant_name_to_discriminator, - size_t max_result_num_variants, - const ColumnDynamic::Statistics & statistics = {}) const + WrapperType createVariantToDynamicWrapper(const DataTypeVariant & from_variant_type, const DataTypeDynamic & dynamic_type) const { - const auto & variant_types = assert_cast(*variant_type).getVariants(); - /// First check if we don't exceed the limit in current Variant column. - if (variant_types.size() < max_result_num_variants || (variant_types.size() == max_result_num_variants && variant_name_to_discriminator.contains("String"))) - return {variant_column.getPtr(), variant_type}; - - /// We want to keep the most frequent variants and convert to string the rarest. - std::vector> variant_sizes; - variant_sizes.reserve(variant_types.size()); - std::optional old_string_discriminator; - /// List of variants that should be converted to a single String variant. - std::vector variants_to_convert_to_string; - for (size_t i = 0; i != variant_types.size(); ++i) + /// First create extended Variant with shared variant type and cast this Variant to it. + auto variants_for_dynamic = from_variant_type.getVariants(); + size_t number_of_variants = variants_for_dynamic.size(); + variants_for_dynamic.push_back(ColumnDynamic::getSharedVariantDataType()); + const auto & variant_type_for_dynamic = std::make_shared(variants_for_dynamic); + auto old_to_new_variant_wrapper = createVariantToVariantWrapper(from_variant_type, *variant_type_for_dynamic); + auto max_dynamic_types = dynamic_type.getMaxDynamicTypes(); + return [old_to_new_variant_wrapper, variant_type_for_dynamic, number_of_variants, max_dynamic_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr { - /// String variant won't be removed. - String variant_name = variant_types[i]->getName(); + auto variant_column_for_dynamic = old_to_new_variant_wrapper(arguments, result_type, col_nullable, input_rows_count); + /// If resulting Dynamic column can contain all variants from this Variant column, just create Dynamic column from it. + if (max_dynamic_types >= number_of_variants) + return ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, max_dynamic_types, max_dynamic_types); - if (variant_name == "String") - { - old_string_discriminator = i; - /// For simplicity, add this variant to the list that will be converted to string, - /// so we will process it with other variants when constructing the new String variant. - variants_to_convert_to_string.push_back(i); - } - else - { - size_t size = 0; - if (statistics.data.empty()) - size = variant_column.getVariantByGlobalDiscriminator(i).size(); - else - size = statistics.data.at(variant_name); - variant_sizes.emplace_back(size, i); - } - } - - /// Sort variants by sizes, so we will keep the most frequent. - std::sort(variant_sizes.begin(), variant_sizes.end(), std::greater()); - - DataTypes remaining_variants; - remaining_variants.reserve(max_result_num_variants); - /// Add String variant in advance. - remaining_variants.push_back(std::make_shared()); - for (auto [_, discr] : variant_sizes) - { - if (remaining_variants.size() != max_result_num_variants) - remaining_variants.push_back(variant_types[discr]); - else - variants_to_convert_to_string.push_back(discr); - } - - auto reduced_variant = std::make_shared(remaining_variants); - const auto & new_variants = reduced_variant->getVariants(); - /// To construct reduced variant column we will need mapping from old to new discriminators. - std::vector old_to_new_discriminators_mapping; - old_to_new_discriminators_mapping.resize(variant_types.size()); - ColumnVariant::Discriminator string_variant_discriminator = 0; - for (size_t i = 0; i != new_variants.size(); ++i) - { - String variant_name = new_variants[i]->getName(); - if (variant_name == "String") - { - string_variant_discriminator = i; - for (auto discr : variants_to_convert_to_string) - old_to_new_discriminators_mapping[discr] = i; - } - else - { - auto old_discr = variant_name_to_discriminator.at(variant_name); - old_to_new_discriminators_mapping[old_discr] = i; - } - } - - /// Convert all reduced variants to String. - std::unordered_map variants_converted_to_string; - variants_converted_to_string.reserve(variants_to_convert_to_string.size()); - size_t string_variant_size = 0; - for (auto discr : variants_to_convert_to_string) - { - auto string_type = std::make_shared(); - auto string_wrapper = prepareUnpackDictionaries(variant_types[discr], string_type); - auto column_to_convert = ColumnWithTypeAndName(variant_column.getVariantPtrByGlobalDiscriminator(discr), variant_types[discr], ""); - ColumnsWithTypeAndName args = {column_to_convert}; - auto variant_string_column = string_wrapper(args, string_type, nullptr, column_to_convert.column->size()); - string_variant_size += variant_string_column->size(); - variants_converted_to_string[discr] = variant_string_column; - } - - /// Create new discriminators and offsets and fill new String variant according to old discriminators. - auto string_variant = ColumnString::create(); - string_variant->reserve(string_variant_size); - auto new_discriminators_column = variant_column.getLocalDiscriminatorsPtr()->cloneEmpty(); - auto & new_discriminators_data = assert_cast(*new_discriminators_column).getData(); - new_discriminators_data.reserve(variant_column.size()); - auto new_offsets = variant_column.getOffsetsPtr()->cloneEmpty(); - auto & new_offsets_data = assert_cast(*new_offsets).getData(); - new_offsets_data.reserve(variant_column.size()); - const auto & old_local_discriminators = variant_column.getLocalDiscriminators(); - const auto & old_offsets = variant_column.getOffsets(); - for (size_t i = 0; i != old_local_discriminators.size(); ++i) - { - auto old_discr = variant_column.globalDiscriminatorByLocal(old_local_discriminators[i]); - - if (old_discr == ColumnVariant::NULL_DISCRIMINATOR) - { - new_discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); - new_offsets_data.push_back(0); - continue; - } - - auto new_discr = old_to_new_discriminators_mapping[old_discr]; - new_discriminators_data.push_back(new_discr); - if (new_discr != string_variant_discriminator) - { - new_offsets_data.push_back(old_offsets[i]); - } - else - { - new_offsets_data.push_back(string_variant->size()); - string_variant->insertFrom(*variants_converted_to_string[old_discr], old_offsets[i]); - } - } - - /// Create new list of variant columns. - Columns new_variant_columns; - new_variant_columns.resize(new_variants.size()); - for (size_t i = 0; i != variant_types.size(); ++i) - { - auto new_discr = old_to_new_discriminators_mapping[i]; - if (new_discr != string_variant_discriminator) - new_variant_columns[new_discr] = variant_column.getVariantPtrByGlobalDiscriminator(i); - } - new_variant_columns[string_variant_discriminator] = std::move(string_variant); - return {ColumnVariant::create(std::move(new_discriminators_column), std::move(new_offsets), new_variant_columns), reduced_variant}; - } - - WrapperType createVariantToDynamicWrapper(const DataTypePtr & from_type, const DataTypeDynamic & dynamic_type) const - { - const auto & from_variant_type = assert_cast(*from_type); - size_t max_dynamic_types = dynamic_type.getMaxDynamicTypes(); - const auto & variants = from_variant_type.getVariants(); - std::unordered_map variant_name_to_discriminator; - variant_name_to_discriminator.reserve(variants.size()); - for (size_t i = 0; i != variants.size(); ++i) - variant_name_to_discriminator[variants[i]->getName()] = i; - - return [from_type, max_dynamic_types, variant_name_to_discriminator, this] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr - { - const auto & variant_column = assert_cast(*arguments.front().column); - auto [reduced_variant_column, reduced_variant_type] = getReducedVariant(variant_column, from_type, variant_name_to_discriminator, max_dynamic_types); - return ColumnDynamic::create(reduced_variant_column, reduced_variant_type, max_dynamic_types); + /// Otherwise some variants should go to the shared variant. Create temporary Dynamic column from this Variant and insert + /// all data to the resulting Dynamic column, this insertion will do all the logic with shared variant. + auto tmp_dynamic_column = ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, number_of_variants, number_of_variants); + auto result_dynamic_column = ColumnDynamic::create(max_dynamic_types); + result_dynamic_column->insertRangeFrom(*tmp_dynamic_column, 0, tmp_dynamic_column->size()); + return result_dynamic_column; }; } WrapperType createColumnToDynamicWrapper(const DataTypePtr & from_type, const DataTypeDynamic & dynamic_type) const { if (const auto * variant_type = typeid_cast(from_type.get())) - return createVariantToDynamicWrapper(from_type, dynamic_type); - - if (dynamic_type.getMaxDynamicTypes() == 1) - { - DataTypePtr string_type = std::make_shared(); - if (from_type->isNullable()) - string_type = makeNullable(string_type); - auto string_wrapper = prepareUnpackDictionaries(from_type, string_type); - auto variant_type = std::make_shared(DataTypes{removeNullable(string_type)}); - auto variant_wrapper = createColumnToVariantWrapper(string_type, *variant_type); - return [string_wrapper, variant_wrapper, string_type, variant_type, max_dynamic_types=dynamic_type.getMaxDynamicTypes()] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr - { - auto string_column = string_wrapper(arguments, string_type, col_nullable, input_rows_count); - auto column = ColumnWithTypeAndName(string_column, string_type, ""); - ColumnsWithTypeAndName args = {column}; - auto variant_column = variant_wrapper(args, variant_type, nullptr, string_column->size()); - return ColumnDynamic::create(variant_column, variant_type, max_dynamic_types); - }; - } + return createVariantToDynamicWrapper(*variant_type, dynamic_type); if (context && context->getSettingsRef().cast_string_to_dynamic_use_inference && isStringOrFixedString(removeNullable(removeLowCardinality(from_type)))) return createStringToDynamicThroughParsingWrapper(); + /// First, cast column to Variant with 2 variants - the type of the column we cast and shared variant type. auto variant_type = std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(from_type)}); - auto variant_wrapper = createColumnToVariantWrapper(from_type, *variant_type); - return [variant_wrapper, variant_type, max_dynamic_types=dynamic_type.getMaxDynamicTypes()] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr + auto column_to_variant_wrapper = createColumnToVariantWrapper(from_type, *variant_type); + /// Second, cast this Variant to Dynamic. + auto variant_to_dynamic_wrapper = createVariantToDynamicWrapper(*variant_type, dynamic_type); + return [column_to_variant_wrapper, variant_to_dynamic_wrapper, variant_type] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr { - auto variant_res = variant_wrapper(arguments, variant_type, col_nullable, input_rows_count); - return ColumnDynamic::create(variant_res, variant_type, max_dynamic_types); + auto variant_res = column_to_variant_wrapper(arguments, variant_type, col_nullable, input_rows_count); + ColumnsWithTypeAndName args = {{variant_res, variant_type, ""}}; + return variant_to_dynamic_wrapper(args, result_type, nullptr, input_rows_count); }; } @@ -4530,21 +4467,26 @@ private: (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr { const auto & column_dynamic = assert_cast(*arguments[0].column); - return ColumnDynamic::create(column_dynamic.getVariantColumnPtr(), column_dynamic.getVariantInfo(), to_max_types); + /// We should use the same limit as already used in column and change only global limit. + /// It's needed because shared variant should contain values only when limit is exceeded, + /// so if there are already some data, we cannot increase the limit. + return ColumnDynamic::create(column_dynamic.getVariantColumnPtr(), column_dynamic.getVariantInfo(), column_dynamic.getMaxDynamicTypes(), to_max_types); }; } - return [to_max_types, this] + return [to_max_types] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr { const auto & column_dynamic = assert_cast(*arguments[0].column); - auto [reduced_variant_column, reduced_variant_type] = getReducedVariant( - column_dynamic.getVariantColumn(), - column_dynamic.getVariantInfo().variant_type, - column_dynamic.getVariantInfo().variant_name_to_discriminator, - to_max_types, - column_dynamic.getStatistics()); - return ColumnDynamic::create(reduced_variant_column, reduced_variant_type, to_max_types); + /// If real limit in the column is not greater than desired, just use the same variant column. + if (column_dynamic.getMaxDynamicTypes() <= to_max_types) + return ColumnDynamic::create(column_dynamic.getVariantColumnPtr(), column_dynamic.getVariantInfo(), column_dynamic.getMaxDynamicTypes(), to_max_types); + + /// Otherwise some variants should go to the shared variant. In this case we can just insert all + /// the data into resulting column and it will do all the logic with shared variant. + auto result_dynamic_column = ColumnDynamic::create(to_max_types); + result_dynamic_column->insertRangeFrom(column_dynamic, 0, column_dynamic.size()); + return result_dynamic_column; }; } diff --git a/src/Functions/dynamicType.cpp b/src/Functions/dynamicType.cpp index e8ca73597d6..327cdfe1616 100644 --- a/src/Functions/dynamicType.cpp +++ b/src/Functions/dynamicType.cpp @@ -2,10 +2,14 @@ #include #include #include +#include +#include #include #include #include #include +#include +#include #include @@ -65,11 +69,15 @@ public: const auto & variant_column = dynamic_column->getVariantColumn(); auto res = result_type->createColumn(); String element_type; + auto shared_variant_discr = dynamic_column->getSharedVariantDiscriminator(); + const auto & shared_variant = dynamic_column->getSharedVariant(); for (size_t i = 0; i != input_rows_count; ++i) { auto global_discr = variant_column.globalDiscriminatorAt(i); if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) element_type = name_for_null; + else if (global_discr == shared_variant_discr) + element_type = getTypeNameFromSharedVariantValue(shared_variant.getDataAt(variant_column.offsetAt(i))); else element_type = variant_info.variant_names[global_discr]; @@ -78,6 +86,63 @@ public: return res; } + + String getTypeNameFromSharedVariantValue(StringRef value) const + { + ReadBufferFromMemory buf(value.data, value.size); + return decodeDataType(buf)->getName(); + } +}; + +class FunctionIsDynamicElementInSharedData : public IFunction +{ +public: + static constexpr auto name = "isDynamicElementInSharedData"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.empty() || arguments.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1", + getName(), arguments.empty()); + + if (!isDynamic(arguments[0].type.get())) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Dynamic, got {} instead", + getName(), arguments[0].type->getName()); + + return DataTypeFactory::instance().get("Bool"); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ColumnDynamic * dynamic_column = checkAndGetColumn(arguments[0].column.get()); + if (!dynamic_column) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Dynamic, got {} instead", + getName(), arguments[0].type->getName()); + + const auto & variant_column = dynamic_column->getVariantColumn(); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + auto res = result_type->createColumn(); + auto & res_data = assert_cast(*res).getData(); + res_data.reserve(dynamic_column->size()); + auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(dynamic_column->getSharedVariantDiscriminator()); + for (size_t i = 0; i != input_rows_count; ++i) + res_data.push_back(local_discriminators[i] == shared_variant_local_discr); + + return res; + } }; } @@ -88,7 +153,7 @@ REGISTER_FUNCTION(DynamicType) .description = R"( Returns the variant type name for each row of `Dynamic` column. If row contains NULL, it returns 'None' for it. )", - .syntax = {"dynamicType(variant)"}, + .syntax = {"dynamicType(dynamic)"}, .arguments = {{"dynamic", "Dynamic column"}}, .examples = {{{ "Example", @@ -104,6 +169,30 @@ SELECT d, dynamicType(d) FROM test; │ Hello, World! │ String │ │ [1,2,3] │ Array(Int64) │ └───────────────┴────────────────┘ +)"}}}, + .categories{"Variant"}, + }); + + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns true for rows in Dynamic column that are not separated into subcolumns and stored inside shared variant in binary form. +)", + .syntax = {"isDynamicElementInSharedData(dynamic)"}, + .arguments = {{"dynamic", "Dynamic column"}}, + .examples = {{{ + "Example", + R"( +CREATE TABLE test (d Dynamic(max_types=2)) ENGINE = Memory; +INSERT INTO test VALUES (NULL), (42), ('Hello, World!'), ([1, 2, 3]); +SELECT d, isDynamicElementInSharedData(d) FROM test; +)", + R"( +┌─d─────────────┬─isDynamicElementInSharedData(d)─┠+│ á´ºáµá´¸á´¸ │ false │ +│ 42 │ false │ +│ Hello, World! │ true │ +│ [1,2,3] │ true │ +└───────────────┴────────────────────┘ )"}}}, .categories{"Variant"}, }); diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql new file mode 100644 index 00000000000..db9dd774484 --- /dev/null +++ b/tests/queries/0_stateless/00000_test.sql @@ -0,0 +1,43 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; +set enable_named_columns_in_function_tuple = 0; +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=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; + + +system stop merges test; +insert into test select number, number from numbers(10); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=3)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(10); + +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=3)) from numbers(4); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; + +drop table test; + diff --git a/tests/queries/0_stateless/03033_dynamic_text_serialization.reference b/tests/queries/0_stateless/03033_dynamic_text_serialization.reference index d965245266c..9fc356cc5e6 100644 --- a/tests/queries/0_stateless/03033_dynamic_text_serialization.reference +++ b/tests/queries/0_stateless/03033_dynamic_text_serialization.reference @@ -11,11 +11,11 @@ JSON {"d":["1","str",["1","2","3"]],"dynamicType(d)":"Tuple(Int64, String, Array(Int64))"} {"d":null,"dynamicType(d)":"None"} {"d":true,"dynamicType(d)":"Bool"} -{"d":"42","dynamicType(d)":"Int64"} -{"d":"42.42","dynamicType(d)":"String"} -{"d":"str","dynamicType(d)":"String"} -{"d":null,"dynamicType(d)":"None"} -{"d":"1","dynamicType(d)":"Int64"} +{"d":"42","dynamicType(d)":"Int64","isDynamicElementInSharedData(d)":false} +{"d":42.42,"dynamicType(d)":"Float64","isDynamicElementInSharedData(d)":false} +{"d":"str","dynamicType(d)":"String","isDynamicElementInSharedData(d)":true} +{"d":null,"dynamicType(d)":"None","isDynamicElementInSharedData(d)":false} +{"d":true,"dynamicType(d)":"Bool","isDynamicElementInSharedData(d)":true} CSV 42,"Int64" 42.42,"Float64" @@ -44,12 +44,12 @@ Cast using parsing [1,2,3] Array(Int64) 2020-01-01 Date 2020-01-01 10:00:00.000000000 DateTime64(9) -\N None +NULL String true Bool -42 Int64 -42.42 Float64 -[1, 2, 3] String -2020-01-01 String -2020-01-01 10:00:00 String -\N None -true String +42 Int64 false +42.42 Float64 false +[1,2,3] Array(Int64) false +2020-01-01 Date true +2020-01-01 10:00:00.000000000 DateTime64(9) true +NULL String true +true Bool true diff --git a/tests/queries/0_stateless/03033_dynamic_text_serialization.sql b/tests/queries/0_stateless/03033_dynamic_text_serialization.sql index d12d110fe28..45539cb13eb 100644 --- a/tests/queries/0_stateless/03033_dynamic_text_serialization.sql +++ b/tests/queries/0_stateless/03033_dynamic_text_serialization.sql @@ -16,7 +16,7 @@ select d, dynamicType(d) from format(JSONEachRow, 'd Dynamic', $$ {"d" : true} $$) format JSONEachRow; -select d, dynamicType(d) from format(JSONEachRow, 'd Dynamic(max_types=2)', $$ +select d, dynamicType(d), isDynamicElementInSharedData(d) from format(JSONEachRow, 'd Dynamic(max_types=2)', $$ {"d" : 42} {"d" : 42.42} {"d" : "str"} @@ -69,6 +69,6 @@ create table test (s String) engine=Memory; insert into test values ('42'), ('42.42'), ('[1, 2, 3]'), ('2020-01-01'), ('2020-01-01 10:00:00'), ('NULL'), ('true'); set cast_string_to_dynamic_use_inference=1; select s::Dynamic as d, dynamicType(d) from test; -select s::Dynamic(max_types=3) as d, dynamicType(d) from test; +select s::Dynamic(max_types=3) as d, dynamicType(d), isDynamicElementInSharedData(d) from test; drop table test; diff --git a/tests/queries/0_stateless/03034_dynamic_conversions.reference b/tests/queries/0_stateless/03034_dynamic_conversions.reference index 45f94f7ecc4..e22b64701a3 100644 --- a/tests/queries/0_stateless/03034_dynamic_conversions.reference +++ b/tests/queries/0_stateless/03034_dynamic_conversions.reference @@ -1,9 +1,9 @@ 0 UInt64 1 UInt64 2 UInt64 -0 String -1 String -2 String +0 UInt64 +1 UInt64 +2 UInt64 0 1 2 @@ -25,15 +25,15 @@ str_1 String \N None 4 UInt64 str_5 String -0 String +0 UInt64 str_1 String -[0,1] String +[0,1] Array(UInt64) \N None -4 String +4 UInt64 str_5 String 0 UInt64 str_1 String -[0,1] String +[0,1] Array(UInt64) \N None 4 UInt64 str_5 String @@ -51,13 +51,13 @@ str_5 String 2 0 UInt64 str_1 String -[0,1] String +[0,1] Array(UInt64) \N None 4 UInt64 str_5 String 0 UInt64 1970-01-02 Date -[0,1] String +[0,1] Array(UInt64) \N None 4 UInt64 1970-01-06 Date diff --git a/tests/queries/0_stateless/03034_dynamic_conversions.sql b/tests/queries/0_stateless/03034_dynamic_conversions.sql index ed75fbf2377..c0b470f29c5 100644 --- a/tests/queries/0_stateless/03034_dynamic_conversions.sql +++ b/tests/queries/0_stateless/03034_dynamic_conversions.sql @@ -3,7 +3,7 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; select number::Dynamic as d, dynamicType(d) from numbers(3); -select number::Dynamic(max_types=1) as d, dynamicType(d) from numbers(3); +select number::Dynamic(max_types=0) as d, dynamicType(d) from numbers(3); select number::Dynamic::UInt64 as v from numbers(3); select number::Dynamic::String as v from numbers(3); select number::Dynamic::Date as v from numbers(3); @@ -12,13 +12,13 @@ select number::Dynamic::Variant(UInt64, String) as v, variantType(v) from number select (number % 2 ? NULL : number)::Dynamic as d, dynamicType(d) from numbers(3); select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic as d, dynamicType(d) from numbers(6); +select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=0) as d, dynamicType(d) from numbers(6); select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=1) as d, dynamicType(d) from numbers(6); select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=2) as d, dynamicType(d) from numbers(6); -select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=3) as d, dynamicType(d) from numbers(6); select number::Dynamic(max_types=2)::Dynamic(max_types=3) as d from numbers(3); select number::Dynamic(max_types=2)::Dynamic(max_types=1) as d from numbers(3); -select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=3)::Dynamic(max_types=2) as d, dynamicType(d) from numbers(6); +select multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=2)::Dynamic(max_types=1) as d, dynamicType(d) from numbers(6); select multiIf(number % 4 == 0, number, number % 4 == 1, toDate(number), number % 4 == 2, range(number), NULL)::Dynamic(max_types=4)::Dynamic(max_types=3) as d, dynamicType(d) from numbers(6); diff --git a/tests/queries/0_stateless/03035_dynamic_sorting.reference b/tests/queries/0_stateless/03035_dynamic_sorting.reference index 9b8df11c7a9..f253c34ce8a 100644 --- a/tests/queries/0_stateless/03035_dynamic_sorting.reference +++ b/tests/queries/0_stateless/03035_dynamic_sorting.reference @@ -1,299 +1,442 @@ order by d1 nulls first -\N None -\N None -\N None -\N None -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String +\N None false +\N None false +\N None false +\N None false +\N None false +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false order by d1 nulls last -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String -\N None -\N None -\N None -\N None +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false +\N None false +\N None false +\N None false +\N None false +\N None false order by d2 nulls first -\N None -\N None -\N None -\N None -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String +\N None false +\N None false +\N None false +\N None false +\N None false +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false order by d2 nulls last -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,3] Array(Int64) -[1,2,4] Array(Int64) -42 Int64 -42 Int64 -42 Int64 -42 Int64 -42 Int64 -43 Int64 -abc String -abc String -abc String -abc String -abc String -abd String -\N None -\N None -\N None -\N None +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,3] Array(Int64) true +[1,2,4] Array(Int64) true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-01 Date true +2020-01-02 Date true +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +42 Int64 false +43 Int64 false +abc String false +abc String false +abc String false +abc String false +abc String false +abc String false +abd String false +\N None false +\N None false +\N None false +\N None false +\N None false order by d1, d2 nulls first -[1,2,3] \N Array(Int64) None -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -[1,2,3] 42 Array(Int64) Int64 -[1,2,3] abc Array(Int64) String -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 \N Int64 None -42 [1,2,3] Int64 Array(Int64) -42 42 Int64 Int64 -42 43 Int64 Int64 -42 abc Int64 String -43 42 Int64 Int64 -abc \N String None -abc [1,2,3] String Array(Int64) -abc 42 String Int64 -abc abc String String -abc abd String String -abd abc String String -\N \N None None -\N [1,2,3] None Array(Int64) -\N 42 None Int64 -\N abc None String +[1,2,3] \N Array(Int64) true None false +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 Array(Int64) true Date true +[1,2,3] 42 Array(Int64) true Int64 false +[1,2,3] abc Array(Int64) true String false +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 \N Date true None false +2020-01-01 [1,2,3] Date true Array(Int64) true +2020-01-01 2020-01-01 Date true Date true +2020-01-01 2020-01-02 Date true Date true +2020-01-01 42 Date true Int64 false +2020-01-01 abc Date true String false +2020-01-02 2020-01-01 Date true Date true +42 \N Int64 false None false +42 [1,2,3] Int64 false Array(Int64) true +42 2020-01-01 Int64 false Date true +42 42 Int64 false Int64 false +42 43 Int64 false Int64 false +42 abc Int64 false String false +43 42 Int64 false Int64 false +abc \N String false None false +abc [1,2,3] String false Array(Int64) true +abc 2020-01-01 String false Date true +abc 42 String false Int64 false +abc abc String false String false +abc abd String false String false +abd abc String false String false +\N \N None false None false +\N [1,2,3] None false Array(Int64) true +\N 2020-01-01 None false Date true +\N 42 None false Int64 false +\N abc None false String false order by d1, d2 nulls last -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -[1,2,3] 42 Array(Int64) Int64 -[1,2,3] abc Array(Int64) String -[1,2,3] \N Array(Int64) None -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 [1,2,3] Int64 Array(Int64) -42 42 Int64 Int64 -42 43 Int64 Int64 -42 abc Int64 String -42 \N Int64 None -43 42 Int64 Int64 -abc [1,2,3] String Array(Int64) -abc 42 String Int64 -abc abc String String -abc abd String String -abc \N String None -abd abc String String -\N [1,2,3] None Array(Int64) -\N 42 None Int64 -\N abc None String -\N \N None None +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 Array(Int64) true Date true +[1,2,3] 42 Array(Int64) true Int64 false +[1,2,3] abc Array(Int64) true String false +[1,2,3] \N Array(Int64) true None false +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] Date true Array(Int64) true +2020-01-01 2020-01-01 Date true Date true +2020-01-01 2020-01-02 Date true Date true +2020-01-01 42 Date true Int64 false +2020-01-01 abc Date true String false +2020-01-01 \N Date true None false +2020-01-02 2020-01-01 Date true Date true +42 [1,2,3] Int64 false Array(Int64) true +42 2020-01-01 Int64 false Date true +42 42 Int64 false Int64 false +42 43 Int64 false Int64 false +42 abc Int64 false String false +42 \N Int64 false None false +43 42 Int64 false Int64 false +abc [1,2,3] String false Array(Int64) true +abc 2020-01-01 String false Date true +abc 42 String false Int64 false +abc abc String false String false +abc abd String false String false +abc \N String false None false +abd abc String false String false +\N [1,2,3] None false Array(Int64) true +\N 2020-01-01 None false Date true +\N 42 None false Int64 false +\N abc None false String false +\N \N None false None false order by d2, d1 nulls first -\N [1,2,3] None Array(Int64) -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 [1,2,3] Int64 Array(Int64) -abc [1,2,3] String Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -\N 42 None Int64 -[1,2,3] 42 Array(Int64) Int64 -42 42 Int64 Int64 -43 42 Int64 Int64 -abc 42 String Int64 -42 43 Int64 Int64 -\N abc None String -[1,2,3] abc Array(Int64) String -42 abc Int64 String -abc abc String String -abd abc String String -abc abd String String -\N \N None None -[1,2,3] \N Array(Int64) None -42 \N Int64 None -abc \N String None +\N [1,2,3] None false Array(Int64) true +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] Date true Array(Int64) true +42 [1,2,3] Int64 false Array(Int64) true +abc [1,2,3] String false Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +\N 2020-01-01 None false Date true +[1,2,3] 2020-01-01 Array(Int64) true Date true +2020-01-01 2020-01-01 Date true Date true +2020-01-02 2020-01-01 Date true Date true +42 2020-01-01 Int64 false Date true +abc 2020-01-01 String false Date true +2020-01-01 2020-01-02 Date true Date true +\N 42 None false Int64 false +[1,2,3] 42 Array(Int64) true Int64 false +2020-01-01 42 Date true Int64 false +42 42 Int64 false Int64 false +43 42 Int64 false Int64 false +abc 42 String false Int64 false +42 43 Int64 false Int64 false +\N abc None false String false +[1,2,3] abc Array(Int64) true String false +2020-01-01 abc Date true String false +42 abc Int64 false String false +abc abc String false String false +abd abc String false String false +abc abd String false String false +\N \N None false None false +[1,2,3] \N Array(Int64) true None false +2020-01-01 \N Date true None false +42 \N Int64 false None false +abc \N String false None false order by d2, d1 nulls last -[1,2,3] [1,2,3] Array(Int64) Array(Int64) -[1,2,4] [1,2,3] Array(Int64) Array(Int64) -42 [1,2,3] Int64 Array(Int64) -abc [1,2,3] String Array(Int64) -\N [1,2,3] None Array(Int64) -[1,2,3] [1,2,4] Array(Int64) Array(Int64) -[1,2,3] 42 Array(Int64) Int64 -42 42 Int64 Int64 -43 42 Int64 Int64 -abc 42 String Int64 -\N 42 None Int64 -42 43 Int64 Int64 -[1,2,3] abc Array(Int64) String -42 abc Int64 String -abc abc String String -abd abc String String -\N abc None String -abc abd String String -[1,2,3] \N Array(Int64) None -42 \N Int64 None -abc \N String None -\N \N None None +[1,2,3] [1,2,3] Array(Int64) true Array(Int64) true +[1,2,4] [1,2,3] Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] Date true Array(Int64) true +42 [1,2,3] Int64 false Array(Int64) true +abc [1,2,3] String false Array(Int64) true +\N [1,2,3] None false Array(Int64) true +[1,2,3] [1,2,4] Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 Array(Int64) true Date true +2020-01-01 2020-01-01 Date true Date true +2020-01-02 2020-01-01 Date true Date true +42 2020-01-01 Int64 false Date true +abc 2020-01-01 String false Date true +\N 2020-01-01 None false Date true +2020-01-01 2020-01-02 Date true Date true +[1,2,3] 42 Array(Int64) true Int64 false +2020-01-01 42 Date true Int64 false +42 42 Int64 false Int64 false +43 42 Int64 false Int64 false +abc 42 String false Int64 false +\N 42 None false Int64 false +42 43 Int64 false Int64 false +[1,2,3] abc Array(Int64) true String false +2020-01-01 abc Date true String false +42 abc Int64 false String false +abc abc String false String false +abd abc String false String false +\N abc None false String false +abc abd String false String false +[1,2,3] \N Array(Int64) true None false +2020-01-01 \N Date true None false +42 \N Int64 false None false +abc \N String false None false +\N \N None false None false d1 = d2 -[1,2,3] [1,2,3] 1 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 0 Array(Int64) Array(Int64) -[1,2,3] 42 0 Array(Int64) Int64 -[1,2,3] abc 0 Array(Int64) String -[1,2,3] \N 0 Array(Int64) None -[1,2,4] [1,2,3] 0 Array(Int64) Array(Int64) -42 [1,2,3] 0 Int64 Array(Int64) -42 42 1 Int64 Int64 -42 43 0 Int64 Int64 -42 abc 0 Int64 String -42 \N 0 Int64 None -43 42 0 Int64 Int64 -abc [1,2,3] 0 String Array(Int64) -abc 42 0 String Int64 -abc abc 1 String String -abc abd 0 String String -abc \N 0 String None -abd abc 0 String String -\N [1,2,3] 0 None Array(Int64) -\N 42 0 None Int64 -\N abc 0 None String -\N \N 1 None None +[1,2,3] [1,2,3] 1 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 0 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 0 Array(Int64) true Date true +[1,2,3] 42 0 Array(Int64) true Int64 false +[1,2,3] abc 0 Array(Int64) true String false +[1,2,3] \N 0 Array(Int64) true None false +[1,2,4] [1,2,3] 0 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 0 Date true Array(Int64) true +2020-01-01 2020-01-01 1 Date true Date true +2020-01-01 2020-01-02 0 Date true Date true +2020-01-01 42 0 Date true Int64 false +2020-01-01 abc 0 Date true String false +2020-01-01 \N 0 Date true None false +2020-01-02 2020-01-01 0 Date true Date true +42 [1,2,3] 0 Int64 false Array(Int64) true +42 2020-01-01 0 Int64 false Date true +42 42 1 Int64 false Int64 false +42 43 0 Int64 false Int64 false +42 abc 0 Int64 false String false +42 \N 0 Int64 false None false +43 42 0 Int64 false Int64 false +abc [1,2,3] 0 String false Array(Int64) true +abc 2020-01-01 0 String false Date true +abc 42 0 String false Int64 false +abc abc 1 String false String false +abc abd 0 String false String false +abc \N 0 String false None false +abd abc 0 String false String false +\N [1,2,3] 0 None false Array(Int64) true +\N 2020-01-01 0 None false Date true +\N 42 0 None false Int64 false +\N abc 0 None false String false +\N \N 1 None false None false d1 < d2 -[1,2,3] [1,2,3] 0 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 1 Array(Int64) Array(Int64) -[1,2,3] 42 1 Array(Int64) Int64 -[1,2,3] abc 1 Array(Int64) String -[1,2,3] \N 1 Array(Int64) None -[1,2,4] [1,2,3] 0 Array(Int64) Array(Int64) -42 [1,2,3] 0 Int64 Array(Int64) -42 42 0 Int64 Int64 -42 43 1 Int64 Int64 -42 abc 1 Int64 String -42 \N 1 Int64 None -43 42 0 Int64 Int64 -abc [1,2,3] 0 String Array(Int64) -abc 42 0 String Int64 -abc abc 0 String String -abc abd 1 String String -abc \N 1 String None -abd abc 0 String String -\N [1,2,3] 0 None Array(Int64) -\N 42 0 None Int64 -\N abc 0 None String -\N \N 0 None None +[1,2,3] [1,2,3] 0 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 1 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 1 Array(Int64) true Date true +[1,2,3] 42 1 Array(Int64) true Int64 false +[1,2,3] abc 1 Array(Int64) true String false +[1,2,3] \N 1 Array(Int64) true None false +[1,2,4] [1,2,3] 0 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 0 Date true Array(Int64) true +2020-01-01 2020-01-01 0 Date true Date true +2020-01-01 2020-01-02 1 Date true Date true +2020-01-01 42 1 Date true Int64 false +2020-01-01 abc 1 Date true String false +2020-01-01 \N 1 Date true None false +2020-01-02 2020-01-01 0 Date true Date true +42 [1,2,3] 0 Int64 false Array(Int64) true +42 2020-01-01 0 Int64 false Date true +42 42 0 Int64 false Int64 false +42 43 1 Int64 false Int64 false +42 abc 1 Int64 false String false +42 \N 1 Int64 false None false +43 42 0 Int64 false Int64 false +abc [1,2,3] 0 String false Array(Int64) true +abc 2020-01-01 0 String false Date true +abc 42 0 String false Int64 false +abc abc 0 String false String false +abc abd 1 String false String false +abc \N 1 String false None false +abd abc 0 String false String false +\N [1,2,3] 0 None false Array(Int64) true +\N 2020-01-01 0 None false Date true +\N 42 0 None false Int64 false +\N abc 0 None false String false +\N \N 0 None false None false d1 <= d2 -[1,2,3] [1,2,3] 1 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 1 Array(Int64) Array(Int64) -[1,2,3] 42 1 Array(Int64) Int64 -[1,2,3] abc 1 Array(Int64) String -[1,2,3] \N 1 Array(Int64) None -[1,2,4] [1,2,3] 0 Array(Int64) Array(Int64) -42 [1,2,3] 0 Int64 Array(Int64) -42 42 1 Int64 Int64 -42 43 1 Int64 Int64 -42 abc 1 Int64 String -42 \N 1 Int64 None -43 42 0 Int64 Int64 -abc [1,2,3] 0 String Array(Int64) -abc 42 0 String Int64 -abc abc 1 String String -abc abd 1 String String -abc \N 1 String None -abd abc 0 String String -\N [1,2,3] 0 None Array(Int64) -\N 42 0 None Int64 -\N abc 0 None String -\N \N 1 None None +[1,2,3] [1,2,3] 1 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 1 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 1 Array(Int64) true Date true +[1,2,3] 42 1 Array(Int64) true Int64 false +[1,2,3] abc 1 Array(Int64) true String false +[1,2,3] \N 1 Array(Int64) true None false +[1,2,4] [1,2,3] 0 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 0 Date true Array(Int64) true +2020-01-01 2020-01-01 1 Date true Date true +2020-01-01 2020-01-02 1 Date true Date true +2020-01-01 42 1 Date true Int64 false +2020-01-01 abc 1 Date true String false +2020-01-01 \N 1 Date true None false +2020-01-02 2020-01-01 0 Date true Date true +42 [1,2,3] 0 Int64 false Array(Int64) true +42 2020-01-01 0 Int64 false Date true +42 42 1 Int64 false Int64 false +42 43 1 Int64 false Int64 false +42 abc 1 Int64 false String false +42 \N 1 Int64 false None false +43 42 0 Int64 false Int64 false +abc [1,2,3] 0 String false Array(Int64) true +abc 2020-01-01 0 String false Date true +abc 42 0 String false Int64 false +abc abc 1 String false String false +abc abd 1 String false String false +abc \N 1 String false None false +abd abc 0 String false String false +\N [1,2,3] 0 None false Array(Int64) true +\N 2020-01-01 0 None false Date true +\N 42 0 None false Int64 false +\N abc 0 None false String false +\N \N 1 None false None false d1 > d2 -[1,2,3] [1,2,3] 0 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 0 Array(Int64) Array(Int64) -[1,2,3] 42 0 Array(Int64) Int64 -[1,2,3] abc 0 Array(Int64) String -[1,2,3] \N 0 Array(Int64) None -[1,2,4] [1,2,3] 1 Array(Int64) Array(Int64) -42 [1,2,3] 1 Int64 Array(Int64) -42 42 0 Int64 Int64 -42 43 0 Int64 Int64 -42 abc 0 Int64 String -42 \N 0 Int64 None -43 42 1 Int64 Int64 -abc [1,2,3] 1 String Array(Int64) -abc 42 1 String Int64 -abc abc 0 String String -abc abd 0 String String -abc \N 0 String None -abd abc 1 String String -\N [1,2,3] 1 None Array(Int64) -\N 42 1 None Int64 -\N abc 1 None String -\N \N 0 None None +[1,2,3] [1,2,3] 0 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 0 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 0 Array(Int64) true Date true +[1,2,3] 42 0 Array(Int64) true Int64 false +[1,2,3] abc 0 Array(Int64) true String false +[1,2,3] \N 0 Array(Int64) true None false +[1,2,4] [1,2,3] 1 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 1 Date true Array(Int64) true +2020-01-01 2020-01-01 0 Date true Date true +2020-01-01 2020-01-02 0 Date true Date true +2020-01-01 42 0 Date true Int64 false +2020-01-01 abc 0 Date true String false +2020-01-01 \N 0 Date true None false +2020-01-02 2020-01-01 1 Date true Date true +42 [1,2,3] 1 Int64 false Array(Int64) true +42 2020-01-01 1 Int64 false Date true +42 42 0 Int64 false Int64 false +42 43 0 Int64 false Int64 false +42 abc 0 Int64 false String false +42 \N 0 Int64 false None false +43 42 1 Int64 false Int64 false +abc [1,2,3] 1 String false Array(Int64) true +abc 2020-01-01 1 String false Date true +abc 42 1 String false Int64 false +abc abc 0 String false String false +abc abd 0 String false String false +abc \N 0 String false None false +abd abc 1 String false String false +\N [1,2,3] 1 None false Array(Int64) true +\N 2020-01-01 1 None false Date true +\N 42 1 None false Int64 false +\N abc 1 None false String false +\N \N 0 None false None false d1 >= d2 -[1,2,3] [1,2,3] 1 Array(Int64) Array(Int64) -[1,2,3] [1,2,4] 1 Array(Int64) Array(Int64) -[1,2,3] 42 1 Array(Int64) Int64 -[1,2,3] abc 1 Array(Int64) String -[1,2,3] \N 1 Array(Int64) None -[1,2,4] [1,2,3] 1 Array(Int64) Array(Int64) -42 [1,2,3] 1 Int64 Array(Int64) -42 42 1 Int64 Int64 -42 43 1 Int64 Int64 -42 abc 1 Int64 String -42 \N 1 Int64 None -43 42 1 Int64 Int64 -abc [1,2,3] 1 String Array(Int64) -abc 42 1 String Int64 -abc abc 1 String String -abc abd 1 String String -abc \N 1 String None -abd abc 1 String String -\N [1,2,3] 1 None Array(Int64) -\N 42 1 None Int64 -\N abc 1 None String -\N \N 1 None None +[1,2,3] [1,2,3] 1 Array(Int64) true Array(Int64) true +[1,2,3] [1,2,4] 1 Array(Int64) true Array(Int64) true +[1,2,3] 2020-01-01 1 Array(Int64) true Date true +[1,2,3] 42 1 Array(Int64) true Int64 false +[1,2,3] abc 1 Array(Int64) true String false +[1,2,3] \N 1 Array(Int64) true None false +[1,2,4] [1,2,3] 1 Array(Int64) true Array(Int64) true +2020-01-01 [1,2,3] 1 Date true Array(Int64) true +2020-01-01 2020-01-01 1 Date true Date true +2020-01-01 2020-01-02 1 Date true Date true +2020-01-01 42 1 Date true Int64 false +2020-01-01 abc 1 Date true String false +2020-01-01 \N 1 Date true None false +2020-01-02 2020-01-01 1 Date true Date true +42 [1,2,3] 1 Int64 false Array(Int64) true +42 2020-01-01 1 Int64 false Date true +42 42 1 Int64 false Int64 false +42 43 1 Int64 false Int64 false +42 abc 1 Int64 false String false +42 \N 1 Int64 false None false +43 42 1 Int64 false Int64 false +abc [1,2,3] 1 String false Array(Int64) true +abc 2020-01-01 1 String false Date true +abc 42 1 String false Int64 false +abc abc 1 String false String false +abc abd 1 String false String false +abc \N 1 String false None false +abd abc 1 String false String false +\N [1,2,3] 1 None false Array(Int64) true +\N 2020-01-01 1 None false Date true +\N 42 1 None false Int64 false +\N abc 1 None false String false +\N \N 1 None false None false diff --git a/tests/queries/0_stateless/03035_dynamic_sorting.sql b/tests/queries/0_stateless/03035_dynamic_sorting.sql index 0487fafc955..e0039a348c6 100644 --- a/tests/queries/0_stateless/03035_dynamic_sorting.sql +++ b/tests/queries/0_stateless/03035_dynamic_sorting.sql @@ -1,80 +1,55 @@ set allow_experimental_dynamic_type = 1; drop table if exists test; -create table test (d1 Dynamic, d2 Dynamic) engine=Memory; - -insert into test values (42, 42); -insert into test values (42, 43); -insert into test values (43, 42); - -insert into test values ('abc', 'abc'); -insert into test values ('abc', 'abd'); -insert into test values ('abd', 'abc'); - -insert into test values ([1,2,3], [1,2,3]); -insert into test values ([1,2,3], [1,2,4]); -insert into test values ([1,2,4], [1,2,3]); - -insert into test values (NULL, NULL); - -insert into test values (42, 'abc'); -insert into test values ('abc', 42); - -insert into test values (42, [1,2,3]); -insert into test values ([1,2,3], 42); - -insert into test values (42, NULL); -insert into test values (NULL, 42); - -insert into test values ('abc', [1,2,3]); -insert into test values ([1,2,3], 'abc'); - -insert into test values ('abc', NULL); -insert into test values (NULL, 'abc'); - -insert into test values ([1,2,3], NULL); -insert into test values (NULL, [1,2,3]); +create table test (d1 Dynamic(max_types=2), d2 Dynamic(max_types=2)) engine=Memory; +insert into test values (42, 42), (42, 43), (43, 42), ('abc', 'abc'), ('abc', 'abd'), ('abd', 'abc'), +([1,2,3], [1,2,3]), ([1,2,3], [1,2,4]), ([1,2,4], [1,2,3]), +('2020-01-01', '2020-01-01'), ('2020-01-01', '2020-01-02'), ('2020-01-02', '2020-01-01'), +(NULL, NULL), (42, 'abc'), ('abc', 42), (42, [1,2,3]), ([1,2,3], 42), (42, NULL), (NULL, 42), +('abc', [1,2,3]), ([1,2,3], 'abc'), ('abc', NULL), (NULL, 'abc'), ([1,2,3], NULL), (NULL, [1,2,3]), +(42, '2020-01-01'), ('2020-01-01', 42), ('2020-01-01', 'abc'), ('abc', '2020-01-01'), +('2020-01-01', [1,2,3]), ([1,2,3], '2020-01-01'), ('2020-01-01', NULL), (NULL, '2020-01-01'); select 'order by d1 nulls first'; -select d1, dynamicType(d1) from test order by d1 nulls first; +select d1, dynamicType(d1), isDynamicElementInSharedData(d1) from test order by d1 nulls first; select 'order by d1 nulls last'; -select d1, dynamicType(d1) from test order by d1 nulls last; +select d1, dynamicType(d1), isDynamicElementInSharedData(d1) from test order by d1 nulls last; select 'order by d2 nulls first'; -select d2, dynamicType(d2) from test order by d2 nulls first; +select d2, dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2 nulls first; select 'order by d2 nulls last'; -select d2, dynamicType(d2) from test order by d2 nulls last; +select d2, dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2 nulls last; select 'order by d1, d2 nulls first'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2 nulls first; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2 nulls first; select 'order by d1, d2 nulls last'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2 nulls last; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2 nulls last; select 'order by d2, d1 nulls first'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d2, d1 nulls first; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2, d1 nulls first; select 'order by d2, d1 nulls last'; -select d1, d2, dynamicType(d1), dynamicType(d2) from test order by d2, d1 nulls last; +select d1, d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d2, d1 nulls last; select 'd1 = d2'; -select d1, d2, d1 = d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 = d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 < d2'; -select d1, d2, d1 < d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 < d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 <= d2'; -select d1, d2, d1 <= d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 <= d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 > d2'; -select d1, d2, d1 > d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d1 > d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; select 'd1 >= d2'; -select d1, d2, d2 >= d2, dynamicType(d1), dynamicType(d2) from test order by d1, d2; +select d1, d2, d2 >= d2, dynamicType(d1), isDynamicElementInSharedData(d1), dynamicType(d2), isDynamicElementInSharedData(d2) from test order by d1, d2; drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference new file mode 100644 index 00000000000..ca6c5dbba82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.reference @@ -0,0 +1,20 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +360000 +360000 +200000 +200000 +0 +0 +20000 +20000 +200000 +200000 +20000 +20000 +200000 +0 diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql new file mode 100644 index 00000000000..bff28fb5c90 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_compact_merge_tree.sql @@ -0,0 +1,43 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`LowCardinality(String)`, d.`Array(String)` from test format Null; +select d, d.UInt64, d.Date, d.`LowCardinality(String)`, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.UInt64, d.Date, d.`LowCardinality(String)`, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test format Null; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test format Null; + +drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference new file mode 100644 index 00000000000..ca6c5dbba82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.reference @@ -0,0 +1,20 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +360000 +360000 +200000 +200000 +0 +0 +20000 +20000 +200000 +200000 +20000 +20000 +200000 +0 diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql new file mode 100644 index 00000000000..4eed3d15529 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_memory.sql @@ -0,0 +1,43 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=2)) engine=Memory; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test format Null; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test format Null; + +drop table test; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 new file mode 100644 index 00000000000..9c1f8fa45e8 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 @@ -0,0 +1,2460 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +36 +36 +20 +20 +0 +0 +2 +2 +20 +20 +2 +2 +20 +0 +0 0 \N [] +1 1 \N [] +2 2 \N [] +3 3 \N [] +4 4 \N [] +5 5 \N [] +6 6 \N [] +7 7 \N [] +8 8 \N [] +9 9 \N [] +str_10 \N str_10 [] +10 10 \N [] +10 10 \N [] +[[0,1]] \N \N [] +str_11 \N \N [] +str_11 \N str_11 [] +str_12 \N str_12 [] +12 12 \N [] +12 12 \N [] +str_13 \N str_13 [] +13 13 \N [] +13 13 \N [] +str_14 \N str_14 [] +14 14 \N [] +14 14 \N [] +str_15 \N str_15 [] +15 15 \N [] +15 15 \N [] +[[0,1,2,3,4,5,6]] \N \N [] +str_16 \N \N [] +str_16 \N str_16 [] +str_17 \N str_17 [] +17 17 \N [] +17 17 \N [] +str_18 \N str_18 [] +18 18 \N [] +18 18 \N [] +str_19 \N str_19 [] +19 19 \N [] +19 19 \N [] +[20] \N \N [20] +['str_21','str_21'] \N \N ['str_21','str_21'] +[22,22,22] \N \N [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +[40] \N \N [40] +41 41 \N [] +\N \N \N [] +str_43 \N str_43 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] +45 45 \N [] +\N \N \N [] +str_47 \N str_47 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 49 \N [] +\N \N \N [] +str_51 \N str_51 [] +[52,52,52] \N \N [52,52,52] +53 53 \N [] +\N \N \N [] +str_55 \N str_55 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] +57 57 \N [] +\N \N \N [] +str_59 \N str_59 [] +[60] \N \N [60] +61 61 \N [] +\N \N \N [] +str_63 \N str_63 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] +65 65 \N [] +\N \N \N [] +str_67 \N str_67 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] +69 69 \N [] +\N \N \N [] +str_71 \N str_71 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] +73 73 \N [] +\N \N \N [] +str_75 \N str_75 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] +77 77 \N [] +\N \N \N [] +str_79 \N str_79 [] +0 \N [] +1 \N [] +2 \N [] +3 \N [] +4 \N [] +5 \N [] +6 \N [] +7 \N [] +8 \N [] +9 \N [] +\N str_10 [] +10 \N [] +10 \N [] +\N \N [] +\N \N [] +\N str_11 [] +\N str_12 [] +12 \N [] +12 \N [] +\N str_13 [] +13 \N [] +13 \N [] +\N str_14 [] +14 \N [] +14 \N [] +\N str_15 [] +15 \N [] +15 \N [] +\N \N [] +\N \N [] +\N str_16 [] +\N str_17 [] +17 \N [] +17 \N [] +\N str_18 [] +18 \N [] +18 \N [] +\N str_19 [] +19 \N [] +19 \N [] +\N \N [20] +\N \N ['str_21','str_21'] +\N \N [22,22,22] +\N \N [23,23,23,23] +\N \N [24,24,24,24,24] +\N \N [25,25,25,25,25,25] +\N \N [26,26,26,26,26,26,26] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] +\N \N [29,29,29,29,29,29,29,29,29,29] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [40] +41 \N [] +\N \N [] +\N str_43 [] +\N \N [44,44,44,44,44] +45 \N [] +\N \N [] +\N str_47 [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] +\N \N [] +\N str_51 [] +\N \N [52,52,52] +53 \N [] +\N \N [] +\N str_55 [] +\N \N [56,56,56,56,56,56,56] +57 \N [] +\N \N [] +\N str_59 [] +\N \N [60] +61 \N [] +\N \N [] +\N str_63 [] +\N \N [64,64,64,64,64] +65 \N [] +\N \N [] +\N str_67 [] +\N \N [68,68,68,68,68,68,68,68,68] +69 \N [] +\N \N [] +\N str_71 [] +\N \N [NULL,NULL,NULL] +73 \N [] +\N \N [] +\N str_75 [] +\N \N [76,76,76,76,76,76,76] +77 \N [] +\N \N [] +\N str_79 [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +0 0 \N [] 0 [] +1 1 \N [] 0 [] +2 2 \N [] 0 [] +3 3 \N [] 0 [] +4 4 \N [] 0 [] +5 5 \N [] 0 [] +6 6 \N [] 0 [] +7 7 \N [] 0 [] +8 8 \N [] 0 [] +9 9 \N [] 0 [] +str_10 \N \N [] 0 [] +10 10 \N [] 0 [] +10 10 \N [] 0 [] +[[0,1]] \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_12 \N \N [] 0 [] +12 12 \N [] 0 [] +12 12 \N [] 0 [] +str_13 \N \N [] 0 [] +13 13 \N [] 0 [] +13 13 \N [] 0 [] +str_14 \N \N [] 0 [] +14 14 \N [] 0 [] +14 14 \N [] 0 [] +str_15 \N \N [] 0 [] +15 15 \N [] 0 [] +15 15 \N [] 0 [] +[[0,1,2,3,4,5,6]] \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_17 \N \N [] 0 [] +17 17 \N [] 0 [] +17 17 \N [] 0 [] +str_18 \N \N [] 0 [] +18 18 \N [] 0 [] +18 18 \N [] 0 [] +str_19 \N \N [] 0 [] +19 19 \N [] 0 [] +19 19 \N [] 0 [] +[20] \N \N [20] 1 [20] +['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] +[22,22,22] \N \N [22,22,22] 3 [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] 4 [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] 5 [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +[40] \N \N [40] 1 [40] +41 41 \N [] 0 [] +\N \N \N [] 0 [] +str_43 \N \N [] 0 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] 5 [44,44,44,44,44] +45 45 \N [] 0 [] +\N \N \N [] 0 [] +str_47 \N \N [] 0 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 49 \N [] 0 [] +\N \N \N [] 0 [] +str_51 \N \N [] 0 [] +[52,52,52] \N \N [52,52,52] 3 [52,52,52] +53 53 \N [] 0 [] +\N \N \N [] 0 [] +str_55 \N \N [] 0 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] +57 57 \N [] 0 [] +\N \N \N [] 0 [] +str_59 \N \N [] 0 [] +[60] \N \N [60] 1 [60] +61 61 \N [] 0 [] +\N \N \N [] 0 [] +str_63 \N \N [] 0 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] 5 [64,64,64,64,64] +65 65 \N [] 0 [] +\N \N \N [] 0 [] +str_67 \N \N [] 0 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] +69 69 \N [] 0 [] +\N \N \N [] 0 [] +str_71 \N \N [] 0 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] +73 73 \N [] 0 [] +\N \N \N [] 0 [] +str_75 \N \N [] 0 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] +77 77 \N [] 0 [] +\N \N \N [] 0 [] +str_79 \N \N [] 0 [] +0 \N [] 0 [] [] +1 \N [] 0 [] [] +2 \N [] 0 [] [] +3 \N [] 0 [] [] +4 \N [] 0 [] [] +5 \N [] 0 [] [] +6 \N [] 0 [] [] +7 \N [] 0 [] [] +8 \N [] 0 [] [] +9 \N [] 0 [] [] +\N \N [] 0 [] [] +10 \N [] 0 [] [] +10 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +12 \N [] 0 [] [] +12 \N [] 0 [] [] +\N \N [] 0 [] [] +13 \N [] 0 [] [] +13 \N [] 0 [] [] +\N \N [] 0 [] [] +14 \N [] 0 [] [] +14 \N [] 0 [] [] +\N \N [] 0 [] [] +15 \N [] 0 [] [] +15 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +17 \N [] 0 [] [] +17 \N [] 0 [] [] +\N \N [] 0 [] [] +18 \N [] 0 [] [] +18 \N [] 0 [] [] +\N \N [] 0 [] [] +19 \N [] 0 [] [] +19 \N [] 0 [] [] +\N \N [20] 1 [20] [NULL] +\N \N ['str_21','str_21'] 2 [NULL,NULL] ['str_21','str_21'] +\N \N [22,22,22] 3 [22,22,22] [NULL,NULL,NULL] +\N \N [23,23,23,23] 4 [23,23,23,23] [NULL,NULL,NULL,NULL] +\N \N [24,24,24,24,24] 5 [24,24,24,24,24] [NULL,NULL,NULL,NULL,NULL] +\N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] [NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [40] 1 [40] [NULL] +41 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [44,44,44,44,44] 5 [44,44,44,44,44] [NULL,NULL,NULL,NULL,NULL] +45 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [52,52,52] 3 [52,52,52] [NULL,NULL,NULL] +53 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +57 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [60] 1 [60] [NULL] +61 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [64,64,64,64,64] 5 [64,64,64,64,64] [NULL,NULL,NULL,NULL,NULL] +65 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +69 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] [NULL,NULL,NULL] +73 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +77 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +0 0 [] [] +1 0 [] [] +2 0 [] [] +3 0 [] [] +4 0 [] [] +5 0 [] [] +6 0 [] [] +7 0 [] [] +8 0 [] [] +9 0 [] [] +str_10 0 [] [] +10 0 [] [] +10 0 [] [] +[[0,1]] 0 [] [] +str_11 0 [] [] +str_11 0 [] [] +str_12 0 [] [] +12 0 [] [] +12 0 [] [] +str_13 0 [] [] +13 0 [] [] +13 0 [] [] +str_14 0 [] [] +14 0 [] [] +14 0 [] [] +str_15 0 [] [] +15 0 [] [] +15 0 [] [] +[[0,1,2,3,4,5,6]] 0 [] [] +str_16 0 [] [] +str_16 0 [] [] +str_17 0 [] [] +17 0 [] [] +17 0 [] [] +str_18 0 [] [] +18 0 [] [] +18 0 [] [] +str_19 0 [] [] +19 0 [] [] +19 0 [] [] +[20] 0 [] [20] +['str_21','str_21'] 0 [] [NULL,NULL] +[22,22,22] 0 [] [22,22,22] +[23,23,23,23] 0 [] [23,23,23,23] +[24,24,24,24,24] 0 [] [24,24,24,24,24] +[25,25,25,25,25,25] 0 [] [25,25,25,25,25,25] +[26,26,26,26,26,26,26] 0 [] [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] 0 [] [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] 0 [] [29,29,29,29,29,29,29,29,29,29] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +[40] 0 [] [40] +41 0 [] [] +\N 0 [] [] +str_43 0 [] [] +[44,44,44,44,44] 0 [] [44,44,44,44,44] +45 0 [] [] +\N 0 [] [] +str_47 0 [] [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 0 [] [] +\N 0 [] [] +str_51 0 [] [] +[52,52,52] 0 [] [52,52,52] +53 0 [] [] +\N 0 [] [] +str_55 0 [] [] +[56,56,56,56,56,56,56] 0 [] [56,56,56,56,56,56,56] +57 0 [] [] +\N 0 [] [] +str_59 0 [] [] +[60] 0 [] [60] +61 0 [] [] +\N 0 [] [] +str_63 0 [] [] +[64,64,64,64,64] 0 [] [64,64,64,64,64] +65 0 [] [] +\N 0 [] [] +str_67 0 [] [] +[68,68,68,68,68,68,68,68,68] 0 [] [68,68,68,68,68,68,68,68,68] +69 0 [] [] +\N 0 [] [] +str_71 0 [] [] +[NULL,NULL,NULL] 0 [] [NULL,NULL,NULL] +73 0 [] [] +\N 0 [] [] +str_75 0 [] [] +[76,76,76,76,76,76,76] 0 [] [76,76,76,76,76,76,76] +77 0 [] [] +\N 0 [] [] +str_79 0 [] [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [20] +[] 0 [NULL,NULL] +[] 0 [22,22,22] +[] 0 [23,23,23,23] +[] 0 [24,24,24,24,24] +[] 0 [25,25,25,25,25,25] +[] 0 [26,26,26,26,26,26,26] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [28,28,28,28,28,28,28,28,28] +[] 0 [29,29,29,29,29,29,29,29,29,29] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [40] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [44,44,44,44,44] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [52,52,52] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [56,56,56,56,56,56,56] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [60] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [64,64,64,64,64] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [68,68,68,68,68,68,68,68,68] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [76,76,76,76,76,76,76] +[] 0 [] +[] 0 [] +[] 0 [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[2] [[0,1]] [[[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[7] [[0,1,2,3,4,5,6]] [[[],[],[],[],[],[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +36 +36 +20 +20 +0 +0 +2 +2 +20 +20 +2 +2 +20 +0 +0 0 \N [] +1 1 \N [] +2 2 \N [] +3 3 \N [] +4 4 \N [] +5 5 \N [] +6 6 \N [] +7 7 \N [] +8 8 \N [] +9 9 \N [] +str_10 \N str_10 [] +10 10 \N [] +10 10 \N [] +[[0,1]] \N \N [] +str_11 \N \N [] +str_11 \N str_11 [] +str_12 \N str_12 [] +12 12 \N [] +12 12 \N [] +str_13 \N str_13 [] +13 13 \N [] +13 13 \N [] +str_14 \N str_14 [] +14 14 \N [] +14 14 \N [] +str_15 \N str_15 [] +15 15 \N [] +15 15 \N [] +[[0,1,2,3,4,5,6]] \N \N [] +str_16 \N \N [] +str_16 \N str_16 [] +str_17 \N str_17 [] +17 17 \N [] +17 17 \N [] +str_18 \N str_18 [] +18 18 \N [] +18 18 \N [] +str_19 \N str_19 [] +19 19 \N [] +19 19 \N [] +[20] \N \N [20] +['str_21','str_21'] \N \N ['str_21','str_21'] +[22,22,22] \N \N [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +[40] \N \N [40] +41 41 \N [] +\N \N \N [] +str_43 \N str_43 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] +45 45 \N [] +\N \N \N [] +str_47 \N str_47 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 49 \N [] +\N \N \N [] +str_51 \N str_51 [] +[52,52,52] \N \N [52,52,52] +53 53 \N [] +\N \N \N [] +str_55 \N str_55 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] +57 57 \N [] +\N \N \N [] +str_59 \N str_59 [] +[60] \N \N [60] +61 61 \N [] +\N \N \N [] +str_63 \N str_63 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] +65 65 \N [] +\N \N \N [] +str_67 \N str_67 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] +69 69 \N [] +\N \N \N [] +str_71 \N str_71 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] +73 73 \N [] +\N \N \N [] +str_75 \N str_75 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] +77 77 \N [] +\N \N \N [] +str_79 \N str_79 [] +0 \N [] +1 \N [] +2 \N [] +3 \N [] +4 \N [] +5 \N [] +6 \N [] +7 \N [] +8 \N [] +9 \N [] +\N str_10 [] +10 \N [] +10 \N [] +\N \N [] +\N \N [] +\N str_11 [] +\N str_12 [] +12 \N [] +12 \N [] +\N str_13 [] +13 \N [] +13 \N [] +\N str_14 [] +14 \N [] +14 \N [] +\N str_15 [] +15 \N [] +15 \N [] +\N \N [] +\N \N [] +\N str_16 [] +\N str_17 [] +17 \N [] +17 \N [] +\N str_18 [] +18 \N [] +18 \N [] +\N str_19 [] +19 \N [] +19 \N [] +\N \N [20] +\N \N ['str_21','str_21'] +\N \N [22,22,22] +\N \N [23,23,23,23] +\N \N [24,24,24,24,24] +\N \N [25,25,25,25,25,25] +\N \N [26,26,26,26,26,26,26] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] +\N \N [29,29,29,29,29,29,29,29,29,29] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [40] +41 \N [] +\N \N [] +\N str_43 [] +\N \N [44,44,44,44,44] +45 \N [] +\N \N [] +\N str_47 [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] +\N \N [] +\N str_51 [] +\N \N [52,52,52] +53 \N [] +\N \N [] +\N str_55 [] +\N \N [56,56,56,56,56,56,56] +57 \N [] +\N \N [] +\N str_59 [] +\N \N [60] +61 \N [] +\N \N [] +\N str_63 [] +\N \N [64,64,64,64,64] +65 \N [] +\N \N [] +\N str_67 [] +\N \N [68,68,68,68,68,68,68,68,68] +69 \N [] +\N \N [] +\N str_71 [] +\N \N [NULL,NULL,NULL] +73 \N [] +\N \N [] +\N str_75 [] +\N \N [76,76,76,76,76,76,76] +77 \N [] +\N \N [] +\N str_79 [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +0 0 \N [] 0 [] +1 1 \N [] 0 [] +2 2 \N [] 0 [] +3 3 \N [] 0 [] +4 4 \N [] 0 [] +5 5 \N [] 0 [] +6 6 \N [] 0 [] +7 7 \N [] 0 [] +8 8 \N [] 0 [] +9 9 \N [] 0 [] +str_10 \N \N [] 0 [] +10 10 \N [] 0 [] +10 10 \N [] 0 [] +[[0,1]] \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_12 \N \N [] 0 [] +12 12 \N [] 0 [] +12 12 \N [] 0 [] +str_13 \N \N [] 0 [] +13 13 \N [] 0 [] +13 13 \N [] 0 [] +str_14 \N \N [] 0 [] +14 14 \N [] 0 [] +14 14 \N [] 0 [] +str_15 \N \N [] 0 [] +15 15 \N [] 0 [] +15 15 \N [] 0 [] +[[0,1,2,3,4,5,6]] \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_17 \N \N [] 0 [] +17 17 \N [] 0 [] +17 17 \N [] 0 [] +str_18 \N \N [] 0 [] +18 18 \N [] 0 [] +18 18 \N [] 0 [] +str_19 \N \N [] 0 [] +19 19 \N [] 0 [] +19 19 \N [] 0 [] +[20] \N \N [20] 1 [20] +['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] +[22,22,22] \N \N [22,22,22] 3 [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] 4 [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] 5 [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +[40] \N \N [40] 1 [40] +41 41 \N [] 0 [] +\N \N \N [] 0 [] +str_43 \N \N [] 0 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] 5 [44,44,44,44,44] +45 45 \N [] 0 [] +\N \N \N [] 0 [] +str_47 \N \N [] 0 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 49 \N [] 0 [] +\N \N \N [] 0 [] +str_51 \N \N [] 0 [] +[52,52,52] \N \N [52,52,52] 3 [52,52,52] +53 53 \N [] 0 [] +\N \N \N [] 0 [] +str_55 \N \N [] 0 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] +57 57 \N [] 0 [] +\N \N \N [] 0 [] +str_59 \N \N [] 0 [] +[60] \N \N [60] 1 [60] +61 61 \N [] 0 [] +\N \N \N [] 0 [] +str_63 \N \N [] 0 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] 5 [64,64,64,64,64] +65 65 \N [] 0 [] +\N \N \N [] 0 [] +str_67 \N \N [] 0 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] +69 69 \N [] 0 [] +\N \N \N [] 0 [] +str_71 \N \N [] 0 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] +73 73 \N [] 0 [] +\N \N \N [] 0 [] +str_75 \N \N [] 0 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] +77 77 \N [] 0 [] +\N \N \N [] 0 [] +str_79 \N \N [] 0 [] +0 \N [] 0 [] [] +1 \N [] 0 [] [] +2 \N [] 0 [] [] +3 \N [] 0 [] [] +4 \N [] 0 [] [] +5 \N [] 0 [] [] +6 \N [] 0 [] [] +7 \N [] 0 [] [] +8 \N [] 0 [] [] +9 \N [] 0 [] [] +\N \N [] 0 [] [] +10 \N [] 0 [] [] +10 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +12 \N [] 0 [] [] +12 \N [] 0 [] [] +\N \N [] 0 [] [] +13 \N [] 0 [] [] +13 \N [] 0 [] [] +\N \N [] 0 [] [] +14 \N [] 0 [] [] +14 \N [] 0 [] [] +\N \N [] 0 [] [] +15 \N [] 0 [] [] +15 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +17 \N [] 0 [] [] +17 \N [] 0 [] [] +\N \N [] 0 [] [] +18 \N [] 0 [] [] +18 \N [] 0 [] [] +\N \N [] 0 [] [] +19 \N [] 0 [] [] +19 \N [] 0 [] [] +\N \N [20] 1 [20] [NULL] +\N \N ['str_21','str_21'] 2 [NULL,NULL] ['str_21','str_21'] +\N \N [22,22,22] 3 [22,22,22] [NULL,NULL,NULL] +\N \N [23,23,23,23] 4 [23,23,23,23] [NULL,NULL,NULL,NULL] +\N \N [24,24,24,24,24] 5 [24,24,24,24,24] [NULL,NULL,NULL,NULL,NULL] +\N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] [NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [40] 1 [40] [NULL] +41 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [44,44,44,44,44] 5 [44,44,44,44,44] [NULL,NULL,NULL,NULL,NULL] +45 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [52,52,52] 3 [52,52,52] [NULL,NULL,NULL] +53 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +57 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [60] 1 [60] [NULL] +61 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [64,64,64,64,64] 5 [64,64,64,64,64] [NULL,NULL,NULL,NULL,NULL] +65 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +69 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] [NULL,NULL,NULL] +73 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +77 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +0 0 [] [] +1 0 [] [] +2 0 [] [] +3 0 [] [] +4 0 [] [] +5 0 [] [] +6 0 [] [] +7 0 [] [] +8 0 [] [] +9 0 [] [] +str_10 0 [] [] +10 0 [] [] +10 0 [] [] +[[0,1]] 0 [] [] +str_11 0 [] [] +str_11 0 [] [] +str_12 0 [] [] +12 0 [] [] +12 0 [] [] +str_13 0 [] [] +13 0 [] [] +13 0 [] [] +str_14 0 [] [] +14 0 [] [] +14 0 [] [] +str_15 0 [] [] +15 0 [] [] +15 0 [] [] +[[0,1,2,3,4,5,6]] 0 [] [] +str_16 0 [] [] +str_16 0 [] [] +str_17 0 [] [] +17 0 [] [] +17 0 [] [] +str_18 0 [] [] +18 0 [] [] +18 0 [] [] +str_19 0 [] [] +19 0 [] [] +19 0 [] [] +[20] 0 [] [20] +['str_21','str_21'] 0 [] [NULL,NULL] +[22,22,22] 0 [] [22,22,22] +[23,23,23,23] 0 [] [23,23,23,23] +[24,24,24,24,24] 0 [] [24,24,24,24,24] +[25,25,25,25,25,25] 0 [] [25,25,25,25,25,25] +[26,26,26,26,26,26,26] 0 [] [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] 0 [] [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] 0 [] [29,29,29,29,29,29,29,29,29,29] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +[40] 0 [] [40] +41 0 [] [] +\N 0 [] [] +str_43 0 [] [] +[44,44,44,44,44] 0 [] [44,44,44,44,44] +45 0 [] [] +\N 0 [] [] +str_47 0 [] [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 0 [] [] +\N 0 [] [] +str_51 0 [] [] +[52,52,52] 0 [] [52,52,52] +53 0 [] [] +\N 0 [] [] +str_55 0 [] [] +[56,56,56,56,56,56,56] 0 [] [56,56,56,56,56,56,56] +57 0 [] [] +\N 0 [] [] +str_59 0 [] [] +[60] 0 [] [60] +61 0 [] [] +\N 0 [] [] +str_63 0 [] [] +[64,64,64,64,64] 0 [] [64,64,64,64,64] +65 0 [] [] +\N 0 [] [] +str_67 0 [] [] +[68,68,68,68,68,68,68,68,68] 0 [] [68,68,68,68,68,68,68,68,68] +69 0 [] [] +\N 0 [] [] +str_71 0 [] [] +[NULL,NULL,NULL] 0 [] [NULL,NULL,NULL] +73 0 [] [] +\N 0 [] [] +str_75 0 [] [] +[76,76,76,76,76,76,76] 0 [] [76,76,76,76,76,76,76] +77 0 [] [] +\N 0 [] [] +str_79 0 [] [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [20] +[] 0 [NULL,NULL] +[] 0 [22,22,22] +[] 0 [23,23,23,23] +[] 0 [24,24,24,24,24] +[] 0 [25,25,25,25,25,25] +[] 0 [26,26,26,26,26,26,26] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [28,28,28,28,28,28,28,28,28] +[] 0 [29,29,29,29,29,29,29,29,29,29] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [40] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [44,44,44,44,44] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [52,52,52] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [56,56,56,56,56,56,56] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [60] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [64,64,64,64,64] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [68,68,68,68,68,68,68,68,68] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [76,76,76,76,76,76,76] +[] 0 [] +[] 0 [] +[] 0 [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[2] [[0,1]] [[[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[7] [[0,1,2,3,4,5,6]] [[[],[],[],[],[],[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +36 +36 +20 +20 +0 +0 +2 +2 +20 +20 +2 +2 +20 +0 +0 0 \N [] +1 1 \N [] +2 2 \N [] +3 3 \N [] +4 4 \N [] +5 5 \N [] +6 6 \N [] +7 7 \N [] +8 8 \N [] +9 9 \N [] +str_10 \N str_10 [] +10 10 \N [] +10 10 \N [] +[[0,1]] \N \N [] +str_11 \N \N [] +str_11 \N str_11 [] +str_12 \N str_12 [] +12 12 \N [] +12 12 \N [] +str_13 \N str_13 [] +13 13 \N [] +13 13 \N [] +str_14 \N str_14 [] +14 14 \N [] +14 14 \N [] +str_15 \N str_15 [] +15 15 \N [] +15 15 \N [] +[[0,1,2,3,4,5,6]] \N \N [] +str_16 \N \N [] +str_16 \N str_16 [] +str_17 \N str_17 [] +17 17 \N [] +17 17 \N [] +str_18 \N str_18 [] +18 18 \N [] +18 18 \N [] +str_19 \N str_19 [] +19 19 \N [] +19 19 \N [] +[20] \N \N [20] +['str_21','str_21'] \N \N ['str_21','str_21'] +[22,22,22] \N \N [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +\N \N \N [] +[40] \N \N [40] +41 41 \N [] +\N \N \N [] +str_43 \N str_43 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] +45 45 \N [] +\N \N \N [] +str_47 \N str_47 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 49 \N [] +\N \N \N [] +str_51 \N str_51 [] +[52,52,52] \N \N [52,52,52] +53 53 \N [] +\N \N \N [] +str_55 \N str_55 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] +57 57 \N [] +\N \N \N [] +str_59 \N str_59 [] +[60] \N \N [60] +61 61 \N [] +\N \N \N [] +str_63 \N str_63 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] +65 65 \N [] +\N \N \N [] +str_67 \N str_67 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] +69 69 \N [] +\N \N \N [] +str_71 \N str_71 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] +73 73 \N [] +\N \N \N [] +str_75 \N str_75 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] +77 77 \N [] +\N \N \N [] +str_79 \N str_79 [] +0 \N [] +1 \N [] +2 \N [] +3 \N [] +4 \N [] +5 \N [] +6 \N [] +7 \N [] +8 \N [] +9 \N [] +\N str_10 [] +10 \N [] +10 \N [] +\N \N [] +\N \N [] +\N str_11 [] +\N str_12 [] +12 \N [] +12 \N [] +\N str_13 [] +13 \N [] +13 \N [] +\N str_14 [] +14 \N [] +14 \N [] +\N str_15 [] +15 \N [] +15 \N [] +\N \N [] +\N \N [] +\N str_16 [] +\N str_17 [] +17 \N [] +17 \N [] +\N str_18 [] +18 \N [] +18 \N [] +\N str_19 [] +19 \N [] +19 \N [] +\N \N [20] +\N \N ['str_21','str_21'] +\N \N [22,22,22] +\N \N [23,23,23,23] +\N \N [24,24,24,24,24] +\N \N [25,25,25,25,25,25] +\N \N [26,26,26,26,26,26,26] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] +\N \N [29,29,29,29,29,29,29,29,29,29] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [40] +41 \N [] +\N \N [] +\N str_43 [] +\N \N [44,44,44,44,44] +45 \N [] +\N \N [] +\N str_47 [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] +\N \N [] +\N str_51 [] +\N \N [52,52,52] +53 \N [] +\N \N [] +\N str_55 [] +\N \N [56,56,56,56,56,56,56] +57 \N [] +\N \N [] +\N str_59 [] +\N \N [60] +61 \N [] +\N \N [] +\N str_63 [] +\N \N [64,64,64,64,64] +65 \N [] +\N \N [] +\N str_67 [] +\N \N [68,68,68,68,68,68,68,68,68] +69 \N [] +\N \N [] +\N str_71 [] +\N \N [NULL,NULL,NULL] +73 \N [] +\N \N [] +\N str_75 [] +\N \N [76,76,76,76,76,76,76] +77 \N [] +\N \N [] +\N str_79 [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +\N \N [] +0 0 \N [] 0 [] +1 1 \N [] 0 [] +2 2 \N [] 0 [] +3 3 \N [] 0 [] +4 4 \N [] 0 [] +5 5 \N [] 0 [] +6 6 \N [] 0 [] +7 7 \N [] 0 [] +8 8 \N [] 0 [] +9 9 \N [] 0 [] +str_10 \N \N [] 0 [] +10 10 \N [] 0 [] +10 10 \N [] 0 [] +[[0,1]] \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_11 \N \N [] 0 [] +str_12 \N \N [] 0 [] +12 12 \N [] 0 [] +12 12 \N [] 0 [] +str_13 \N \N [] 0 [] +13 13 \N [] 0 [] +13 13 \N [] 0 [] +str_14 \N \N [] 0 [] +14 14 \N [] 0 [] +14 14 \N [] 0 [] +str_15 \N \N [] 0 [] +15 15 \N [] 0 [] +15 15 \N [] 0 [] +[[0,1,2,3,4,5,6]] \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_16 \N \N [] 0 [] +str_17 \N \N [] 0 [] +17 17 \N [] 0 [] +17 17 \N [] 0 [] +str_18 \N \N [] 0 [] +18 18 \N [] 0 [] +18 18 \N [] 0 [] +str_19 \N \N [] 0 [] +19 19 \N [] 0 [] +19 19 \N [] 0 [] +[20] \N \N [20] 1 [20] +['str_21','str_21'] \N \N ['str_21','str_21'] 2 [NULL,NULL] +[22,22,22] \N \N [22,22,22] 3 [22,22,22] +[23,23,23,23] \N \N [23,23,23,23] 4 [23,23,23,23] +[24,24,24,24,24] \N \N [24,24,24,24,24] 5 [24,24,24,24,24] +[25,25,25,25,25,25] \N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] +[26,26,26,26,26,26,26] \N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] \N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] \N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] \N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +\N \N \N [] 0 [] +[40] \N \N [40] 1 [40] +41 41 \N [] 0 [] +\N \N \N [] 0 [] +str_43 \N \N [] 0 [] +[44,44,44,44,44] \N \N [44,44,44,44,44] 5 [44,44,44,44,44] +45 45 \N [] 0 [] +\N \N \N [] 0 [] +str_47 \N \N [] 0 [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] \N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 49 \N [] 0 [] +\N \N \N [] 0 [] +str_51 \N \N [] 0 [] +[52,52,52] \N \N [52,52,52] 3 [52,52,52] +53 53 \N [] 0 [] +\N \N \N [] 0 [] +str_55 \N \N [] 0 [] +[56,56,56,56,56,56,56] \N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] +57 57 \N [] 0 [] +\N \N \N [] 0 [] +str_59 \N \N [] 0 [] +[60] \N \N [60] 1 [60] +61 61 \N [] 0 [] +\N \N \N [] 0 [] +str_63 \N \N [] 0 [] +[64,64,64,64,64] \N \N [64,64,64,64,64] 5 [64,64,64,64,64] +65 65 \N [] 0 [] +\N \N \N [] 0 [] +str_67 \N \N [] 0 [] +[68,68,68,68,68,68,68,68,68] \N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] +69 69 \N [] 0 [] +\N \N \N [] 0 [] +str_71 \N \N [] 0 [] +[NULL,NULL,NULL] \N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] +73 73 \N [] 0 [] +\N \N \N [] 0 [] +str_75 \N \N [] 0 [] +[76,76,76,76,76,76,76] \N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] +77 77 \N [] 0 [] +\N \N \N [] 0 [] +str_79 \N \N [] 0 [] +0 \N [] 0 [] [] +1 \N [] 0 [] [] +2 \N [] 0 [] [] +3 \N [] 0 [] [] +4 \N [] 0 [] [] +5 \N [] 0 [] [] +6 \N [] 0 [] [] +7 \N [] 0 [] [] +8 \N [] 0 [] [] +9 \N [] 0 [] [] +\N \N [] 0 [] [] +10 \N [] 0 [] [] +10 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +12 \N [] 0 [] [] +12 \N [] 0 [] [] +\N \N [] 0 [] [] +13 \N [] 0 [] [] +13 \N [] 0 [] [] +\N \N [] 0 [] [] +14 \N [] 0 [] [] +14 \N [] 0 [] [] +\N \N [] 0 [] [] +15 \N [] 0 [] [] +15 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +17 \N [] 0 [] [] +17 \N [] 0 [] [] +\N \N [] 0 [] [] +18 \N [] 0 [] [] +18 \N [] 0 [] [] +\N \N [] 0 [] [] +19 \N [] 0 [] [] +19 \N [] 0 [] [] +\N \N [20] 1 [20] [NULL] +\N \N ['str_21','str_21'] 2 [NULL,NULL] ['str_21','str_21'] +\N \N [22,22,22] 3 [22,22,22] [NULL,NULL,NULL] +\N \N [23,23,23,23] 4 [23,23,23,23] [NULL,NULL,NULL,NULL] +\N \N [24,24,24,24,24] 5 [24,24,24,24,24] [NULL,NULL,NULL,NULL,NULL] +\N \N [25,25,25,25,25,25] 6 [25,25,25,25,25,25] [NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [26,26,26,26,26,26,26] 7 [26,26,26,26,26,26,26] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 8 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [28,28,28,28,28,28,28,28,28] 9 [28,28,28,28,28,28,28,28,28] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [29,29,29,29,29,29,29,29,29,29] 10 [29,29,29,29,29,29,29,29,29,29] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [40] 1 [40] [NULL] +41 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [44,44,44,44,44] 5 [44,44,44,44,44] [NULL,NULL,NULL,NULL,NULL] +45 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 9 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] ['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] +49 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [52,52,52] 3 [52,52,52] [NULL,NULL,NULL] +53 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [56,56,56,56,56,56,56] 7 [56,56,56,56,56,56,56] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +57 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [60] 1 [60] [NULL] +61 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [64,64,64,64,64] 5 [64,64,64,64,64] [NULL,NULL,NULL,NULL,NULL] +65 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [68,68,68,68,68,68,68,68,68] 9 [68,68,68,68,68,68,68,68,68] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +69 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [NULL,NULL,NULL] 3 [NULL,NULL,NULL] [NULL,NULL,NULL] +73 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [76,76,76,76,76,76,76] 7 [76,76,76,76,76,76,76] [NULL,NULL,NULL,NULL,NULL,NULL,NULL] +77 \N [] 0 [] [] +\N \N [] 0 [] [] +\N \N [] 0 [] [] +0 0 [] [] +1 0 [] [] +2 0 [] [] +3 0 [] [] +4 0 [] [] +5 0 [] [] +6 0 [] [] +7 0 [] [] +8 0 [] [] +9 0 [] [] +str_10 0 [] [] +10 0 [] [] +10 0 [] [] +[[0,1]] 0 [] [] +str_11 0 [] [] +str_11 0 [] [] +str_12 0 [] [] +12 0 [] [] +12 0 [] [] +str_13 0 [] [] +13 0 [] [] +13 0 [] [] +str_14 0 [] [] +14 0 [] [] +14 0 [] [] +str_15 0 [] [] +15 0 [] [] +15 0 [] [] +[[0,1,2,3,4,5,6]] 0 [] [] +str_16 0 [] [] +str_16 0 [] [] +str_17 0 [] [] +17 0 [] [] +17 0 [] [] +str_18 0 [] [] +18 0 [] [] +18 0 [] [] +str_19 0 [] [] +19 0 [] [] +19 0 [] [] +[20] 0 [] [20] +['str_21','str_21'] 0 [] [NULL,NULL] +[22,22,22] 0 [] [22,22,22] +[23,23,23,23] 0 [] [23,23,23,23] +[24,24,24,24,24] 0 [] [24,24,24,24,24] +[25,25,25,25,25,25] 0 [] [25,25,25,25,25,25] +[26,26,26,26,26,26,26] 0 [] [26,26,26,26,26,26,26] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[28,28,28,28,28,28,28,28,28] 0 [] [28,28,28,28,28,28,28,28,28] +[29,29,29,29,29,29,29,29,29,29] 0 [] [29,29,29,29,29,29,29,29,29,29] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +\N 0 [] [] +[40] 0 [] [40] +41 0 [] [] +\N 0 [] [] +str_43 0 [] [] +[44,44,44,44,44] 0 [] [44,44,44,44,44] +45 0 [] [] +\N 0 [] [] +str_47 0 [] [] +['str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48','str_48'] 0 [] [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +49 0 [] [] +\N 0 [] [] +str_51 0 [] [] +[52,52,52] 0 [] [52,52,52] +53 0 [] [] +\N 0 [] [] +str_55 0 [] [] +[56,56,56,56,56,56,56] 0 [] [56,56,56,56,56,56,56] +57 0 [] [] +\N 0 [] [] +str_59 0 [] [] +[60] 0 [] [60] +61 0 [] [] +\N 0 [] [] +str_63 0 [] [] +[64,64,64,64,64] 0 [] [64,64,64,64,64] +65 0 [] [] +\N 0 [] [] +str_67 0 [] [] +[68,68,68,68,68,68,68,68,68] 0 [] [68,68,68,68,68,68,68,68,68] +69 0 [] [] +\N 0 [] [] +str_71 0 [] [] +[NULL,NULL,NULL] 0 [] [NULL,NULL,NULL] +73 0 [] [] +\N 0 [] [] +str_75 0 [] [] +[76,76,76,76,76,76,76] 0 [] [76,76,76,76,76,76,76] +77 0 [] [] +\N 0 [] [] +str_79 0 [] [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [20] +[] 0 [NULL,NULL] +[] 0 [22,22,22] +[] 0 [23,23,23,23] +[] 0 [24,24,24,24,24] +[] 0 [25,25,25,25,25,25] +[] 0 [26,26,26,26,26,26,26] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [28,28,28,28,28,28,28,28,28] +[] 0 [29,29,29,29,29,29,29,29,29,29] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [40] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [44,44,44,44,44] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [52,52,52] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [56,56,56,56,56,56,56] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [60] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [64,64,64,64,64] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [68,68,68,68,68,68,68,68,68] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [NULL,NULL,NULL] +[] 0 [] +[] 0 [] +[] 0 [] +[] 0 [76,76,76,76,76,76,76] +[] 0 [] +[] 0 [] +[] 0 [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[2] [[0,1]] [[[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[7] [[0,1,2,3,4,5,6]] [[[],[],[],[],[],[],[]]] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] +[] [] [] diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 new file mode 100644 index 00000000000..0c123d5f6fe --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 @@ -0,0 +1,46 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; + +{% for engine in ['Memory', 'MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1'] -%} + +create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; + +insert into test select number, number from numbers(10); +insert into test select number, 'str_' || toString(number) from numbers(10, 10); +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(20, 10); +insert into test select number, NULL from numbers(30, 10); +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(40, 40); +insert into test select number, if(number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(10, 10); +insert into test select number, if(number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(10, 10); + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test order by id, d; +select d.Int8, d.Date, d.`Array(String)` from test order by id, d; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test order by id, d; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test order by id, d; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test order by id, d; + +drop table test; + +{% endfor -%} diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference new file mode 100644 index 00000000000..ca6c5dbba82 --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.reference @@ -0,0 +1,20 @@ +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +LowCardinality(String) +None +String +UInt64 +360000 +360000 +200000 +200000 +0 +0 +20000 +20000 +200000 +200000 +20000 +20000 +200000 +0 diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql new file mode 100644 index 00000000000..61dc8fca01a --- /dev/null +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_wide_merge_tree.sql @@ -0,0 +1,43 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +drop table if exists test; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; + +insert into test select number, number from numbers(100000) settings min_insert_block_size_rows=50000; +insert into test select number, 'str_' || toString(number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(200000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, NULL from numbers(300000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(400000, 400000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)), number) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000, 100000) settings min_insert_block_size_rows=50000; + +select distinct dynamicType(d) as type from test order by type; +select count() from test where dynamicType(d) == 'UInt64'; +select count() from test where d.UInt64 is not NULL; +select count() from test where dynamicType(d) == 'String'; +select count() from test where d.String is not NULL; +select count() from test where dynamicType(d) == 'Date'; +select count() from test where d.Date is not NULL; +select count() from test where dynamicType(d) == 'LowCardinality(String)'; +select count() from test where d.`LowCardinality(String)` is not NULL; +select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'; +select count() from test where not empty(d.`Array(Variant(String, UInt64))`); +select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'; +select count() from test where not empty(d.`Array(Array(Dynamic))`); +select count() from test where d is NULL; +select count() from test where not empty(d.`Tuple(a Array(Dynamic))`.a.String); + +select d, d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.UInt64, d.String, d.`Array(Variant(String, UInt64))` from test format Null; +select d.Int8, d.Date, d.`Array(String)` from test format Null; +select d, d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.UInt64, d.Date, d.`Array(Variant(String, UInt64))`, d.`Array(Variant(String, UInt64))`.size0, d.`Array(Variant(String, UInt64))`.UInt64, d.`Array(Variant(String, UInt64))`.String from test format Null; +select d, d.`Tuple(a UInt64, b String)`.a, d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Dynamic)`.`Variant(String, UInt64)`.UInt64, d.`Array(Dynamic)`.size0, d.`Array(Variant(String, UInt64))`.UInt64 from test format Null; +select d.`Array(Array(Dynamic))`.size1, d.`Array(Array(Dynamic))`.UInt64, d.`Array(Array(Dynamic))`.`Map(String, Tuple(a UInt64))`.values.a from test format Null; + +drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql index d2c787040e5..fb23e15738e 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_merge_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) 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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,22 +12,37 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql index f99bf771608..c098a3191e0 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal_compact_wide_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) 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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,22 +12,36 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql index be81596d043..17b1e451143 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_compact_merge_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) 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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,23 +12,36 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); -system start merges test; -optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; optimize table test final;; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference index d0d777a5a38..b0be05f07a2 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.reference @@ -1,28 +1,66 @@ -50000 DateTime -60000 Date -70000 Array(UInt16) -80000 String -100000 None -100000 UInt64 -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -70000 Array(UInt16) -100000 None -100000 UInt64 -190000 String -200000 Map(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -10000 Tuple(UInt64, UInt64) -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -260000 String -100000 None -100000 UInt64 -200000 Map(UInt64, UInt64) -270000 String +50000 DateTime false +60000 Date false +70000 Array(UInt16) false +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String false +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +30000 String false +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +80000 String true +100000 None false +100000 UInt64 false +200000 Map(UInt64, UInt64) false +--------------------- +10000 Tuple(UInt64, UInt64) true +50000 DateTime true +60000 Date true +70000 Array(UInt16) true +100000 None false +100000 UInt64 true +110000 String false +200000 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql index f6396af42a8..fd6c0109263 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical_wide_merge_tree.sql @@ -2,7 +2,7 @@ set allow_experimental_dynamic_type=1; drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) 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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); @@ -12,22 +12,36 @@ insert into test select number, toDate(number) from numbers(60000); insert into test select number, toDateTime(number) from numbers(50000); insert into test select number, NULL from numbers(100000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(200000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(10000); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(30000); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 b/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 index 96a854630ed..7d3bc371e36 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 +++ b/tests/queries/0_stateless/03037_dynamic_merges_small.reference.j2 @@ -1,112 +1,264 @@ -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String -5 DateTime -6 Date -7 Array(UInt16) -8 String -10 None -10 UInt64 -7 Array(UInt16) -10 None -10 UInt64 -19 String -7 Array(UInt16) -10 None -10 UInt64 -19 String -20 Map(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -1 Tuple(UInt64, UInt64) -10 None -10 UInt64 -20 Map(UInt64, UInt64) -26 String -10 None -10 UInt64 -20 Map(UInt64, UInt64) -27 String +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false +5 DateTime false +6 Date false +7 Array(UInt16) false +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String false +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +3 String false +5 DateTime true +6 Date true +7 Array(UInt16) true +8 String true +10 None false +10 UInt64 false +20 Map(UInt64, UInt64) false +--------------------- +1 Tuple(UInt64, UInt64) true +5 DateTime true +6 Date true +7 Array(UInt16) true +10 None false +10 UInt64 true +11 String false +20 Map(UInt64, UInt64) false diff --git a/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 index 263e92be403..3778399d0a4 100644 --- a/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 +++ b/tests/queries/0_stateless/03037_dynamic_merges_small.sql.j2 @@ -9,7 +9,7 @@ drop table if exists test; 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1', '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 table test (id UInt64, d Dynamic(max_types=3)) engine={{ engine }}; +create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; system stop merges test; insert into test select number, number from numbers(10); @@ -19,23 +19,37 @@ insert into test select number, toDate(number) from numbers(6); insert into test select number, toDateTime(number) from numbers(5); insert into test select number, NULL from numbers(10); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final;; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, map(number, number) from numbers(20); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system stop merges test; insert into test select number, tuple(number, number) from numbers(1); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); system start merges test; optimize table test final; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); + +system stop merges test; +insert into test select number, 'str_' || number from numbers(3); +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql index 81888946681..8ba192cb5db 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_horizontal.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql index ba58ca471a2..1ea7eefdd53 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_compact_vertical.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) 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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 index ae07c164074..3d7e8b60f73 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.reference.j2 @@ -1,84 +1,256 @@ -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None -2 Tuple(a Dynamic(max_types=3)):Date -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):String -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 UInt64:None -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 UInt64:None -2 Tuple(a Dynamic(max_types=3)):DateTime -3 Tuple(a Dynamic(max_types=3)):Array(UInt8) -5 Tuple(a Dynamic(max_types=3)):UInt64 -7 Tuple(a Dynamic(max_types=3)):String -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -13 Tuple(a Dynamic(max_types=3)):None -5 Tuple(a Dynamic(max_types=3)):UInt64 -10 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -10 UInt64:None -12 Tuple(a Dynamic(max_types=3)):String -13 Tuple(a Dynamic(max_types=3)):None +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +14 Tuple(a Dynamic(max_types=2)):DateTime true +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +14 Tuple(a Dynamic(max_types=2)):DateTime true +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String true +14 Tuple(a Dynamic(max_types=2)):DateTime false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +test +2 Tuple(a Dynamic(max_types=2)):DateTime false +2 Tuple(a Dynamic(max_types=2)):IPv4 false +4 Tuple(a Dynamic(max_types=2)):Date false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):DateTime true +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +4 Tuple(a Dynamic(max_types=2)):DateTime false +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 false +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +6 Tuple(a Dynamic(max_types=2)):DateTime true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +8 Tuple(a Dynamic(max_types=2)):DateTime false +13 Tuple(a Dynamic(max_types=2)):String false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false +--------------------- +2 Tuple(a Dynamic(max_types=2)):IPv4 true +4 Tuple(a Dynamic(max_types=2)):Date true +6 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +7 Tuple(a Dynamic(max_types=2)):UInt64 true +8 String:None false +13 Tuple(a Dynamic(max_types=2)):String true +14 Tuple(a Dynamic(max_types=2)):DateTime false +20 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +26 Tuple(a Dynamic(max_types=2)):None false +28 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 index 7828c2af49c..8682b6cef81 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_small.sql.j2 @@ -10,25 +10,41 @@ drop table if exists test; 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1', '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 table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +select 'test'; +create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; system stop merges test; -insert into test select number, number from numbers(10); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); +insert into test select number, number from numbers(20); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(20); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(10); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(20); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(10); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(40); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(8); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql index a53c5b0b2a5..c6a09036c30 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_horizontal.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference index 4be740f6050..a4c2df74a74 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.reference @@ -1,21 +1,63 @@ -16667 Tuple(a Dynamic(max_types=3)):Date -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):String -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 UInt64:None -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 UInt64:None -16667 Tuple(a Dynamic(max_types=3)):DateTime -33333 Tuple(a Dynamic(max_types=3)):Array(UInt8) -50000 Tuple(a Dynamic(max_types=3)):UInt64 -66667 Tuple(a Dynamic(max_types=3)):String -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -133333 Tuple(a Dynamic(max_types=3)):None -50000 Tuple(a Dynamic(max_types=3)):UInt64 -100000 Tuple(a Dynamic(max_types=3)):Tuple(UInt64) -100000 UInt64:None -116667 Tuple(a Dynamic(max_types=3)):String -133333 Tuple(a Dynamic(max_types=3)):None +6667 Tuple(a Dynamic(max_types=2)):DateTime false +13333 Tuple(a Dynamic(max_types=2)):IPv4 false +16667 Tuple(a Dynamic(max_types=2)):Date false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) false +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +140000 UInt64:None false +--------------------- +6667 Tuple(a Dynamic(max_types=2)):DateTime true +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +16667 Tuple(a Dynamic(max_types=2)):DateTime false +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 false +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +23334 Tuple(a Dynamic(max_types=2)):DateTime true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +40000 Tuple(a Dynamic(max_types=2)):DateTime false +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false +--------------------- +13333 Tuple(a Dynamic(max_types=2)):IPv4 true +16667 Tuple(a Dynamic(max_types=2)):Date true +33333 Tuple(a Dynamic(max_types=2)):Array(UInt8) true +33334 Tuple(a Dynamic(max_types=2)):UInt64 true +40000 String:None false +63334 Tuple(a Dynamic(max_types=2)):DateTime true +66666 Tuple(a Dynamic(max_types=2)):String false +100000 Tuple(a Dynamic(max_types=2)):Tuple(UInt64) false +133333 Tuple(a Dynamic(max_types=2)):None false +140000 UInt64:None false diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql index 4256b010ec0..c1964c45d98 100644 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges_wide_vertical.sql @@ -6,24 +6,39 @@ set allow_experimental_dynamic_type = 1; set enable_named_columns_in_function_tuple = 0; drop table if exists test;; -create table test (id UInt64, d Dynamic(max_types=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, lock_acquire_timeout_for_background_operations=600; +create table test (id UInt64, d Dynamic(max_types=2)) 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, lock_acquire_timeout_for_background_operations=600; system stop merges test; insert into test select number, number from numbers(100000); -insert into test select number, tuple(if(number % 2 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(100000); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(50000); +insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=2)) from numbers(100000); +insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=2)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(100000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(50000); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(200000); +system stop merges test; +insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(50000); +insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=2)) from numbers(200000); -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; system start merges test; optimize table test final; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type from test group by type order by count(), type; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; + +system stop merges test; +insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=2)) from numbers(40000); + +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; +system start merges test; +optimize table test final; +select '---------------------'; +select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=2))`.a) as type, isDynamicElementInSharedData(d.`Tuple(a Dynamic(max_types=2))`.a) as flag from test group by type, flag order by count(), type; drop table test; diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference index 2ec301b747b..9386548c74d 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.reference @@ -1,13 +1,13 @@ initial insert alter add column 1 -3 None +3 None false 0 0 \N \N \N 0 1 1 \N \N \N 0 2 2 \N \N \N 0 insert after alter add column 1 -4 String -4 UInt64 -7 None +4 String false +4 UInt64 false +7 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 @@ -24,147 +24,158 @@ insert after alter add column 1 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 alter modify column 1 -7 None -8 String +4 String true +4 UInt64 true +7 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 insert after alter modify column 1 -8 None -11 String +1 Date true +5 String true +5 UInt64 true +8 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 alter modify column 2 -8 None -11 String +1 Date true +5 String true +5 UInt64 true +8 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 insert after alter modify column 2 -1 Date -1 UInt64 -9 None -12 String +1 String false +1 UInt64 false +2 Date true +5 String true +5 UInt64 true +9 None false 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 19 19 \N \N \N \N 0 20 20 20 \N 20 \N 0 21 21 str_21 str_21 \N \N 0 22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 -1 Date -1 UInt64 -9 None -12 String +1 String false +1 UInt64 false +2 Date true +5 String true +5 UInt64 true +9 None false 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 -1 Date -1 UInt64 -12 None -12 String +1 String false +1 UInt64 false +2 Date true +5 String true +5 UInt64 true +12 None false 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql index 4ab700306d4..de05ba36177 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_compact_merge_tree.sql @@ -9,7 +9,7 @@ insert into test select number, number from numbers(3); select 'alter add column 1'; alter table test add column d Dynamic(max_types=3) settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter add column 1'; @@ -17,37 +17,37 @@ insert into test select number, number, number from numbers(3, 3); insert into test select number, number, 'str_' || toString(number) from numbers(6, 3); insert into test select number, number, NULL from numbers(9, 3); insert into test select number, number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL) from numbers(12, 3); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'alter modify column 1'; -alter table test modify column d Dynamic(max_types=1) settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +alter table test modify column d Dynamic(max_types=0) settings mutations_sync=1; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter modify column 1'; insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(15, 4); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'alter modify column 2'; -alter table test modify column d Dynamic(max_types=3) settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +alter table test modify column d Dynamic(max_types=2) settings mutations_sync=1; +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter modify column 2'; insert into test select number, number, multiIf(number % 4 == 0, number, number % 4 == 1, 'str_' || toString(number), number % 4 == 2, toDate(number), NULL) from numbers(19, 4); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, d, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'alter modify column 3'; alter table test modify column y Dynamic settings mutations_sync=1; -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; select 'insert after alter modify column 3'; insert into test select number, multiIf(number % 3 == 0, number, number % 3 == 1, 'str_' || toString(number), NULL), NULL from numbers(23, 3); -select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d); +select count(), dynamicType(d), isDynamicElementInSharedData(d) from test group by dynamicType(d), isDynamicElementInSharedData(d) order by count(), dynamicType(d); select x, y, y.UInt64, y.String, y.`Tuple(a UInt64)`.a, d.String, d.UInt64, d.Date, d.`Tuple(a UInt64)`.a from test order by x; drop table test; \ No newline at end of file diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference index c592528c3cd..d7123288280 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_memory.reference @@ -24,48 +24,28 @@ insert after alter add column 1 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 alter modify column 1 +4 String +4 UInt64 7 None -8 String 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -4 UInt64 -7 String +1 Date +5 String +5 UInt64 8 None 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 @@ -83,13 +63,37 @@ alter modify column 2 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 +18 18 1970-01-19 \N \N 1970-01-19 0 +alter modify column 2 1 Date +5 String 5 UInt64 -8 String +8 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 \N 16 \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 +insert after alter modify column 2 +2 Date +6 String +6 UInt64 9 None 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 @@ -107,17 +111,17 @@ insert after alter modify column 2 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 19 19 \N \N \N \N 0 20 20 20 \N 20 \N 0 21 21 str_21 str_21 \N \N 0 22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 -1 Date -5 UInt64 -8 String +2 Date +6 String +6 UInt64 9 None 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 @@ -135,17 +139,17 @@ alter modify column 3 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 -1 Date -5 UInt64 -8 String +2 Date +6 String +6 UInt64 12 None 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 @@ -163,9 +167,9 @@ insert after alter modify column 3 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference index 2ec301b747b..d7123288280 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1_wide_merge_tree.reference @@ -24,147 +24,152 @@ insert after alter add column 1 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 alter modify column 1 +4 String +4 UInt64 7 None -8 String 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 insert after alter modify column 1 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -alter modify column 2 -8 None -11 String -0 0 \N \N \N \N 0 -1 1 \N \N \N \N 0 -2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 -6 6 str_6 str_6 \N \N 0 -7 7 str_7 str_7 \N \N 0 -8 8 str_8 str_8 \N \N 0 -9 9 \N \N \N \N 0 -10 10 \N \N \N \N 0 -11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 -13 13 str_13 str_13 \N \N 0 -14 14 \N \N \N \N 0 -15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 -17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 -insert after alter modify column 2 1 Date -1 UInt64 -9 None -12 String +5 String +5 UInt64 +8 None 0 0 \N \N \N \N 0 1 1 \N \N \N \N 0 2 2 \N \N \N \N 0 -3 3 3 3 \N \N 0 -4 4 4 4 \N \N 0 -5 5 5 5 \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 6 6 str_6 str_6 \N \N 0 7 7 str_7 str_7 \N \N 0 8 8 str_8 str_8 \N \N 0 9 9 \N \N \N \N 0 10 10 \N \N \N \N 0 11 11 \N \N \N \N 0 -12 12 12 12 \N \N 0 +12 12 12 \N 12 \N 0 13 13 str_13 str_13 \N \N 0 14 14 \N \N \N \N 0 15 15 \N \N \N \N 0 -16 16 16 16 \N \N 0 +16 16 16 \N 16 \N 0 17 17 str_17 str_17 \N \N 0 -18 18 1970-01-19 1970-01-19 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 +alter modify column 2 +1 Date +5 String +5 UInt64 +8 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 \N 16 \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 +insert after alter modify column 2 +2 Date +6 String +6 UInt64 +9 None +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 \N 16 \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 \N \N 1970-01-19 0 19 19 \N \N \N \N 0 20 20 20 \N 20 \N 0 21 21 str_21 str_21 \N \N 0 22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 -1 Date -1 UInt64 +2 Date +6 String +6 UInt64 9 None -12 String 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 -1 Date -1 UInt64 +2 Date +6 String +6 UInt64 12 None -12 String 0 0 0 \N 0 \N \N \N 0 1 1 1 \N 0 \N \N \N 0 2 2 2 \N 0 \N \N \N 0 -3 3 3 \N 0 3 \N \N 0 -4 4 4 \N 0 4 \N \N 0 -5 5 5 \N 0 5 \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 6 6 6 \N 0 str_6 \N \N 0 7 7 7 \N 0 str_7 \N \N 0 8 8 8 \N 0 str_8 \N \N 0 9 9 9 \N 0 \N \N \N 0 10 10 10 \N 0 \N \N \N 0 11 11 11 \N 0 \N \N \N 0 -12 12 12 \N 0 12 \N \N 0 +12 12 12 \N 0 \N 12 \N 0 13 13 13 \N 0 str_13 \N \N 0 14 14 14 \N 0 \N \N \N 0 15 15 15 \N 0 \N \N \N 0 -16 16 16 \N 0 16 \N \N 0 +16 16 16 \N 0 \N 16 \N 0 17 17 17 \N 0 str_17 \N \N 0 -18 18 18 \N 0 1970-01-19 \N \N 0 +18 18 18 \N 0 \N \N 1970-01-19 0 19 19 19 \N 0 \N \N \N 0 20 20 20 \N 0 \N 20 \N 0 21 21 21 \N 0 str_21 \N \N 0 diff --git a/tests/queries/0_stateless/03041_dynamic_type_check_table.sh b/tests/queries/0_stateless/03041_dynamic_type_check_table.sh index c8bd533e253..da24b892cbd 100755 --- a/tests/queries/0_stateless/03041_dynamic_type_check_table.sh +++ b/tests/queries/0_stateless/03041_dynamic_type_check_table.sh @@ -13,7 +13,7 @@ function run() $CH_CLIENT -q "insert into test select number, number from numbers(3)" echo "alter add column" - $CH_CLIENT -q "alter table test add column d Dynamic(max_types=3) settings mutations_sync=1" + $CH_CLIENT -q "alter table test add column d Dynamic(max_types=2) settings mutations_sync=1" $CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)" $CH_CLIENT -q "select x, y, d, d.String, d.UInt64, d.\`Tuple(a UInt64)\`.a from test order by x" diff --git a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference index 0b76d30953e..370e6352657 100644 --- a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference +++ b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.reference @@ -1,35 +1,35 @@ 1 2024-01-01 Date 2 1704056400 Decimal(18, 3) -3 1 String -4 2 String +3 1 Float32 +4 2 Float64 1 2024-01-01 Date 1 2024-01-01 Date 2 1704056400 Decimal(18, 3) 2 1704056400 Decimal(18, 3) -3 1 String -3 1 String -4 2 String -4 2 String - -1 2024-01-01 String -1 2024-01-01 String -2 1704056400 String -2 1704056400 String -3 1 String -3 1 String -4 2 String -4 2 String +3 1 Float32 +3 1 Float32 +4 2 Float64 +4 2 Float64 1 2024-01-01 Date -1 2024-01-01 String -1 2024-01-01 String +1 2024-01-01 Date 2 1704056400 Decimal(18, 3) -2 1704056400 String -2 1704056400 String -3 1 String -3 1 String -3 1 String -4 2 String -4 2 String -4 2 String +2 1704056400 Decimal(18, 3) +3 1 Float32 +3 1 Float32 +4 2 Float64 +4 2 Float64 + +1 2024-01-01 Date +1 2024-01-01 Date +1 2024-01-01 Date +2 1704056400 Decimal(18, 3) +2 1704056400 Decimal(18, 3) +2 1704056400 Decimal(18, 3) +3 1 Float32 +3 1 Float32 +3 1 Float32 +4 2 Float64 +4 2 Float64 +4 2 Float64 diff --git a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql index ad5ea9512c6..71d5dd4abd1 100644 --- a/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql +++ b/tests/queries/0_stateless/03150_dynamic_type_mv_insert.sql @@ -1,5 +1,6 @@ SET allow_experimental_dynamic_type=1; +DROP TABLE IF EXISTS null_table; CREATE TABLE null_table ( n1 UInt8, @@ -7,9 +8,11 @@ CREATE TABLE null_table ) ENGINE = Null; +DROP VIEW IF EXISTS dummy_rmv; CREATE MATERIALIZED VIEW dummy_rmv TO to_table AS SELECT * FROM null_table; +DROP TABLE IF EXISTS to_table; CREATE TABLE to_table ( n1 UInt8, @@ -32,3 +35,7 @@ select ''; ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=10); INSERT INTO null_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; + +DROP TABLE null_table; +DROP VIEW dummy_rmv; +DROP TABLE to_table; diff --git a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference index d96fbf658d8..2d3b2f118f6 100644 --- a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference +++ b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.reference @@ -1,26 +1,26 @@ -1 2024-01-01 Date -2 1704056400 String -3 1 String -4 2 String +1 2024-01-01 Date false +2 1704056400 Decimal(18, 3) false +3 1 Float32 true +4 2 Float64 true -1 2024-01-01 Date -1 2024-01-01 Date -2 1704056400 Decimal(18, 3) -2 1704056400 String -3 1 Float32 -3 1 String -4 2 Float64 -4 2 String +1 2024-01-01 Date false +1 2024-01-01 Date false +2 1704056400 Decimal(18, 3) false +2 1704056400 Decimal(18, 3) false +3 1 Float32 false +3 1 Float32 false +4 2 Float64 false +4 2 Float64 false -1 2024-01-01 String -1 2024-01-01 String -1 2024-01-01 String -2 1704056400 String -2 1704056400 String -2 1704056400 String -3 1 String -3 1 String -3 1 String -4 2 String -4 2 String -4 2 String +1 2024-01-01 Date true +1 2024-01-01 Date true +1 2024-01-01 Date true +2 1704056400 Decimal(18, 3) true +2 1704056400 Decimal(18, 3) true +2 1704056400 Decimal(18, 3) true +3 1 Float32 true +3 1 Float32 true +3 1 Float32 true +4 2 Float64 true +4 2 Float64 true +4 2 Float64 true diff --git a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql index 632f3504fdb..e476d34a1db 100644 --- a/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql +++ b/tests/queries/0_stateless/03151_dynamic_type_scale_max_types.sql @@ -11,16 +11,16 @@ CREATE TABLE to_table ENGINE = MergeTree ORDER BY n1; INSERT INTO to_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); -SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; +SELECT *, dynamicType(n2), isDynamicElementInSharedData(n2) FROM to_table ORDER BY ALL; select ''; ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=5); INSERT INTO to_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); -SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; +SELECT *, dynamicType(n2), isDynamicElementInSharedData(n2) FROM to_table ORDER BY ALL; select ''; -ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=1); +ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=0); INSERT INTO to_table ( n1, n2 ) VALUES (1, '2024-01-01'), (2, toDateTime64('2024-01-01', 3, 'Asia/Istanbul')), (3, toFloat32(1)), (4, toFloat64(2)); -SELECT *, dynamicType(n2) FROM to_table ORDER BY ALL; +SELECT *, dynamicType(n2), isDynamicElementInSharedData(n2) FROM to_table ORDER BY ALL; ALTER TABLE to_table MODIFY COLUMN n2 Dynamic(max_types=500); -- { serverError UNEXPECTED_AST_STRUCTURE } diff --git a/tests/queries/0_stateless/03152_dynamic_type_simple.reference b/tests/queries/0_stateless/03152_dynamic_type_simple.reference index 5f243209ff3..e508bdd1990 100644 --- a/tests/queries/0_stateless/03152_dynamic_type_simple.reference +++ b/tests/queries/0_stateless/03152_dynamic_type_simple.reference @@ -7,7 +7,7 @@ string2 String \N None 42 Int64 42 Int64 string String -string String [1, 2] String +string String [1,2] Array(Int64) [1,2] Array(Int64) \N None ┌─d────────────────────────┬─dynamicType(d)─┬─d.Int64─┬─d.String─┬─────d.Date─┬─d.Float64─┬──────────d.DateTime─┬─d.Array(Int64)─┬─d.Array(String)──────────┠1. │ 42 │ Int64 │ 42 │ á´ºáµá´¸á´¸ │ á´ºáµá´¸á´¸ │ á´ºáµá´¸á´¸ │ á´ºáµá´¸á´¸ │ [] │ [] │ diff --git a/tests/queries/0_stateless/03152_dynamic_type_simple.sql b/tests/queries/0_stateless/03152_dynamic_type_simple.sql index fd5328faf15..ed24b213b1c 100644 --- a/tests/queries/0_stateless/03152_dynamic_type_simple.sql +++ b/tests/queries/0_stateless/03152_dynamic_type_simple.sql @@ -1,14 +1,17 @@ SET allow_experimental_dynamic_type=1; +DROP TABLE IF EXISTS test_max_types; CREATE TABLE test_max_types (d Dynamic(max_types=5)) ENGINE = Memory; INSERT INTO test_max_types VALUES ('string1'), (42), (3.14), ([1, 2]), (toDate('2021-01-01')), ('string2'); SELECT d, dynamicType(d) FROM test_max_types; SELECT ''; +DROP TABLE IF EXISTS test_nested_dynamic; CREATE TABLE test_nested_dynamic (d1 Dynamic, d2 Dynamic(max_types=2)) ENGINE = Memory; INSERT INTO test_nested_dynamic VALUES (NULL, 42), (42, 'string'), ('string', [1, 2]), ([1, 2], NULL); SELECT d1, dynamicType(d1), d2, dynamicType(d2) FROM test_nested_dynamic; +DROP TABLE IF EXISTS test_rapid_schema; CREATE TABLE test_rapid_schema (d Dynamic) ENGINE = Memory; INSERT INTO test_rapid_schema VALUES (42), ('string1'), (toDate('2021-01-01')), ([1, 2, 3]), (3.14), ('string2'), (toDateTime('2021-01-01 12:00:00')), (['array', 'of', 'strings']), (NULL), (toFloat64(42.42)); @@ -27,3 +30,8 @@ FROM FROM numbers(10000) ) ); + +DROP TABLE test_max_types; +DROP TABLE test_nested_dynamic; +DROP TABLE test_rapid_schema; + diff --git a/tests/queries/0_stateless/03153_dynamic_type_empty.sql b/tests/queries/0_stateless/03153_dynamic_type_empty.sql index 8e942fe6f6e..3a0c98e63ee 100644 --- a/tests/queries/0_stateless/03153_dynamic_type_empty.sql +++ b/tests/queries/0_stateless/03153_dynamic_type_empty.sql @@ -1,5 +1,7 @@ SET allow_experimental_dynamic_type=1; +DROP TABLE IF EXISTS test_null_empty; CREATE TABLE test_null_empty (d Dynamic) ENGINE = Memory; INSERT INTO test_null_empty VALUES ([]), ([1]), ([]), (['1']), ([]), (()),((1)), (()), (('1')), (()), ({}), ({1:2}), ({}), ({'1':'2'}), ({}); SELECT d, dynamicType(d) FROM test_null_empty; +DROP TABLE test_null_empty; diff --git a/tests/queries/0_stateless/03159_dynamic_type_all_types.sql b/tests/queries/0_stateless/03159_dynamic_type_all_types.sql index d302205ca23..fffea1bd0f5 100644 --- a/tests/queries/0_stateless/03159_dynamic_type_all_types.sql +++ b/tests/queries/0_stateless/03159_dynamic_type_all_types.sql @@ -6,7 +6,7 @@ SET allow_experimental_variant_type=1; SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t (d Dynamic(max_types=255)) ENGINE = Memory; +CREATE TABLE t (d Dynamic(max_types=254)) ENGINE = Memory; -- Integer types: signed and unsigned integers (UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256) INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); @@ -84,7 +84,7 @@ INSERT INTO t VALUES ([(1, (2, ['aa', 'bb']), [(3, 'cc'), (4, 'dd')]), (5, (6, [ SELECT dynamicType(d), d FROM t ORDER BY substring(dynamicType(d),1,1), length(dynamicType(d)), d; -CREATE TABLE t2 (d Dynamic(max_types=255)) ENGINE = Memory; +CREATE TABLE t2 (d Dynamic(max_types=254)) ENGINE = Memory; INSERT INTO t2 SELECT * FROM t; SELECT ''; diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh index 9b57e5c8718..b9bab2bd70b 100755 --- a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test" -$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 -q "create table test (id UInt64, d Dynamic(max_types=255)) engine=Memory" +$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 -q "create table test (id UInt64, d Dynamic(max_types=254)) engine=Memory" $CLICKHOUSE_CLIENT -q "insert into test select 0, NULL" $CLICKHOUSE_CLIENT -q "insert into test select 1, materialize(42)::UInt8" @@ -58,6 +58,6 @@ $CLICKHOUSE_CLIENT -q "insert into test select 47, materialize([[(20, 20), (50, $CLICKHOUSE_CLIENT -q "insert into test select 48, materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]])::MultiPolygon" $CLICKHOUSE_CLIENT -q "insert into test select 49, materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" -$CLICKHOUSE_CLIENT -q "select * from test format RowBinary" | $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --input-format RowBinary --structure 'id UInt64, d Dynamic(max_types=255)' -q "select d, dynamicType(d) from table order by id" +$CLICKHOUSE_CLIENT -q "select * from test format RowBinary" | $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --input-format RowBinary --structure 'id UInt64, d Dynamic(max_types=254)' -q "select d, dynamicType(d) from table order by id" $CLICKHOUSE_CLIENT -q "drop table test" diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference index 6d2c1334d6e..8d2470dea44 100644 --- a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.reference @@ -1,10 +1,10 @@ -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +1 +2 +3 +4 +0 +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql index 25f3bb0f4c8..939b49e1599 100644 --- a/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql +++ b/tests/queries/0_stateless/03200_memory_engine_alter_dynamic.sql @@ -1,6 +1,8 @@ set allow_experimental_dynamic_type=1; +drop table if exists test; create table test (d Dynamic) engine=Memory; insert into table test select * from numbers(5); -alter table test modify column d Dynamic(max_types=1); +alter table test modify column d Dynamic(max_types=0); select d.UInt64 from test settings enable_analyzer=1; select d.UInt64 from test settings enable_analyzer=0; +drop table test; From c1c32daf01bba08129dd17e2b3a108cd7e837528 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 9 Aug 2024 21:59:40 +0000 Subject: [PATCH 0716/1722] Fix tests --- .../Serializations/SerializationDynamic.cpp | 2 +- tests/queries/0_stateless/00000_test.sql | 43 ------------------- ...native_with_binary_encoded_types.reference | 4 +- ...ry_and_native_with_binary_encoded_types.sh | 6 +-- 4 files changed, 6 insertions(+), 49 deletions(-) delete mode 100644 tests/queries/0_stateless/00000_test.sql diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 67b29750948..5fadb6e4de4 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -130,7 +130,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( for (const auto & variant: variants) { if (variant->getName() != ColumnDynamic::getSharedVariantTypeName()) - encodeDataType(dynamic_state->variant_type, *stream); + encodeDataType(variant, *stream); } } else diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql deleted file mode 100644 index db9dd774484..00000000000 --- a/tests/queries/0_stateless/00000_test.sql +++ /dev/null @@ -1,43 +0,0 @@ -set allow_experimental_variant_type = 1; -set use_variant_as_common_type = 1; -set allow_experimental_dynamic_type = 1; -set enable_named_columns_in_function_tuple = 0; -drop table if exists test; -create table test (id UInt64, d Dynamic(max_types=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; - - -system stop merges test; -insert into test select number, number from numbers(10); -insert into test select number, tuple(if(number % 3 == 0, number, 'str_' || toString(number)))::Tuple(a Dynamic(max_types=3)) from numbers(10); -insert into test select number, tuple(if(number % 3 == 0, toDate(number), range(number % 10)))::Tuple(a Dynamic(max_types=3)) from numbers(5); -insert into test select number, multiIf(number % 5 == 0, tuple(if(number % 3 == 0, toDateTime(number), toIPv4(number)))::Tuple(a Dynamic(max_types=3)), number % 5 == 1 or number % 5 == 2, number, 'str_' || number) from numbers(10); - -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; -system start merges test; -optimize table test final; -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; - -system stop merges test; -insert into test select number, tuple(if(number % 3 == 0, toDateTime(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(5); -insert into test select number, tuple(if(number % 2 == 0, tuple(number), NULL))::Tuple(a Dynamic(max_types=3)) from numbers(20); - -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; -system start merges test; -optimize table test final; -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; - -system stop merges test; -insert into test select number, tuple(toDateTime(number))::Tuple(a Dynamic(max_types=3)) from numbers(4); - -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; -system start merges test; -optimize table test final; -select '---------------------'; -select count(), dynamicType(d) || ':' || dynamicType(d.`Tuple(a Dynamic(max_types=3))`.a) as type, isDynamicElementInSharedVariant(d.`Tuple(a Dynamic(max_types=3))`.a) as flag from test group by type, flag order by count(), type; - -drop table test; - diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference index 1ba147f9627..7de0804e0f2 100644 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference @@ -110,5 +110,5 @@ true Bool [{42:(1,[(2,{1:2})])}] Dynamic [{42:(1,[(2,{1:2})])}] Dynamic(max_types=10) [{42:(1,[(2,{1:2})])}] Dynamic(max_types=10) -[{42:(1,[(2,{1:2})])}] Dynamic(max_types=255) -[{42:(1,[(2,{1:2})])}] Dynamic(max_types=255) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=254) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=254) diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh index 0c585d36348..1e674a29072 100755 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test { - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" - $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --stacktrace --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --stacktrace --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" } test "materialize(42)::UInt8" @@ -66,4 +66,4 @@ test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" test "materialize(42::UInt32)::Variant(UInt32, String, Tuple(a UInt32, b Array(Map(String, String))))" test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic" test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=10)" -test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=255)" +test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=254)" From 384aedccaeece56456ad1e5ea17a8da4f56a69a4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sat, 10 Aug 2024 00:09:50 +0200 Subject: [PATCH 0717/1722] Update getLeastSupertype.h --- src/DataTypes/getLeastSupertype.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 5ea2b6417b2..8dd1685e6e9 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -54,5 +54,3 @@ DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); static std::vector interval_conversions = {1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4, 1}; } - -} From cebcc88b312b0702de9866c229e67097c738c4d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Aug 2024 00:18:54 +0200 Subject: [PATCH 0718/1722] Fix build --- src/IO/tests/gtest_s3_uri.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 7216c8077e3..c0bf7fcb28a 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -218,4 +218,5 @@ TEST(S3UriTest, versionIdChecks) } } +} #endif From c716315c3fdfd57719daf0f7f42b786afe6e68af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Aug 2024 00:20:46 +0200 Subject: [PATCH 0719/1722] Annotations --- .../0_stateless/00375_shard_group_uniq_array_of_string.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql index 8db91904a6a..c8a243d9b27 100644 --- a/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql +++ b/tests/queries/0_stateless/00375_shard_group_uniq_array_of_string.sql @@ -1,4 +1,4 @@ --- Tags: shard +-- Tags: shard, long DROP TABLE IF EXISTS group_uniq_str; CREATE TABLE group_uniq_str ENGINE = Memory AS SELECT number % 10 as id, toString(intDiv((number%10000), 10)) as v FROM system.numbers LIMIT 10000000; From 805a2e33bfedbdfc2393217e7b485761b1943e1d Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 10 Aug 2024 00:47:43 +0000 Subject: [PATCH 0720/1722] Fix unit tests --- src/Columns/tests/gtest_column_dynamic.cpp | 74 +++++++++++----------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index 5445bd525d9..de76261229d 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -7,7 +7,7 @@ using namespace DB; TEST(ColumnDynamic, CreateEmpty) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); ASSERT_TRUE(column->empty()); ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); ASSERT_EQ(column->getVariantInfo().variant_names.size(), 1); @@ -19,7 +19,7 @@ TEST(ColumnDynamic, CreateEmpty) TEST(ColumnDynamic, InsertDefault) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); column->insertDefault(); ASSERT_TRUE(column->size() == 1); ASSERT_EQ(column->getVariantInfo().variant_type->getName(), "Variant(SharedVariant)"); @@ -34,7 +34,7 @@ TEST(ColumnDynamic, InsertDefault) TEST(ColumnDynamic, InsertFields) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); column->insert(Field(42)); column->insert(Field(-42)); column->insert(Field("str1")); @@ -56,7 +56,7 @@ TEST(ColumnDynamic, InsertFields) ColumnDynamic::MutablePtr getDynamicWithManyVariants(size_t num_variants, Field tuple_element = Field(42)) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); for (size_t i = 0; i != num_variants; ++i) { Tuple tuple; @@ -136,7 +136,7 @@ TEST(ColumnDynamic, InsertFieldsOverflow2) ColumnDynamic::MutablePtr getInsertFromColumn(size_t num = 1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); for (size_t i = 0; i != num; ++i) { column_from->insert(Field(42)); @@ -170,13 +170,13 @@ void checkInsertFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDynami TEST(ColumnDynamic, InsertFrom1) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); checkInsertFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertFrom2) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -186,7 +186,7 @@ TEST(ColumnDynamic, InsertFrom2) TEST(ColumnDynamic, InsertFrom3) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -197,7 +197,7 @@ TEST(ColumnDynamic, InsertFrom3) TEST(ColumnDynamic, InsertFromOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -227,7 +227,7 @@ TEST(ColumnDynamic, InsertFromOverflow1) TEST(ColumnDynamic, InsertFromOverflow2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); @@ -251,7 +251,7 @@ TEST(ColumnDynamic, InsertFromOverflow3) column_from->insert(Field(42)); column_from->insert(Field(42.42)); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(41)); column_to->insertFrom(*column_from, 0); @@ -297,13 +297,13 @@ void checkInsertManyFrom(const ColumnDynamic::MutablePtr & column_from, ColumnDy TEST(ColumnDynamic, InsertManyFrom1) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); checkInsertManyFrom(getInsertFromColumn(), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertManyFrom2) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -313,7 +313,7 @@ TEST(ColumnDynamic, InsertManyFrom2) TEST(ColumnDynamic, InsertManyFrom3) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str")); @@ -324,7 +324,7 @@ TEST(ColumnDynamic, InsertManyFrom3) TEST(ColumnDynamic, InsertManyFromOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -361,7 +361,7 @@ TEST(ColumnDynamic, InsertManyFromOverflow1) TEST(ColumnDynamic, InsertManyFromOverflow2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); @@ -393,7 +393,7 @@ TEST(ColumnDynamic, InsertManyFromOverflow3) column_from->insert(Field(42)); column_from->insert(Field(42.42)); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(41)); column_to->insertManyFrom(*column_from, 0, 2); @@ -441,13 +441,13 @@ void checkInsertRangeFrom(const ColumnDynamic::MutablePtr & column_from, ColumnD TEST(ColumnDynamic, InsertRangeFrom1) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); checkInsertRangeFrom(getInsertFromColumn(2), column_to, "Variant(Float64, Int8, SharedVariant, String)", {"Float64", "Int8", "SharedVariant", "String"}, {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}); } TEST(ColumnDynamic, InsertRangeFrom2) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str1")); @@ -457,7 +457,7 @@ TEST(ColumnDynamic, InsertRangeFrom2) TEST(ColumnDynamic, InsertRangeFrom3) { - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insert(Field(42.42)); column_to->insert(Field("str1")); @@ -468,7 +468,7 @@ TEST(ColumnDynamic, InsertRangeFrom3) TEST(ColumnDynamic, InsertRangeFromOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -494,7 +494,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow1) TEST(ColumnDynamic, InsertRangeFromOverflow2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -516,7 +516,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow2) TEST(ColumnDynamic, InsertRangeFromOverflow3) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -539,7 +539,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow3) TEST(ColumnDynamic, InsertRangeFromOverflow4) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -561,7 +561,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow4) TEST(ColumnDynamic, InsertRangeFromOverflow5) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(42.42)); @@ -587,7 +587,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow5) TEST(ColumnDynamic, InsertRangeFromOverflow6) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(43)); column_from->insert(Field(44)); @@ -619,7 +619,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow6) TEST(ColumnDynamic, InsertRangeFromOverflow7) { - auto column_from = ColumnDynamic::create(3); + auto column_from = ColumnDynamic::create(2); column_from->insert(Field(42.42)); column_from->insert(Field("str1")); column_from->insert(Field(42)); @@ -629,7 +629,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow7) column_from->insert(Field("str2")); column_from->insert(Field(Array({Field(42)}))); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); column_to->insert(Field(42)); column_to->insertRangeFrom(*column_from, 0, 8); @@ -659,7 +659,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow7) TEST(ColumnDynamic, InsertRangeFromOverflow8) { - auto column_from = ColumnDynamic::create(3); + auto column_from = ColumnDynamic::create(2); column_from->insert(Field(42.42)); column_from->insert(Field("str1")); column_from->insert(Field(42)); @@ -669,7 +669,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow8) column_from->insert(Field("str2")); column_from->insert(Field(Array({Field(42)}))); - auto column_to = ColumnDynamic::create(3); + auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42)); column_from->insert(Field("str1")); @@ -711,7 +711,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow9) column_from->insert(Field("str2")); column_from->insert(Field(Array({Field(42)}))); - auto column_to = ColumnDynamic::create(3); + auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42)); column_to->insertRangeFrom(*column_from, 0, 9); @@ -743,7 +743,7 @@ TEST(ColumnDynamic, InsertRangeFromOverflow9) TEST(ColumnDynamic, SerializeDeserializeFromArena1) { - auto column = ColumnDynamic::create(255); + auto column = ColumnDynamic::create(254); column->insert(Field(42)); column->insert(Field(42.42)); column->insert(Field("str")); @@ -768,7 +768,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena1) TEST(ColumnDynamic, SerializeDeserializeFromArena2) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -781,7 +781,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) column_from->serializeValueIntoArena(2, arena, pos); column_from->serializeValueIntoArena(3, arena, pos); - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); pos = column_to->deserializeAndInsertFromArena(ref1.data); pos = column_to->deserializeAndInsertFromArena(pos); pos = column_to->deserializeAndInsertFromArena(pos); @@ -800,7 +800,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow1) { - auto column_from = ColumnDynamic::create(255); + auto column_from = ColumnDynamic::create(254); column_from->insert(Field(42)); column_from->insert(Field(42.42)); column_from->insert(Field("str")); @@ -846,7 +846,7 @@ TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow2) column_from->serializeValueIntoArena(3, arena, pos); column_from->serializeValueIntoArena(4, arena, pos); - auto column_to = ColumnDynamic::create(3); + auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42.42)); pos = column_to->deserializeAndInsertFromArena(ref1.data); pos = column_to->deserializeAndInsertFromArena(pos); @@ -882,7 +882,7 @@ TEST(ColumnDynamic, skipSerializedInArena) auto ref4 = column_from->serializeValueIntoArena(3, arena, pos); const char * end = ref4.data + ref4.size; - auto column_to = ColumnDynamic::create(255); + auto column_to = ColumnDynamic::create(254); pos = column_to->skipSerializedInArena(ref1.data); pos = column_to->skipSerializedInArena(pos); pos = column_to->skipSerializedInArena(pos); From c26b3cb4452931ee3bb3355b47dafb364744c9ab Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 02:27:23 +0000 Subject: [PATCH 0721/1722] handle the case of packed storage --- src/Storages/MergeTree/MutateTask.cpp | 11 ++++++++--- .../03161_lightweight_delete_projection.sql | 4 +++- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3d9f49c9a7a..0f0428287b6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1554,6 +1554,10 @@ private: removed_projections.insert(command.column_name); } + bool lightweight_delete_mode = ctx->updated_header.has(RowExistsColumn::name); + bool lightweight_delete_drop = lightweight_delete_mode + && ctx->data->getSettings()->lightweight_mutation_projection_mode == LightweightMutationProjectionMode::DROP; + const auto & projections = ctx->metadata_snapshot->getProjections(); for (const auto & projection : projections) { @@ -1561,10 +1565,11 @@ private: continue; bool need_recalculate = - ctx->materialized_projections.contains(projection.name) + (ctx->materialized_projections.contains(projection.name) || (!is_full_part_storage && ctx->source_part->hasProjection(projection.name) - && !ctx->source_part->hasBrokenProjection(projection.name)); + && !ctx->source_part->hasBrokenProjection(projection.name))) + && !lightweight_delete_drop; if (need_recalculate) { @@ -1572,7 +1577,7 @@ private: } else { - if (!ctx->updated_header.has(RowExistsColumn::name) && ctx->source_part->checksums.has(projection.getDirectoryName())) + if (!lightweight_delete_mode && ctx->source_part->checksums.has(projection.getDirectoryName())) entries_to_hardlink.insert(projection.getDirectoryName()); } } diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql index 0b05326e2c1..da6427cbf22 100644 --- a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -1,5 +1,7 @@ +-- For cloud version, should also consider min_bytes_for_full_part_storage since packed storage exists, +-- but for less redundancy, just let CI test the parameter. -SET max_threads = 1, lightweight_deletes_sync = 2, alter_sync = 2; +SET lightweight_deletes_sync = 2, alter_sync = 2; DROP TABLE IF EXISTS users_compact; From a837df164c48f0b8041b122adc8fa80a148629f8 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 02:37:42 +0000 Subject: [PATCH 0722/1722] fix squash related and projection collection --- src/Storages/MergeTree/MergeTask.cpp | 47 ++++++++++++------- .../0_stateless/02968_projection_merge.sql | 12 +++-- 2 files changed, 38 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 4133e47d4bc..3b117168e33 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -502,10 +502,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo { const auto & projection = *global_ctx->projections_to_rebuild[i]; Block block_to_squash = projection.calculate(block, global_ctx->context); - auto chunk = ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); - if (chunk) + ctx->projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); + auto squashed_chunk = Squashing::squash(ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); + if (squashed_chunk) { - auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(chunk.detachColumns()); + auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); tmp_part.finalize(); @@ -590,8 +591,23 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() return true; } - // finalize projections - calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); + /// finalize projections + // calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + const auto & projection = *global_ctx->projections_to_rebuild[i]; + auto & projection_squash_plan = ctx->projection_squashes[i]; + auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + if (squashed_chunk) + { + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( + *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + temp_part.finalize(); + temp_part.part->getDataPartStorage().commitTransaction(); + ctx->projection_parts[projection.name].emplace_back(std::move(temp_part.part)); + } + } ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) @@ -878,16 +894,6 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c ReadableSize(global_ctx->merge_list_element_ptr->bytes_read_uncompressed / elapsed_seconds)); } - const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; - /// Under throw mode, we still choose to drop projections due to backward compatibility since some - /// users might have projections before this change. - if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary - && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) - { - ctx->projections_iterator = ctx->tasks_for_projections.begin(); - return false; - } - for (const auto & projection : global_ctx->projections_to_merge) { MergeTreeData::DataPartsVector projection_parts = global_ctx->projections_to_merge_parts[projection->name]; @@ -1281,6 +1287,13 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const { + const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; + /// Under throw mode, we still choose to drop projections due to backward compatibility since some + /// users might have projections before this change. + if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary + && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) + return; + // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. const bool merge_may_reduce_rows = global_ctx->cleanup || @@ -1324,9 +1337,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe const auto & settings = global_ctx->context->getSettingsRef(); - for (auto projection : global_ctx->projections_to_rebuild) + for (const auto * projection : global_ctx->projections_to_rebuild) { - ctx->projection_squashes.emplace_back(projection->sample_block, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + ctx->projection_squashes.emplace_back(projection->sample_block.cloneEmpty(), settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } } diff --git a/tests/queries/0_stateless/02968_projection_merge.sql b/tests/queries/0_stateless/02968_projection_merge.sql index 07d40e30c2f..03488042a4e 100644 --- a/tests/queries/0_stateless/02968_projection_merge.sql +++ b/tests/queries/0_stateless/02968_projection_merge.sql @@ -11,7 +11,8 @@ CREATE TABLE tp ) ) ENGINE = ReplacingMergeTree -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number%3, 1 FROM numbers(3); INSERT INTO tp SELECT number%3, 2 FROM numbers(3); @@ -39,7 +40,8 @@ CREATE TABLE tp ) ) ENGINE = CollapsingMergeTree(sign) -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number % 3, 1, 1 FROM numbers(3); INSERT INTO tp SELECT number % 3, 1, -1 FROM numbers(3); @@ -70,7 +72,8 @@ CREATE TABLE tp ) ) ENGINE = VersionedCollapsingMergeTree(sign,version) -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number % 3, 1, -1, 0 FROM numbers(3); INSERT INTO tp SELECT number % 3, 2, 1, 1 FROM numbers(3); @@ -97,7 +100,8 @@ CREATE TABLE tp ) ) ENGINE = MergeTree -ORDER BY type; +ORDER BY type +SETTINGS deduplicate_merge_projection_mode = 'rebuild'; INSERT INTO tp SELECT number % 3, 1 FROM numbers(3); INSERT INTO tp SELECT number % 3, 2 FROM numbers(3); From 9b9fff4232d80e579b1d23ced8bfbb1b2c5e2147 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Sat, 10 Aug 2024 08:48:08 +0200 Subject: [PATCH 0723/1722] Push CI From e582118544f3c49c3c6600ac8fa252151714d25f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 10 Aug 2024 13:09:40 +0200 Subject: [PATCH 0724/1722] review changes --- .../functions/type-conversion-functions.md | 916 +++++++++++++++--- 1 file changed, 801 insertions(+), 115 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 8e72fea7fdb..5c06e72f977 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -64,9 +64,8 @@ toInt8(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -251,9 +250,8 @@ toInt8OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int8` is unsuccessful. [Int8](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -314,9 +312,8 @@ toInt16(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -501,9 +498,8 @@ toInt16OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int16` is unsuccessful. [Int16](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -564,9 +560,8 @@ toInt32(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -750,9 +745,8 @@ toInt32OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int32` is unsuccessful. [Int32](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -813,9 +807,8 @@ toInt64(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported types: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1000,9 +993,8 @@ toInt64OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int64` is unsuccessful. [Int64](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1063,9 +1055,8 @@ toInt128(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1312,9 +1303,8 @@ toInt256(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1498,9 +1488,8 @@ toInt256OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `Int256` is unsuccessful. [Int256](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf` @@ -1561,9 +1550,8 @@ toUInt8(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1748,9 +1736,8 @@ toUInt8OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt8` is unsuccessful. [UInt8](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1811,9 +1798,8 @@ toUInt16(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -1998,9 +1984,8 @@ toUInt16OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt16` is unsuccessful. [UInt16](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2061,9 +2046,8 @@ toUInt32(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2249,9 +2233,8 @@ toUInt32OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt32` is unsuccessful. [UInt32](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2312,9 +2295,8 @@ toUInt64(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported types: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2499,9 +2481,8 @@ toUInt64OrDefault(expr[, default]) - `defauult` (optional) — The default value to return if parsing to type `UInt64` is unsuccessful. [UInt64](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2562,9 +2543,8 @@ toUInt128(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2811,9 +2791,8 @@ toUInt256(expr) - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Unsupported arguments: - String representations of Float32/64 values, including `NaN` and `Inf`. @@ -2997,9 +2976,8 @@ toUInt256OrDefault(expr[, default]) - `default` (optional) — The default value to return if parsing to type `UInt256` is unsuccessful. [UInt256](../data-types/int-uint.md). Supported arguments: -- Values of type (U)Int8/16/32/64/128/256. +- Values or string representations of type (U)Int8/16/32/64/128/256. - Values of type Float32/64. -- String representations of (U)Int8/16/32/128/256. Arguments for which the default value is returned: - String representations of Float32/64 values, including `NaN` and `Inf` @@ -3555,7 +3533,7 @@ toDecimal32(expr, S) **Arguments** - `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). Supported arguments: - Values or string representations of type (U)Int8/16/32/64/128/256. @@ -3566,7 +3544,7 @@ Unsupported arguments: - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). Excessive digits in the integer part will lead to an exception. ::: @@ -3619,20 +3597,20 @@ toDecimal32OrZero(expr, S) **Arguments** - `expr` — A String representation of a number. [String](../data-types/string.md). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). Supported arguments: - String representations of type (U)Int8/16/32/64/128/256. - String representations of type Float32/64. Unsupported arguments: -- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of Float32/64 values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrZero('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). -Excessive digits in the integer part will lead to an exception. +Excessive digits in the integer part will lead to an error. ::: **Returned value** @@ -3645,8 +3623,10 @@ Query: ``` sql SELECT - toDecimal32OrZero(toString(-1.111), 5) AS val, - toTypeName(val) + toDecimal32OrZero(toString(-1.111), 5) AS a, + toTypeName(a), + toDecimal32OrZero(toString('Inf'), 5) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3655,26 +3635,10 @@ Result: ```response Row 1: ────── -val: -1.111 -toTypeName(val): Decimal(9, 5) -``` - -Query: - -``` sql -SELECT - toDecimal32OrZero(toString(-1.111), 2) AS val, - toTypeName(val) -FORMAT Vertical; -``` - -Result: - -```response -Row 1: -────── -val: -1.11 -toTypeName(val): Decimal(9, 2) +a: -1.111 +toTypeName(a): Decimal(9, 5) +b: 0 +toTypeName(b): Decimal(9, 5) ``` **See also** @@ -3696,20 +3660,20 @@ toDecimal32OrNull(expr, S) **Arguments** - `expr` — A String representation of a number. [String](../data-types/string.md). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). Supported arguments: - String representations of type (U)Int8/16/32/64/128/256. - String representations of type Float32/64. Unsupported arguments: -- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of Float32/64 values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrNull('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). -Excessive digits in the integer part will lead to an exception. +Excessive digits in the integer part will lead to an error. ::: **Returned value** @@ -3722,8 +3686,10 @@ Query: ``` sql SELECT - toDecimal32OrNull(toString(-1.111), 5) AS val, - toTypeName(val) + toDecimal32OrNull(toString(-1.111), 5) AS a, + toTypeName(a), + toDecimal32OrNull(toString('Inf'), 5) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3732,26 +3698,10 @@ Result: ```response Row 1: ────── -val: -1.111 -toTypeName(val): Nullable(Decimal(9, 5)) -``` - -Query: - -``` sql -SELECT - toDecimal32OrNull(toString(-1.111), 2) AS val, - toTypeName(val) -FORMAT Vertical; -``` - -Result: - -```response -Row 1: -────── -val: -1.11 -toTypeName(val): Nullable(Decimal(9, 2)) +a: -1.111 +toTypeName(a): Nullable(Decimal(9, 5)) +b: á´ºáµá´¸á´¸ +toTypeName(b): Nullable(Decimal(9, 5)) ``` **See also** @@ -3773,21 +3723,21 @@ toDecimal32OrDefault(expr, S[, default]) **Arguments** - `expr` — A String representation of a number. [String](../data-types/string.md). -- `S` — Scale parameter from [ 1 : 9 ] specifying how many decimal digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). -- `default` (optional) — The default value to return if parsing to type `Decimal32(S)` is unsuccessful. [Decimal32(S](../data-types/decimal.md). +- `S` — Scale parameter between 0 and 9, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal32(S)` is unsuccessful. [Decimal32(S)](../data-types/decimal.md). Supported arguments: - String representations of type (U)Int8/16/32/64/128/256. - String representations of type Float32/64. Unsupported arguments: -- String representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of Float32/64 values `NaN` and `Inf`. - String representations of binary and hexadecimal values, e.g. `SELECT toDecimal32OrDefault('0xc0fe', 1);`. :::note -Integer overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal32`: `( -1 * 10^(9 - S), 1 * 10^(9 - S) )`. Excessive digits in a fraction are discarded (not rounded). -Excessive digits in the integer part will lead to an exception. +Excessive digits in the integer part will lead to an error. ::: **Returned value** @@ -3800,8 +3750,10 @@ Query: ``` sql SELECT - toDecimal32OrDefault(toString(-1.111), 5) AS val, - toTypeName(val) + toDecimal32OrDefault(toString(0.0001), 5) AS a, + toTypeName(a), + toDecimal32OrDefault('Inf', 0, CAST('-1', 'Decimal32(0)')) AS b, + toTypeName(b) FORMAT Vertical; ``` @@ -3810,16 +3762,125 @@ Result: ```response Row 1: ────── -val: -1.111 -toTypeName(val): Decimal(9, 5) +a: 0.0001 +toTypeName(a): Decimal(9, 5) +b: -1 +toTypeName(b): Decimal(9, 0) ``` +**See also** + +- [`toDecimal32`](#todecimal32). +- [`toDecimal32OrZero`](#todecimal32orzero). +- [`toDecimal32OrNull`](#todecimal32ornull). + +## toDecimal64 + +Converts an input value to a value of type [`Decimal(18, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. + +**Syntax** + +```sql +toDecimal64(expr, S) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(18, S)`. [Decimal64(S)](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toDecimal64(2, 1) AS a, toTypeName(a) AS type_a, + toDecimal64(4.2, 2) AS b, toTypeName(b) AS type_b, + toDecimal64('4.2', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 2 +type_a: Decimal(18, 1) +b: 4.2 +type_b: Decimal(18, 2) +c: 4.2 +type_c: Decimal(18, 3) +``` + +**See also** + +- [`toDecimal64OrZero`](#todecimal64orzero). +- [`toDecimal64OrNull`](#todecimal64ornull). +- [`toDecimal64OrDefault`](#todecimal64ordefault). + +## toDecimal64OrZero + +Like [`toDecimal64`](#todecimal64), this function converts an input value to a value of type [Decimal(18, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal64OrZero(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64OrZero('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(18, S)` if successful, otherwise `0` with `S` decimal places. [Decimal64(S)](../data-types/decimal.md). + +**Example** + Query: ``` sql SELECT - toDecimal32OrDefault(toString(-1.111), 2) AS val, - toTypeName(val) + toDecimal64OrZero(toString(0.0001), 18) AS a, + toTypeName(a), + toDecimal64OrZero(toString('Inf'), 18) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3828,16 +3889,61 @@ Result: ```response Row 1: ────── -val: -1.11 -toTypeName(val): Decimal(9, 2) +a: 0.0001 +toTypeName(a): Decimal(18, 18) +b: 0 +toTypeName(b): Decimal(18, 18) ``` +**See also** + +- [`toDecimal64`](#todecimal64). +- [`toDecimal64OrNull`](#todecimal64ornull). +- [`toDecimal64OrDefault`](#todecimal64ordefault). + +## toDecimal64OrNull + +Like [`toDecimal64`](#todecimal64), this function converts an input value to a value of type [Nullable(Decimal(18, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal64OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64OrNull('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(18, S))` if successful, otherwise value `NULL` of the same type. [Decimal64(S)](../data-types/decimal.md). + +**Examples** + Query: ``` sql SELECT - toDecimal32OrDefault('Inf', 2, CAST('0', 'Decimal32(2)')) AS val, - toTypeName(val) + toDecimal64OrNull(toString(0.0001), 18) AS a, + toTypeName(a), + toDecimal64OrNull(toString('Inf'), 18) as b, + toTypeName(b) FORMAT Vertical; ``` @@ -3846,10 +3952,590 @@ Result: ```response Row 1: ────── -val: 0 -toTypeName(val): Decimal(9, 2) +a: 0.0001 +toTypeName(a): Nullable(Decimal(18, 18)) +b: á´ºáµá´¸á´¸ +toTypeName(b): Nullable(Decimal(18, 18)) ``` +**See also** + +- [`toDecimal64`](#todecimal64). +- [`toDecimal64OrZero`](#todecimal64orzero). +- [`toDecimal64OrDefault`](#todecimal64ordefault). + +## toDecimal64OrDefault + +Like [`toDecimal64`](#todecimal64), this function converts an input value to a value of type [Decimal(18, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal64OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 18, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal64(S)` is unsuccessful. [Decimal64(S)](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal64OrDefault('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal64`: `( -1 * 10^(18 - S), 1 * 10^(18 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(18, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal64(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal64OrDefault(toString(0.0001), 18) AS a, + toTypeName(a), + toDecimal64OrDefault('Inf', 0, CAST('-1', 'Decimal64(0)')) AS b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.0001 +toTypeName(a): Decimal(18, 18) +b: -1 +toTypeName(b): Decimal(18, 0) +``` + +**See also** + +- [`toDecimal64`](#todecimal64). +- [`toDecimal64OrZero`](#todecimal64orzero). +- [`toDecimal64OrNull`](#todecimal64ornull). + +## toDecimal128 + +Converts an input value to a value of type [`Decimal(38, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. + +**Syntax** + +```sql +toDecimal128(expr, S) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(38, S)`. [Decimal128(S)](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toDecimal128(99, 1) AS a, toTypeName(a) AS type_a, + toDecimal128(99.67, 2) AS b, toTypeName(b) AS type_b, + toDecimal128('99.67', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 99 +type_a: Decimal(38, 1) +b: 99.67 +type_b: Decimal(38, 2) +c: 99.67 +type_c: Decimal(38, 3) +``` + +**See also** + +- [`toDecimal128OrZero`](#todecimal128orzero). +- [`toDecimal128OrNull`](#todecimal128ornull). +- [`toDecimal128OrDefault`](#todecimal128ordefault). + +## toDecimal128OrZero + +Like [`toDecimal128`](#todecimal128), this function converts an input value to a value of type [Decimal(38, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal128OrZero(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128OrZero('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(38, S)` if successful, otherwise `0` with `S` decimal places. [Decimal128(S)](../data-types/decimal.md). + +**Example** + +Query: + +``` sql +SELECT + toDecimal128OrZero(toString(0.0001), 38) AS a, + toTypeName(a), + toDecimal128OrZero(toString('Inf'), 38) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.0001 +toTypeName(a): Decimal(38, 38) +b: 0 +toTypeName(b): Decimal(38, 38) +``` + +**See also** + +- [`toDecimal128`](#todecimal128). +- [`toDecimal128OrNull`](#todecimal128ornull). +- [`toDecimal128OrDefault`](#todecimal128ordefault). + +## toDecimal128OrNull + +Like [`toDecimal128`](#todecimal128), this function converts an input value to a value of type [Nullable(Decimal(38, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal128OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128OrNull('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(38, S))` if successful, otherwise value `NULL` of the same type. [Decimal128(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal128OrNull(toString(1/42), 38) AS a, + toTypeName(a), + toDecimal128OrNull(toString('Inf'), 38) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Nullable(Decimal(38, 38)) +b: á´ºáµá´¸á´¸ +toTypeName(b): Nullable(Decimal(38, 38)) +``` + +**See also** + +- [`toDecimal128`](#todecimal128). +- [`toDecimal128OrZero`](#todecimal128orzero). +- [`toDecimal128OrDefault`](#todecimal128ordefault). + +## toDecimal128OrDefault + +Like [`toDecimal128`](#todecimal128), this function converts an input value to a value of type [Decimal(38, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal128OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 38, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal128(S)` is unsuccessful. [Decimal128(S)](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal128OrDefault('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal128`: `( -1 * 10^(38 - S), 1 * 10^(38 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(38, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal128(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal128OrDefault(toString(1/42), 18) AS a, + toTypeName(a), + toDecimal128OrDefault('Inf', 0, CAST('-1', 'Decimal128(0)')) AS b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Decimal(38, 18) +b: -1 +toTypeName(b): Decimal(38, 0) +``` + +**See also** + +- [`toDecimal128`](#todecimal128). +- [`toDecimal128OrZero`](#todecimal128orzero). +- [`toDecimal128OrNull`](#todecimal128ornull). + +## toDecimal256 + +Converts an input value to a value of type [`Decimal(76, S)`](../data-types/decimal.md) with scale of `S`. Throws an exception in case of an error. + +**Syntax** + +```sql +toDecimal256(expr, S) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string representation of a number. [Expression](../syntax.md/#syntax-expressions). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- Values or string representations of type (U)Int8/16/32/64/128/256. +- Values or string representations of type Float32/64. + +Unsupported arguments: +- Values or string representations of Float32/64 values `NaN` and `Inf` (case-insensitive). +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an exception. +::: + +**Returned value** + +- Value of type `Decimal(76, S)`. [Decimal256(S)](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toDecimal256(99, 1) AS a, toTypeName(a) AS type_a, + toDecimal256(99.67, 2) AS b, toTypeName(b) AS type_b, + toDecimal256('99.67', 3) AS c, toTypeName(c) AS type_c +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 99 +type_a: Decimal(76, 1) +b: 99.67 +type_b: Decimal(76, 2) +c: 99.67 +type_c: Decimal(76, 3) +``` + +**See also** + +- [`toDecimal256OrZero`](#todecimal256orzero). +- [`toDecimal256OrNull`](#todecimal256ornull). +- [`toDecimal256OrDefault`](#todecimal256ordefault). + +## toDecimal256OrZero + +Like [`toDecimal256`](#todecimal256), this function converts an input value to a value of type [Decimal(76, S)](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal256OrZero(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256OrZero('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(76, S)` if successful, otherwise `0` with `S` decimal places. [Decimal256(S)](../data-types/decimal.md). + +**Example** + +Query: + +``` sql +SELECT + toDecimal256OrZero(toString(0.0001), 76) AS a, + toTypeName(a), + toDecimal256OrZero(toString('Inf'), 76) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.0001 +toTypeName(a): Decimal(76, 76) +b: 0 +toTypeName(b): Decimal(76, 76) +``` + +**See also** + +- [`toDecimal256`](#todecimal256). +- [`toDecimal256OrNull`](#todecimal256ornull). +- [`toDecimal256OrDefault`](#todecimal256ordefault). + +## toDecimal256OrNull + +Like [`toDecimal256`](#todecimal256), this function converts an input value to a value of type [Nullable(Decimal(76, S))](../data-types/decimal.md) but returns `0` in case of an error. + +**Syntax** + +```sql +toDecimal256OrNull(expr, S) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256OrNull('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Nullable(Decimal(76, S))` if successful, otherwise value `NULL` of the same type. [Decimal256(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal256OrNull(toString(1/42), 76) AS a, + toTypeName(a), + toDecimal256OrNull(toString('Inf'), 76) as b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Nullable(Decimal(76, 76)) +b: á´ºáµá´¸á´¸ +toTypeName(b): Nullable(Decimal(76, 76)) +``` + +**See also** + +- [`toDecimal256`](#todecimal256). +- [`toDecimal256OrZero`](#todecimal256orzero). +- [`toDecimal256OrDefault`](#todecimal256ordefault). + +## toDecimal256OrDefault + +Like [`toDecimal256`](#todecimal256), this function converts an input value to a value of type [Decimal(76, S)](../data-types/decimal.md) but returns the default value in case of an error. + +**Syntax** + +```sql +toDecimal256OrDefault(expr, S[, default]) +``` + +**Arguments** + +- `expr` — A String representation of a number. [String](../data-types/string.md). +- `S` — Scale parameter between 0 and 76, specifying how many digits the fractional part of a number can have. [UInt8](../data-types/int-uint.md). +- `default` (optional) — The default value to return if parsing to type `Decimal256(S)` is unsuccessful. [Decimal256(S)](../data-types/decimal.md). + +Supported arguments: +- String representations of type (U)Int8/16/32/64/128/256. +- String representations of type Float32/64. + +Unsupported arguments: +- String representations of Float32/64 values `NaN` and `Inf`. +- String representations of binary and hexadecimal values, e.g. `SELECT toDecimal256OrDefault('0xc0fe', 1);`. + +:::note +An overflow can occur if the value of `expr` exceeds the bounds of `Decimal256`: `( -1 * 10^(76 - S), 1 * 10^(76 - S) )`. +Excessive digits in a fraction are discarded (not rounded). +Excessive digits in the integer part will lead to an error. +::: + +**Returned value** + +- Value of type `Decimal(76, S)` if successful, otherwise returns the default value if passed or `0` if not. [Decimal256(S)](../data-types/decimal.md). + +**Examples** + +Query: + +``` sql +SELECT + toDecimal256OrDefault(toString(1/42), 76) AS a, + toTypeName(a), + toDecimal256OrDefault('Inf', 0, CAST('-1', 'Decimal256(0)')) AS b, + toTypeName(b) +FORMAT Vertical; +``` + +Result: + +```response +Row 1: +────── +a: 0.023809523809523808 +toTypeName(a): Decimal(76, 76) +b: -1 +toTypeName(b): Decimal(76, 0) +``` + +**See also** + +- [`toDecimal256`](#todecimal256). +- [`toDecimal256OrZero`](#todecimal256orzero). +- [`toDecimal256OrNull`](#todecimal256ornull). + ## toString Functions for converting between numbers, strings (but not fixed strings), dates, and dates with times. From d243feea2136bbfa5f1e943f64e5ebd851f2b103 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 10 Aug 2024 11:21:28 +0000 Subject: [PATCH 0725/1722] Fix special builds --- src/Columns/ColumnDynamic.cpp | 8 ++++---- src/Columns/ColumnDynamic.h | 2 +- src/Columns/ColumnVariant.cpp | 2 +- src/Columns/ColumnVariant.h | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 454f7956f48..7246be29592 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -325,7 +325,7 @@ void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) /// Check if we insert from shared variant and process it separately. if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) { - auto & src_shared_variant = dynamic_src.getSharedVariant(); + const auto & src_shared_variant = dynamic_src.getSharedVariant(); auto value = src_shared_variant.getDataAt(src_offset); /// Decode data type of this value. ReadBufferFromMemory buf(value.data, value.size); @@ -469,9 +469,9 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t if (variant_info.variant_names.size() - 1 == max_dynamic_types) { auto shared_variant_discr = getSharedVariantDiscriminator(); - for (size_t i = 0; i != dynamic_src.variant_info.variant_names.size(); ++i) + for (const auto & variant_name : dynamic_src.variant_info.variant_names) { - auto it = variant_info.variant_name_to_discriminator.find(dynamic_src.variant_info.variant_names[i]); + auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it == variant_info.variant_name_to_discriminator.end()) other_to_new_discriminators.push_back(shared_variant_discr); else @@ -618,7 +618,7 @@ void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size /// Check if we insert from shared variant and process it separately. if (src_global_discr == dynamic_src.getSharedVariantDiscriminator()) { - auto & src_shared_variant = dynamic_src.getSharedVariant(); + const auto & src_shared_variant = dynamic_src.getSharedVariant(); auto value = src_shared_variant.getDataAt(src_offset); /// Decode data type of this value. ReadBufferFromMemory buf(value.data, value.size); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 8b815e2b015..a595a990964 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -37,7 +37,7 @@ public: MERGE, /// Statistics were calculated during merge of several MergeTree parts. }; - Statistics(Source source_) : source(source_) {} + explicit Statistics(Source source_) : source(source_) {} /// Source of the statistics. Source source; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 7531e976926..0402e1a0690 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -476,7 +476,7 @@ void ColumnVariant::insertFromImpl(const DB::IColumn & src_, size_t n, const std } } -void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator) +void ColumnVariant::insertRangeFromImpl(const DB::IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, const Discriminator * skip_discriminator) { const size_t num_variants = variants.size(); const auto & src = assert_cast(src_); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 571a843d113..7c8093e385d 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -329,7 +329,7 @@ public: private: void insertFromImpl(const IColumn & src_, size_t n, const std::vector * global_discriminators_mapping); - void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, Discriminator * skip_discriminator); + void insertRangeFromImpl(const IColumn & src_, size_t start, size_t length, const std::vector * global_discriminators_mapping, const Discriminator * skip_discriminator); void insertManyFromImpl(const IColumn & src_, size_t position, size_t length, const std::vector * global_discriminators_mapping); void initIdentityGlobalToLocalDiscriminatorsMapping(); From c1b5b908ba13f863de4e9621d3fe2c8139758650 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 13:01:55 +0000 Subject: [PATCH 0726/1722] hide Field::get in private, only use Field::safeGet --- programs/client/Client.cpp | 2 +- programs/keeper-client/Commands.cpp | 6 +- .../AggregateFunctionGroupArray.cpp | 12 +- .../AggregateFunctionGroupArrayIntersect.cpp | 6 +- .../AggregateFunctionGroupArrayMoving.cpp | 6 +- .../AggregateFunctionGroupArraySorted.cpp | 6 +- .../AggregateFunctionGroupConcat.cpp | 10 +- .../AggregateFunctionGroupUniqArray.cpp | 6 +- ...AggregateFunctionKolmogorovSmirnovTest.cpp | 4 +- ...ateFunctionLargestTriangleThreeBuckets.cpp | 2 +- .../AggregateFunctionMannWhitney.cpp | 4 +- .../AggregateFunctionQuantile.h | 6 +- .../AggregateFunctionSumMap.cpp | 10 +- .../Passes/ComparisonTupleEliminationPass.cpp | 2 +- .../Passes/ConvertOrLikeChainPass.cpp | 2 +- .../Passes/FunctionToSubcolumnsPass.cpp | 6 +- src/Analyzer/Passes/FuseFunctionsPass.cpp | 2 +- .../Passes/IfTransformStringsToEnumPass.cpp | 10 +- .../Passes/NormalizeCountVariantsPass.cpp | 2 +- ...ateOrDateTimeConverterWithPreimagePass.cpp | 8 +- .../RewriteAggregateFunctionWithIfPass.cpp | 4 +- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 6 +- src/Analyzer/QueryTreeBuilder.cpp | 2 +- src/Analyzer/Resolve/IdentifierResolver.cpp | 8 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +- src/Analyzer/SetUtils.cpp | 8 +- src/Backups/BackupSettings.cpp | 4 +- src/Backups/DDLAdjustingForBackupVisitor.cpp | 4 +- src/Backups/RestoreSettings.cpp | 8 +- src/Backups/SettingsFieldOptionalString.cpp | 2 +- src/Backups/SettingsFieldOptionalUUID.cpp | 2 +- src/Client/QueryFuzzer.cpp | 20 +-- src/Client/Suggest.cpp | 2 +- src/Columns/ColumnAggregateFunction.cpp | 19 +-- src/Columns/ColumnArray.cpp | 6 +- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnFixedString.cpp | 4 +- src/Columns/ColumnMap.cpp | 10 +- src/Columns/ColumnObject.cpp | 4 +- src/Columns/ColumnString.h | 2 +- src/Columns/ColumnTuple.cpp | 6 +- src/Columns/ColumnVector.h | 4 +- src/Columns/tests/gtest_column_variant.cpp | 120 +++++++++--------- src/Columns/tests/gtest_low_cardinality.cpp | 6 +- src/Common/CollectionOfDerived.h | 2 +- src/Common/FieldBinaryEncoding.cpp | 2 +- src/Common/FieldVisitorSum.cpp | 2 +- src/Common/FieldVisitorSum.h | 2 +- src/Common/FieldVisitorToString.cpp | 2 +- src/Common/HashTable/HashMap.h | 2 +- src/Common/examples/arena_with_free_lists.cpp | 22 ++-- src/Core/ExternalTable.cpp | 2 +- src/Core/Field.h | 71 ++++------- src/Core/Range.cpp | 16 +-- src/Core/Settings.cpp | 2 +- src/Core/SettingsFields.cpp | 14 +- src/Core/SettingsQuirks.cpp | 6 +- src/Core/examples/field.cpp | 2 +- src/Core/tests/gtest_field.cpp | 28 ++-- src/DataTypes/DataTypeAggregateFunction.cpp | 4 +- src/DataTypes/DataTypeDynamic.cpp | 4 +- src/DataTypes/DataTypeEnum.cpp | 18 +-- src/DataTypes/DataTypeFixedString.cpp | 4 +- src/DataTypes/DataTypeObject.cpp | 2 +- src/DataTypes/DataTypesDecimal.cpp | 6 +- src/DataTypes/ObjectUtils.cpp | 2 +- .../Serializations/JSONDataParser.cpp | 4 +- .../SerializationAggregateFunction.cpp | 4 +- .../Serializations/SerializationArray.cpp | 4 +- .../SerializationDecimalBase.cpp | 2 +- .../SerializationFixedString.cpp | 4 +- .../SerializationIPv4andIPv6.cpp | 2 +- .../Serializations/SerializationMap.cpp | 4 +- .../Serializations/SerializationNumber.cpp | 2 +- .../Serializations/SerializationString.cpp | 4 +- .../Serializations/SerializationTuple.cpp | 4 +- .../Serializations/SerializationUUID.cpp | 2 +- src/DataTypes/registerDataTypeDateTime.cpp | 2 +- src/Databases/DDLLoadingDependencyVisitor.cpp | 4 +- src/Databases/DDLRenamingVisitor.cpp | 6 +- src/Databases/DatabaseReplicated.cpp | 4 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 18 +-- .../MySQL/tests/gtest_mysql_binlog.cpp | 30 ++--- src/Dictionaries/CacheDictionaryStorage.h | 12 +- src/Dictionaries/DictionaryHelpers.h | 4 +- src/Dictionaries/FlatDictionary.cpp | 6 +- src/Dictionaries/HashedArrayDictionary.cpp | 8 +- src/Dictionaries/HashedDictionary.h | 8 +- .../HierarchyDictionariesUtils.cpp | 6 +- src/Dictionaries/IPAddressDictionary.cpp | 8 +- src/Dictionaries/MongoDBDictionarySource.cpp | 3 +- src/Dictionaries/PolygonDictionary.cpp | 10 +- src/Dictionaries/RangeHashedDictionary.h | 4 +- src/Dictionaries/RedisDictionarySource.cpp | 3 +- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/Functions/DateTimeTransforms.h | 4 +- src/Functions/FunctionsConsistentHashing.h | 4 +- src/Functions/FunctionsConversion.cpp | 30 ++--- src/Functions/FunctionsJSON.cpp | 2 +- src/Functions/FunctionsLogical.cpp | 6 +- src/Functions/FunctionsRound.h | 6 +- src/Functions/IFunctionCustomWeek.h | 8 +- src/Functions/IFunctionDateOrDateTime.h | 16 +-- .../JSONPath/Parsers/ParserJSONPathRange.cpp | 4 +- src/Functions/MultiMatchAllIndicesImpl.h | 2 +- src/Functions/MultiMatchAnyImpl.h | 2 +- src/Functions/MultiSearchAllPositionsImpl.h | 2 +- src/Functions/MultiSearchFirstIndexImpl.h | 2 +- src/Functions/MultiSearchFirstPositionImpl.h | 2 +- src/Functions/MultiSearchImpl.h | 2 +- src/Functions/URL/cutURLParameter.cpp | 2 +- src/Functions/array/arrayElement.cpp | 38 +++--- src/Functions/array/mapOp.cpp | 2 +- src/Functions/getClientHTTPHeader.cpp | 2 +- src/Functions/multiIf.cpp | 2 +- src/Functions/nested.cpp | 4 +- src/IO/S3Common.cpp | 8 +- src/Interpreters/ActionsVisitor.cpp | 12 +- src/Interpreters/AddDefaultDatabaseVisitor.h | 2 +- .../ComparisonTupleEliminationVisitor.cpp | 2 +- .../ConvertFunctionOrLikeVisitor.cpp | 4 +- .../ConvertStringsToEnumVisitor.cpp | 12 +- src/Interpreters/DDLTask.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 4 +- .../InterpreterKillQueryQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- ...OrDateTimeConverterWithPreimageVisitor.cpp | 8 +- ...OptimizeIfWithConstantConditionVisitor.cpp | 4 +- .../OptimizeShardingKeyRewriteInVisitor.cpp | 21 ++- .../RewriteCountVariantsVisitor.cpp | 2 +- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Interpreters/WindowDescription.cpp | 8 +- src/Interpreters/convertFieldToType.cpp | 62 ++++----- .../evaluateConstantExpression.cpp | 4 +- .../replaceForPositionalArguments.cpp | 4 +- .../tests/gtest_comparison_graph.cpp | 4 +- src/Parsers/ASTFunction.cpp | 2 +- src/Parsers/ASTLiteral.cpp | 8 +- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 10 +- src/Parsers/ParserAlterQuery.cpp | 12 +- src/Parsers/ParserCheckQuery.cpp | 2 +- src/Parsers/ParserCreateQuery.cpp | 4 +- src/Parsers/ParserDictionary.cpp | 6 +- src/Parsers/ParserPartition.cpp | 2 +- src/Parsers/ParserSystemQuery.cpp | 8 +- src/Parsers/ParserUndropQuery.cpp | 2 +- src/Parsers/tests/gtest_dictionary_parser.cpp | 40 +++--- .../Formats/Impl/CHColumnToArrowColumn.cpp | 2 +- .../Impl/ConstantExpressionTemplate.cpp | 6 +- .../Impl/NativeORCBlockInputFormat.cpp | 4 +- .../Impl/PrometheusTextOutputFormat.cpp | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 8 +- .../Algorithms/SummingSortedAlgorithm.cpp | 12 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 10 +- src/Processors/Sources/MySQLSource.cpp | 4 +- .../Transforms/FillingTransform.cpp | 14 +- src/Processors/Transforms/WindowTransform.cpp | 20 +-- src/Storages/AlterCommands.cpp | 8 +- src/Storages/ColumnsDescription.cpp | 2 +- ...pproximateNearestNeighborIndexesCommon.cpp | 12 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 +- src/Storages/MergeTree/KeyCondition.cpp | 12 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 6 +- .../MergeTree/MergeTreeIndexBloomFilter.cpp | 12 +- .../MergeTreeIndexBloomFilterText.cpp | 50 ++++---- .../MergeTree/MergeTreeIndexFullText.cpp | 42 +++--- .../MergeTree/MergeTreeIndexHypothesis.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 +- .../MergeTree/MergeTreeIndexUSearch.cpp | 8 +- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 6 +- .../MergeTree/registerStorageMergeTree.cpp | 2 +- src/Storages/Statistics/Statistics.cpp | 16 +-- .../Statistics/StatisticsCountMinSketch.cpp | 2 +- src/Storages/StorageFactory.cpp | 2 +- src/Storages/StorageFile.cpp | 6 +- src/Storages/StorageFuzzJSON.cpp | 2 +- src/Storages/StorageJoin.cpp | 4 +- src/Storages/StoragePostgreSQL.cpp | 4 +- src/Storages/System/StorageSystemColumns.cpp | 6 +- .../System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.h | 10 +- src/Storages/VirtualColumnUtils.h | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 12 +- src/TableFunctions/TableFunctionExplain.cpp | 4 +- src/TableFunctions/TableFunctionFile.cpp | 2 +- .../TableFunctionMergeTreeIndex.cpp | 4 +- 192 files changed, 749 insertions(+), 774 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 1d99d223ee9..25c94c56aa6 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -223,7 +223,7 @@ std::vector Client::loadWarningMessages() size_t rows = packet.block.rows(); for (size_t i = 0; i < rows; ++i) - messages.emplace_back(column[i].get()); + messages.emplace_back(column[i].safeGet()); } continue; diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index df9da8e9613..7226bd82df7 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -95,7 +95,7 @@ void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co client->zookeeper->set( client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet(), - static_cast(query->args[2].get())); + static_cast(query->args[2].safeGet())); } bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const @@ -494,7 +494,7 @@ void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con { Int32 version{-1}; if (query->args.size() == 2) - version = static_cast(query->args[1].get()); + version = static_cast(query->args[1].safeGet()); client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet()), version); } @@ -549,7 +549,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient String leaving; String new_members; - auto operation = query->args[0].get(); + auto operation = query->args[0].safeGet(); switch (operation) { case static_cast(ReconfigCommand::Operation::ADD): diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 7034e6373b1..5cc9f725b46 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -780,12 +780,12 @@ AggregateFunctionPtr createAggregateFunctionGroupArray( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); has_limit = true; - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -816,11 +816,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[i].get() < 0) || - (type == Field::Types::UInt64 && parameters[i].get() == 0)) + if ((type == Field::Types::Int64 && parameters[i].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[i].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - return parameters[i].get(); + return parameters[i].safeGet(); }; UInt64 max_elems = get_parameter(0); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 38f2fcb9fb9..36d00b1d9ec 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -83,16 +83,16 @@ public: if (version == 1) { for (size_t i = 0; i < arr_size; ++i) - set.insert(static_cast((*data_column)[offset + i].get())); + set.insert(static_cast((*data_column)[offset + i].safeGet())); } else if (!set.empty()) { typename State::Set new_set; for (size_t i = 0; i < arr_size; ++i) { - typename State::Set::LookupResult set_value = set.find(static_cast((*data_column)[offset + i].get())); + typename State::Set::LookupResult set_value = set.find(static_cast((*data_column)[offset + i].safeGet())); if (set_value != nullptr) - new_set.insert(static_cast((*data_column)[offset + i].get())); + new_set.insert(static_cast((*data_column)[offset + i].safeGet())); } set = std::move(new_set); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index 026b8d1956f..2c3ac7f883e 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -269,12 +269,12 @@ AggregateFunctionPtr createAggregateFunctionMoving( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); - if ((type == Field::Types::Int64 && parameters[0].get() <= 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() <= 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); limit_size = true; - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp index d41d743e17a..27043ed6aa6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp @@ -397,11 +397,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArray( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp index 5494ef74705..636ac80e350 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -247,7 +247,7 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat( if (type != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name); - delimiter = parameters[0].get(); + delimiter = parameters[0].safeGet(); } if (parameters.size() == 2) { @@ -256,12 +256,12 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name); - if ((type == Field::Types::Int64 && parameters[1].get() <= 0) || - (type == Field::Types::UInt64 && parameters[1].get() == 0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get()); + if ((type == Field::Types::Int64 && parameters[1].safeGet() <= 0) || + (type == Field::Types::UInt64 && parameters[1].safeGet() == 0)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].safeGet()); has_limit = true; - limit = parameters[1].get(); + limit = parameters[1].safeGet(); } if (has_limit) diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index 7b4300b3568..5cbf449c946 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -323,12 +323,12 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || - (type == Field::Types::UInt64 && parameters[0].get() == 0)) + if ((type == Field::Types::Int64 && parameters[0].safeGet() < 0) || + (type == Field::Types::UInt64 && parameters[0].safeGet() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name); limit_size = true; - max_elems = parameters[0].get(); + max_elems = parameters[0].safeGet(); } else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, diff --git a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp index 04eebe9f485..28e8d37b8c8 100644 --- a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp @@ -238,7 +238,7 @@ public: if (params[0].getType() != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName()); - const auto & param = params[0].get(); + const auto & param = params[0].safeGet(); if (param == "two-sided") alternative = Alternative::TwoSided; else if (param == "less") @@ -255,7 +255,7 @@ public: if (params[1].getType() != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName()); - method = params[1].get(); + method = params[1].safeGet(); if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. " "It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName()); diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp index 6d1e3c0f64b..813b13b6f7b 100644 --- a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp @@ -181,7 +181,7 @@ public: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName()); - total_buckets = params[0].get(); + total_buckets = params[0].safeGet(); this->x_type = WhichDataType(arguments[0]).idx; this->y_type = WhichDataType(arguments[1]).idx; diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp index f088737c340..fa90846650d 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp @@ -152,7 +152,7 @@ public: if (params[0].getType() != Field::Types::String) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName()); - const auto & param = params[0].get(); + const auto & param = params[0].safeGet(); if (param == "two-sided") alternative = Alternative::TwoSided; else if (param == "less") @@ -169,7 +169,7 @@ public: if (params[1].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName()); - continuity_correction = static_cast(params[1].get()); + continuity_correction = static_cast(params[1].safeGet()); } String getName() const override diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 127dc06b642..423fd4bc569 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -117,7 +117,7 @@ public: throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires relative accuracy parameter with Float64 type", getName()); - relative_accuracy = relative_accuracy_field.get(); + relative_accuracy = relative_accuracy_field.safeGet(); if (relative_accuracy <= 0 || relative_accuracy >= 1 || isNaN(relative_accuracy)) throw Exception( @@ -147,9 +147,9 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires accuracy parameter with integer type", getName()); if (accuracy_field.getType() == Field::Types::Int64) - accuracy = accuracy_field.get(); + accuracy = accuracy_field.safeGet(); else - accuracy = accuracy_field.get(); + accuracy = accuracy_field.safeGet(); if (accuracy <= 0) throw Exception( diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 666136a91b7..9a94c3dfe1a 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -300,12 +300,12 @@ public: /// Compatibility with previous versions. if (value.getType() == Field::Types::Decimal32) { - auto source = value.get>(); + auto source = value.safeGet>(); value = DecimalField(source.getValue(), source.getScale()); } else if (value.getType() == Field::Types::Decimal64) { - auto source = value.get>(); + auto source = value.safeGet>(); value = DecimalField(source.getValue(), source.getScale()); } @@ -355,7 +355,7 @@ public: /// Compatibility with previous versions. if (value.getType() == Field::Types::Decimal128) { - auto source = value.get>(); + auto source = value.safeGet>(); WhichDataType value_type(values_types[col_idx]); if (value_type.isDecimal32()) { @@ -560,7 +560,7 @@ private: template bool compareImpl(FieldType & x) const { - auto val = rhs.get(); + auto val = rhs.safeGet(); if (val > x) { x = val; @@ -600,7 +600,7 @@ private: template bool compareImpl(FieldType & x) const { - auto val = rhs.get(); + auto val = rhs.safeGet(); if (val < x) { x = val; diff --git a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp index 180470952cd..4bc4255e2e2 100644 --- a/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp +++ b/src/Analyzer/Passes/ComparisonTupleEliminationPass.cpp @@ -129,7 +129,7 @@ private: if (constant_node_value.getType() != Field::Types::Which::Tuple) return {}; - const auto & constant_tuple = constant_node_value.get(); + const auto & constant_tuple = constant_node_value.safeGet(); const auto & function_arguments_nodes = function_node_typed.getArguments().getNodes(); size_t function_arguments_nodes_size = function_arguments_nodes.size(); diff --git a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp index 2b2ac95d7b9..6c4ce789993 100644 --- a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp +++ b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp @@ -89,7 +89,7 @@ public: if (!pattern || !isString(pattern->getResultType())) continue; - auto regexp = likePatternToRegexp(pattern->getValue().get()); + auto regexp = likePatternToRegexp(pattern->getValue().safeGet()); /// Case insensitive. Works with UTF-8 as well. if (is_ilike) regexp = "(?i)" + regexp; diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index b968f43c6a6..1fc3eec6833 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -68,10 +68,10 @@ void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, Col String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & data_type_tuple) { if (value.getType() == Field::Types::String) - return value.get(); + return value.safeGet(); if (value.getType() == Field::Types::UInt64) - return data_type_tuple.getNameByPosition(value.get()); + return data_type_tuple.getNameByPosition(value.safeGet()); return ""; } @@ -79,7 +79,7 @@ String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & dat String getSubcolumnNameForElement(const Field & value, const DataTypeVariant &) { if (value.getType() == Field::Types::String) - return value.get(); + return value.safeGet(); return ""; } diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 0175e304a2b..f3b109a10ed 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -187,7 +187,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector & nodes /// Sort nodes and parameters in ascending order of quantile level std::vector permutation(nodes.size()); iota(permutation.data(), permutation.size(), size_t(0)); - std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get() < parameters[j].get(); }); + std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].safeGet() < parameters[j].safeGet(); }); std::vector new_nodes; new_nodes.reserve(permutation.size()); diff --git a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp index d966f129d08..f81327c5d55 100644 --- a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp +++ b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp @@ -134,8 +134,8 @@ public: return; std::set string_values; - string_values.insert(first_literal->getValue().get()); - string_values.insert(second_literal->getValue().get()); + string_values.insert(first_literal->getValue().safeGet()); + string_values.insert(second_literal->getValue().safeGet()); changeIfArguments(*function_if_node, string_values, context); wrapIntoToString(*function_node, std::move(modified_if_node), context); @@ -163,7 +163,7 @@ public: if (!isArray(literal_to->getResultType()) || !isString(literal_default->getResultType())) return; - auto array_to = literal_to->getValue().get(); + auto array_to = literal_to->getValue().safeGet(); if (array_to.empty()) return; @@ -178,9 +178,9 @@ public: std::set string_values; for (const auto & value : array_to) - string_values.insert(value.get()); + string_values.insert(value.safeGet()); - string_values.insert(literal_default->getValue().get()); + string_values.insert(literal_default->getValue().safeGet()); changeTransformArguments(*function_modified_transform_node, string_values, context); wrapIntoToString(*function_node, std::move(modified_transform_node), context); diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index 3a8b6e75d40..02f1c93ea7f 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -54,7 +54,7 @@ public: } else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && - first_argument_constant_literal.get() == 1) + first_argument_constant_literal.safeGet() == 1) { function_node->getArguments().getNodes().clear(); resolveAggregateFunctionNodeByName(*function_node, "count"); diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp index feb8bcc792d..0f33c302265 100644 --- a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp @@ -143,13 +143,13 @@ private: const auto & column_type = column_node_typed.getColumnType().get(); if (isDateOrDate32(column_type)) { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); + start_date_or_date_time = date_lut.dateToString(range.first.safeGet()); + end_date_or_date_time = date_lut.dateToString(range.second.safeGet()); } else if (isDateTime(column_type) || isDateTime64(column_type)) { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); + start_date_or_date_time = date_lut.timeToString(range.first.safeGet()); + end_date_or_date_time = date_lut.timeToString(range.second.safeGet()); } else [[unlikely]] return {}; diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index a48e88132a6..091061ceb81 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -60,7 +60,7 @@ public: { const auto & second_const_value = second_const_node->getValue(); if (second_const_value.isNull() - || (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.get() == 0 + || (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.safeGet() == 0 && !if_node->getResultType()->isNullable())) { /// avg(if(cond, a, null)) -> avgIf(a::ResultTypeIf, cond) @@ -89,7 +89,7 @@ public: { const auto & first_const_value = first_const_node->getValue(); if (first_const_value.isNull() - || (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.get() == 0 + || (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.safeGet() == 0 && !if_node->getResultType()->isNullable())) { /// avg(if(cond, null, a) -> avgIf(a::ResultTypeIf, !cond)) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index 1524629dc81..a987ced497a 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -66,7 +66,7 @@ public: resolveAggregateFunctionNodeByName(*function_node, "countIf"); - if (constant_value_literal.get() != 1) + if (constant_value_literal.safeGet() != 1) { /// Rewrite `sumIf(123, cond)` into `123 * countIf(cond)` node = getMultiplyFunction(std::move(multiplier_node), node); @@ -105,8 +105,8 @@ public: const auto & if_true_condition_constant_value_literal = if_true_condition_constant_node->getValue(); const auto & if_false_condition_constant_value_literal = if_false_condition_constant_node->getValue(); - auto if_true_condition_value = if_true_condition_constant_value_literal.get(); - auto if_false_condition_value = if_false_condition_constant_value_literal.get(); + auto if_true_condition_value = if_true_condition_constant_value_literal.safeGet(); + auto if_false_condition_value = if_false_condition_constant_value_literal.safeGet(); if (if_false_condition_value == 0) { diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index fb41826929f..9754897d54d 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -471,7 +471,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express std::shared_ptr collator; if (order_by_element.getCollation()) - collator = std::make_shared(order_by_element.getCollation()->as().value.get()); + collator = std::make_shared(order_by_element.getCollation()->as().value.safeGet()); const auto & sort_expression_ast = order_by_element.children.at(0); auto sort_expression = buildExpression(sort_expression_ast, context); diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 447bf825836..a79433ac130 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -1273,7 +1273,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns( const auto & constant_node_value = constant_node.getValue(); if (constant_node_value.getType() == Field::Types::String) { - array_join_subcolumn_prefix = constant_node_value.get() + "."; + array_join_subcolumn_prefix = constant_node_value.safeGet() + "."; array_join_parent_column = argument_nodes.at(0).get(); } } @@ -1287,7 +1287,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns( if (!second_argument || second_argument->getValue().getType() != Field::Types::String) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected constant string as second argument of getSubcolumn function {}", resolved_function->dumpTree()); - const auto & resolved_subcolumn_path = second_argument->getValue().get(); + const auto & resolved_subcolumn_path = second_argument->getValue().safeGet(); if (!startsWith(resolved_subcolumn_path, array_join_subcolumn_prefix)) return {}; @@ -1331,7 +1331,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression size_t nested_function_arguments_size = nested_function_arguments.size(); const auto & nested_keys_names_constant_node = nested_function_arguments[0]->as(); - const auto & nested_keys_names = nested_keys_names_constant_node.getValue().get(); + const auto & nested_keys_names = nested_keys_names_constant_node.getValue().safeGet(); size_t nested_keys_names_size = nested_keys_names.size(); if (nested_keys_names_size == nested_function_arguments_size - 1) @@ -1344,7 +1344,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression auto array_join_column = std::make_shared(array_join_column_expression_typed.getColumn(), array_join_column_expression_typed.getColumnSource()); - const auto & nested_key_name = nested_keys_names[i - 1].get(); + const auto & nested_key_name = nested_keys_names[i - 1].safeGet(); Identifier nested_identifier = Identifier(nested_key_name); array_join_resolved_expression = wrapExpressionNodeInTupleElement(array_join_column, nested_identifier, scope.context); break; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 767d5c11075..2ce79b7bddd 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -746,11 +746,11 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ UInt64 pos; if (constant_node->getValue().getType() == Field::Types::UInt64) { - pos = constant_node->getValue().get(); + pos = constant_node->getValue().safeGet(); } else // Int64 { - auto value = constant_node->getValue().get(); + auto value = constant_node->getValue().safeGet(); if (value > 0) pos = value; else diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0ecb3545225..095be1fb9b8 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -93,7 +93,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& "Invalid type in set. Expected tuple, got {}", value.getTypeName()); - const auto & tuple = value.template get(); + const auto & tuple = value.template safeGet(); const DataTypePtr & value_type = value_types[collection_index]; const DataTypes & tuple_value_type = typeid_cast(value_type.get())->getElements(); @@ -169,15 +169,15 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const if (rhs_which_type.isArray()) { const DataTypeArray * value_array_type = assert_cast(value_type.get()); - size_t value_array_size = value.get().size(); + size_t value_array_size = value.safeGet().size(); DataTypes value_types(value_array_size, value_array_type->getNestedType()); - result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); + result_block = createBlockFromCollection(value.safeGet(), value_types, set_element_types, transform_null_in); } else if (rhs_which_type.isTuple()) { const DataTypeTuple * value_tuple_type = assert_cast(value_type.get()); const DataTypes & value_types = value_tuple_type->getElements(); - result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); + result_block = createBlockFromCollection(value.safeGet(), value_types, set_element_types, transform_null_in); } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index e33880f88e3..7f353028346 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -125,7 +125,7 @@ std::vector BackupSettings::Util::clusterHostIDsFromAST(const IAST & as throw Exception( ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Setting cluster_host_ids has wrong format, must be array of arrays of string literals"); - const auto & replicas = array_of_replicas->value.get(); + const auto & replicas = array_of_replicas->value.safeGet(); res[i].resize(replicas.size()); for (size_t j = 0; j != replicas.size(); ++j) { @@ -134,7 +134,7 @@ std::vector BackupSettings::Util::clusterHostIDsFromAST(const IAST & as throw Exception( ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Setting cluster_host_ids has wrong format, must be array of arrays of string literals"); - res[i][j] = replica.get(); + res[i][j] = replica.safeGet(); } } } diff --git a/src/Backups/DDLAdjustingForBackupVisitor.cpp b/src/Backups/DDLAdjustingForBackupVisitor.cpp index 910831195a3..4dcbdcc1617 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.cpp +++ b/src/Backups/DDLAdjustingForBackupVisitor.cpp @@ -46,8 +46,8 @@ namespace if (zookeeper_path_ast && (zookeeper_path_ast->value.getType() == Field::Types::String) && replica_name_ast && (replica_name_ast->value.getType() == Field::Types::String)) { - String & zookeeper_path_arg = zookeeper_path_ast->value.get(); - String & replica_name_arg = replica_name_ast->value.get(); + String & zookeeper_path_arg = zookeeper_path_ast->value.safeGet(); + String & replica_name_arg = replica_name_ast->value.safeGet(); if (create.uuid != UUIDHelpers::Nil) { String table_uuid_str = toString(create.uuid); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 7bbfd9ed751..4662305cdd6 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -31,7 +31,7 @@ namespace { if (field.getType() == Field::Types::String) { - const String & str = field.get(); + const String & str = field.safeGet(); if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create")) { value = RestoreTableCreationMode::kCreate; @@ -54,7 +54,7 @@ namespace if (field.getType() == Field::Types::UInt64) { - UInt64 number = field.get(); + UInt64 number = field.safeGet(); if (number == 1) { value = RestoreTableCreationMode::kCreate; @@ -95,7 +95,7 @@ namespace { if (field.getType() == Field::Types::String) { - const String & str = field.get(); + const String & str = field.safeGet(); if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create")) { value = RestoreAccessCreationMode::kCreate; @@ -118,7 +118,7 @@ namespace if (field.getType() == Field::Types::UInt64) { - UInt64 number = field.get(); + UInt64 number = field.safeGet(); if (number == 1) { value = RestoreAccessCreationMode::kCreate; diff --git a/src/Backups/SettingsFieldOptionalString.cpp b/src/Backups/SettingsFieldOptionalString.cpp index 573fd1e052c..684407a533d 100644 --- a/src/Backups/SettingsFieldOptionalString.cpp +++ b/src/Backups/SettingsFieldOptionalString.cpp @@ -19,7 +19,7 @@ SettingFieldOptionalString::SettingFieldOptionalString(const Field & field) if (field.getType() == Field::Types::String) { - value = field.get(); + value = field.safeGet(); return; } diff --git a/src/Backups/SettingsFieldOptionalUUID.cpp b/src/Backups/SettingsFieldOptionalUUID.cpp index 3f14608b206..0011f7f1073 100644 --- a/src/Backups/SettingsFieldOptionalUUID.cpp +++ b/src/Backups/SettingsFieldOptionalUUID.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes if (field.getType() == Field::Types::String) { - const String & str = field.get(); + const String & str = field.safeGet(); if (str.empty()) { value = std::nullopt; diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index f5b700ea529..cc7b37aad52 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -133,7 +133,7 @@ Field QueryFuzzer::fuzzField(Field field) if (type == Field::Types::String) { - auto & str = field.get(); + auto & str = field.safeGet(); UInt64 action = fuzz_rand() % 10; switch (action) { @@ -159,7 +159,7 @@ Field QueryFuzzer::fuzzField(Field field) } else if (type == Field::Types::Array) { - auto & arr = field.get(); + auto & arr = field.safeGet(); if (fuzz_rand() % 5 == 0 && !arr.empty()) { @@ -191,7 +191,7 @@ Field QueryFuzzer::fuzzField(Field field) } else if (type == Field::Types::Tuple) { - auto & arr = field.get(); + auto & arr = field.safeGet(); if (fuzz_rand() % 5 == 0 && !arr.empty()) { @@ -912,17 +912,17 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child) auto type = l->value.getType(); if (type == Field::Types::Which::String && fuzz_rand() % 7 == 0) { - String value = l->value.get(); + String value = l->value.safeGet(); child = makeASTFunction( "toFixedString", std::make_shared(value), std::make_shared(static_cast(value.size()))); } else if (type == Field::Types::Which::UInt64 && fuzz_rand() % 7 == 0) { - child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared(l->value.get())); + child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared(l->value.safeGet())); } else if (type == Field::Types::Which::Int64 && fuzz_rand() % 7 == 0) { - child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared(l->value.get())); + child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared(l->value.safeGet())); } else if (type == Field::Types::Which::Float64 && fuzz_rand() % 7 == 0) { @@ -930,22 +930,22 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child) if (decimal == 0) child = makeASTFunction( "toDecimal32", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 9))); else if (decimal == 1) child = makeASTFunction( "toDecimal64", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 18))); else if (decimal == 2) child = makeASTFunction( "toDecimal128", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 38))); else child = makeASTFunction( "toDecimal256", - std::make_shared(l->value.get()), + std::make_shared(l->value.safeGet()), std::make_shared(static_cast(fuzz_rand() % 76))); } diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 0188ebc8173..affd620f83a 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -214,7 +214,7 @@ void Suggest::fillWordsFromBlock(const Block & block) Words new_words; new_words.reserve(rows); for (size_t i = 0; i < rows; ++i) - new_words.emplace_back(column[i].get()); + new_words.emplace_back(column[i].safeGet()); addWords(std::move(new_words)); } diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index e26fe790a8e..c41c340b069 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -426,9 +426,9 @@ MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const Field ColumnAggregateFunction::operator[](size_t n) const { Field field = AggregateFunctionStateData(); - field.get().name = type_string; + field.safeGet().name = type_string; { - WriteBufferFromString buffer(field.get().data); + WriteBufferFromString buffer(field.safeGet().data); func->serialize(data[n], buffer, version); } return field; @@ -436,12 +436,7 @@ Field ColumnAggregateFunction::operator[](size_t n) const void ColumnAggregateFunction::get(size_t n, Field & res) const { - res = AggregateFunctionStateData(); - res.get().name = type_string; - { - WriteBufferFromString buffer(res.get().data); - func->serialize(data[n], buffer, version); - } + res = operator[](n); } StringRef ColumnAggregateFunction::getDataAt(size_t n) const @@ -521,7 +516,7 @@ void ColumnAggregateFunction::insert(const Field & x) "Inserting field of type {} into ColumnAggregateFunction. Expected {}", x.getTypeName(), Field::Types::AggregateFunctionState); - const auto & field_name = x.get().name; + const auto & field_name = x.safeGet().name; if (type_string != field_name) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot insert filed with type {} into column with type {}", field_name, type_string); @@ -529,7 +524,7 @@ void ColumnAggregateFunction::insert(const Field & x) ensureOwnership(); Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); - ReadBufferFromString read_buffer(x.get().data); + ReadBufferFromString read_buffer(x.safeGet().data); func->deserialize(data.back(), read_buffer, version, &arena); } @@ -538,14 +533,14 @@ bool ColumnAggregateFunction::tryInsert(const DB::Field & x) if (x.getType() != Field::Types::AggregateFunctionState) return false; - const auto & field_name = x.get().name; + const auto & field_name = x.safeGet().name; if (type_string != field_name) return false; ensureOwnership(); Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); - ReadBufferFromString read_buffer(x.get().data); + ReadBufferFromString read_buffer(x.safeGet().data); func->deserialize(data.back(), read_buffer, version, &arena); return true; } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 19cce678cc7..5379adc0bf7 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const size, max_array_size_as_field); res = Array(); - Array & res_arr = res.get(); + Array & res_arr = res.safeGet(); res_arr.reserve(size); for (size_t i = 0; i < size; ++i) @@ -309,7 +309,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const void ColumnArray::insert(const Field & x) { - const Array & array = x.get(); + const Array & array = x.safeGet(); size_t size = array.size(); for (size_t i = 0; i < size; ++i) getData().insert(array[i]); @@ -321,7 +321,7 @@ bool ColumnArray::tryInsert(const Field & x) if (x.getType() != Field::Types::Which::Array) return false; - const Array & array = x.get(); + const Array & array = x.safeGet(); size_t size = array.size(); for (size_t i = 0; i < size; ++i) { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 59bfbd2159c..07120f5f035 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -74,7 +74,7 @@ public: void insertData(const char * src, size_t /*length*/) override; void insertDefault() override { data.push_back(T()); } void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); } - void insert(const Field & x) override { data.push_back(x.get()); } + void insert(const Field & x) override { data.push_back(x.safeGet()); } bool tryInsert(const Field & x) override; #if !defined(DEBUG_OR_SANITIZER_BUILD) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 0bb3f7edb14..04e894ee5ab 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const void ColumnFixedString::insert(const Field & x) { - const String & s = x.get(); + const String & s = x.safeGet(); insertData(s.data(), s.size()); } @@ -67,7 +67,7 @@ bool ColumnFixedString::tryInsert(const Field & x) { if (x.getType() != Field::Types::Which::String) return false; - const String & s = x.get(); + const String & s = x.safeGet(); if (s.size() > n) return false; insertData(s.data(), s.size()); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 1025b4e77b9..6a6618bd81e 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -72,7 +72,7 @@ void ColumnMap::get(size_t n, Field & res) const size_t size = offsets[n] - offsets[n - 1]; res = Map(); - auto & map = res.get(); + auto & map = res.safeGet(); map.reserve(size); for (size_t i = 0; i < size; ++i) @@ -96,7 +96,7 @@ void ColumnMap::insertData(const char *, size_t) void ColumnMap::insert(const Field & x) { - const auto & map = x.get(); + const auto & map = x.safeGet(); nested->insert(Array(map.begin(), map.end())); } @@ -105,7 +105,7 @@ bool ColumnMap::tryInsert(const Field & x) if (x.getType() != Field::Types::Which::Map) return false; - const auto & map = x.get(); + const auto & map = x.safeGet(); return nested->tryInsert(Array(map.begin(), map.end())); } @@ -288,8 +288,8 @@ void ColumnMap::getExtremes(Field & min, Field & max) const /// Convert result Array fields to Map fields because client expect min and max field to have type Map - Array nested_min_value = nested_min.get(); - Array nested_max_value = nested_max.get(); + Array nested_min_value = nested_min.safeGet(); + Array nested_max_value = nested_max.safeGet(); Map map_min_value(nested_min_value.begin(), nested_min_value.end()); Map map_max_value(nested_max_value.begin(), nested_max_value.end()); diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index a6431007cb6..eb99bb4081b 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -698,7 +698,7 @@ void ColumnObject::forEachSubcolumnRecursively(RecursiveMutableColumnCallback ca void ColumnObject::insert(const Field & field) { - const auto & object = field.get(); + const auto & object = field.safeGet(); HashSet inserted_paths; size_t old_size = size(); @@ -754,7 +754,7 @@ void ColumnObject::get(size_t n, Field & res) const { assert(n < size()); res = Object(); - auto & object = res.get(); + auto & object = res.safeGet(); for (const auto & entry : subcolumns) { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c1012e1e55e..65aff9e6255 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -123,7 +123,7 @@ public: void insert(const Field & x) override { - const String & s = x.get(); + const String & s = x.safeGet(); const size_t old_size = chars.size(); const size_t size_to_append = s.size() + 1; const size_t new_size = old_size + size_to_append; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 4fc3f88a87c..251be8d9986 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -141,7 +141,7 @@ void ColumnTuple::get(size_t n, Field & res) const const size_t tuple_size = columns.size(); res = Tuple(); - Tuple & res_tuple = res.get(); + Tuple & res_tuple = res.safeGet(); res_tuple.reserve(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -169,7 +169,7 @@ void ColumnTuple::insertData(const char *, size_t) void ColumnTuple::insert(const Field & x) { - const auto & tuple = x.get(); + const auto & tuple = x.safeGet(); const size_t tuple_size = columns.size(); if (tuple.size() != tuple_size) @@ -185,7 +185,7 @@ bool ColumnTuple::tryInsert(const Field & x) if (x.getType() != Field::Types::Which::Tuple) return false; - const auto & tuple = x.get(); + const auto & tuple = x.safeGet(); const size_t tuple_size = columns.size(); if (tuple.size() != tuple_size) diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 2fe5b635bd2..a5e1ee4b462 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -85,7 +85,7 @@ public: void insertMany(const Field & field, size_t length) override { - data.resize_fill(data.size() + length, static_cast(field.get())); + data.resize_fill(data.size() + length, static_cast(field.safeGet())); } void insertData(const char * pos, size_t) override @@ -235,7 +235,7 @@ public: void insert(const Field & x) override { - data.push_back(static_cast(x.get())); + data.push_back(static_cast(x.safeGet())); } bool tryInsert(const DB::Field & x) override; diff --git a/src/Columns/tests/gtest_column_variant.cpp b/src/Columns/tests/gtest_column_variant.cpp index 25f276b9600..5e481b88409 100644 --- a/src/Columns/tests/gtest_column_variant.cpp +++ b/src/Columns/tests/gtest_column_variant.cpp @@ -108,10 +108,10 @@ void checkColumnVariant1(ColumnVariant * column) ASSERT_EQ(offsets[1], 0); ASSERT_EQ(offsets[3], 1); ASSERT_TRUE(column->isDefaultAt(2) && column->isDefaultAt(4)); - ASSERT_EQ((*column)[0].get(), 42); - ASSERT_EQ((*column)[1].get(), "Hello"); + ASSERT_EQ((*column)[0].safeGet(), 42); + ASSERT_EQ((*column)[1].safeGet(), "Hello"); ASSERT_TRUE((*column)[2].isNull()); - ASSERT_EQ((*column)[3].get(), "World"); + ASSERT_EQ((*column)[3].safeGet(), "World"); ASSERT_TRUE((*column)[4].isNull()); } @@ -209,9 +209,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNulls) ASSERT_EQ(offsets[0], 0); ASSERT_EQ(offsets[1], 1); ASSERT_EQ(offsets[2], 2); - ASSERT_EQ((*column)[0].get(), 0); - ASSERT_EQ((*column)[1].get(), 1); - ASSERT_EQ((*column)[2].get(), 2); + ASSERT_EQ((*column)[0].safeGet(), 0); + ASSERT_EQ((*column)[1].safeGet(), 1); + ASSERT_EQ((*column)[2].safeGet(), 2); } TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrder) @@ -222,9 +222,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrde ASSERT_EQ(offsets[0], 0); ASSERT_EQ(offsets[1], 1); ASSERT_EQ(offsets[2], 2); - ASSERT_EQ((*column)[0].get(), 0); - ASSERT_EQ((*column)[1].get(), 1); - ASSERT_EQ((*column)[2].get(), 2); + ASSERT_EQ((*column)[0].safeGet(), 0); + ASSERT_EQ((*column)[1].safeGet(), 1); + ASSERT_EQ((*column)[2].safeGet(), 2); ASSERT_EQ(column->localDiscriminatorAt(0), 2); ASSERT_EQ(column->localDiscriminatorAt(1), 2); ASSERT_EQ(column->localDiscriminatorAt(2), 2); @@ -331,9 +331,9 @@ TEST(ColumnVariant, CloneResizedGeneral1) ASSERT_EQ(offsets[0], 0); ASSERT_EQ(offsets[1], 0); ASSERT_EQ(offsets[3], 1); - ASSERT_EQ((*resized_column_variant)[0].get(), 42); - ASSERT_EQ((*resized_column_variant)[1].get(), "Hello"); - ASSERT_EQ((*resized_column_variant)[3].get(), 43); + ASSERT_EQ((*resized_column_variant)[0].safeGet(), 42); + ASSERT_EQ((*resized_column_variant)[1].safeGet(), "Hello"); + ASSERT_EQ((*resized_column_variant)[3].safeGet(), 43); } TEST(ColumnVariant, CloneResizedGeneral2) @@ -367,7 +367,7 @@ TEST(ColumnVariant, CloneResizedGeneral2) ASSERT_EQ(discriminators[2], ColumnVariant::NULL_DISCRIMINATOR); const auto & offsets = resized_column_variant->getOffsets(); ASSERT_EQ(offsets[0], 0); - ASSERT_EQ((*resized_column_variant)[0].get(), 42); + ASSERT_EQ((*resized_column_variant)[0].safeGet(), 42); } TEST(ColumnVariant, CloneResizedGeneral3) @@ -405,10 +405,10 @@ TEST(ColumnVariant, CloneResizedGeneral3) ASSERT_EQ(offsets[1], 0); ASSERT_EQ(offsets[2], 1); ASSERT_EQ(offsets[3], 1); - ASSERT_EQ((*resized_column_variant)[0].get(), 42); - ASSERT_EQ((*resized_column_variant)[1].get(), "Hello"); - ASSERT_EQ((*resized_column_variant)[2].get(), "World"); - ASSERT_EQ((*resized_column_variant)[3].get(), 43); + ASSERT_EQ((*resized_column_variant)[0].safeGet(), 42); + ASSERT_EQ((*resized_column_variant)[1].safeGet(), "Hello"); + ASSERT_EQ((*resized_column_variant)[2].safeGet(), "World"); + ASSERT_EQ((*resized_column_variant)[3].safeGet(), 43); } MutableColumnPtr createDiscriminators2() @@ -465,7 +465,7 @@ TEST(ColumnVariant, InsertFrom) auto column_from = createVariantColumn2(change_order); column_to->insertFrom(*column_from, 3); ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0); - ASSERT_EQ((*column_to)[5].get(), 43); + ASSERT_EQ((*column_to)[5].safeGet(), 43); } } @@ -478,8 +478,8 @@ TEST(ColumnVariant, InsertRangeFromOneColumnNoNulls) column_to->insertRangeFrom(*column_from, 2, 2); ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0); ASSERT_EQ(column_to->globalDiscriminatorAt(8), 0); - ASSERT_EQ((*column_to)[7].get(), 2); - ASSERT_EQ((*column_to)[8].get(), 3); + ASSERT_EQ((*column_to)[7].safeGet(), 2); + ASSERT_EQ((*column_to)[8].safeGet(), 3); } } @@ -494,9 +494,9 @@ TEST(ColumnVariant, InsertRangeFromGeneral) ASSERT_EQ(column_to->globalDiscriminatorAt(6), ColumnVariant::NULL_DISCRIMINATOR); ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0); ASSERT_EQ(column_to->globalDiscriminatorAt(8), 1); - ASSERT_EQ((*column_to)[5].get(), "Hello"); - ASSERT_EQ((*column_to)[7].get(), 43); - ASSERT_EQ((*column_to)[8].get(), "World"); + ASSERT_EQ((*column_to)[5].safeGet(), "Hello"); + ASSERT_EQ((*column_to)[7].safeGet(), 43); + ASSERT_EQ((*column_to)[8].safeGet(), "World"); } } @@ -509,8 +509,8 @@ TEST(ColumnVariant, InsertManyFrom) column_to->insertManyFrom(*column_from, 3, 2); ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0); ASSERT_EQ(column_to->globalDiscriminatorAt(6), 0); - ASSERT_EQ((*column_to)[5].get(), 43); - ASSERT_EQ((*column_to)[6].get(), 43); + ASSERT_EQ((*column_to)[5].safeGet(), 43); + ASSERT_EQ((*column_to)[6].safeGet(), 43); } } @@ -520,8 +520,8 @@ TEST(ColumnVariant, PopBackOneColumnNoNulls) column->popBack(3); ASSERT_EQ(column->size(), 2); ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 2); - ASSERT_EQ((*column)[0].get(), 0); - ASSERT_EQ((*column)[1].get(), 1); + ASSERT_EQ((*column)[0].safeGet(), 0); + ASSERT_EQ((*column)[1].safeGet(), 1); } TEST(ColumnVariant, PopBackGeneral) @@ -531,8 +531,8 @@ TEST(ColumnVariant, PopBackGeneral) ASSERT_EQ(column->size(), 3); ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 1); ASSERT_EQ(column->getVariantByLocalDiscriminator(1).size(), 1); - ASSERT_EQ((*column)[0].get(), 42); - ASSERT_EQ((*column)[1].get(), "Hello"); + ASSERT_EQ((*column)[0].safeGet(), 42); + ASSERT_EQ((*column)[1].safeGet(), "Hello"); ASSERT_TRUE((*column)[2].isNull()); } @@ -545,8 +545,8 @@ TEST(ColumnVariant, FilterOneColumnNoNulls) filter.push_back(1); auto filtered_column = column->filter(filter, -1); ASSERT_EQ(filtered_column->size(), 2); - ASSERT_EQ((*filtered_column)[0].get(), 0); - ASSERT_EQ((*filtered_column)[1].get(), 2); + ASSERT_EQ((*filtered_column)[0].safeGet(), 0); + ASSERT_EQ((*filtered_column)[1].safeGet(), 2); } TEST(ColumnVariant, FilterGeneral) @@ -562,7 +562,7 @@ TEST(ColumnVariant, FilterGeneral) filter.push_back(0); auto filtered_column = column->filter(filter, -1); ASSERT_EQ(filtered_column->size(), 3); - ASSERT_EQ((*filtered_column)[0].get(), "Hello"); + ASSERT_EQ((*filtered_column)[0].safeGet(), "Hello"); ASSERT_TRUE((*filtered_column)[1].isNull()); ASSERT_TRUE((*filtered_column)[2].isNull()); } @@ -577,9 +577,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls) permutation.push_back(0); auto permuted_column = column->permute(permutation, 3); ASSERT_EQ(permuted_column->size(), 3); - ASSERT_EQ((*permuted_column)[0].get(), 1); - ASSERT_EQ((*permuted_column)[1].get(), 3); - ASSERT_EQ((*permuted_column)[2].get(), 2); + ASSERT_EQ((*permuted_column)[0].safeGet(), 1); + ASSERT_EQ((*permuted_column)[1].safeGet(), 3); + ASSERT_EQ((*permuted_column)[2].safeGet(), 2); auto index = ColumnUInt64::create(); index->getData().push_back(1); @@ -588,9 +588,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls) index->getData().push_back(0); auto indexed_column = column->index(*index, 3); ASSERT_EQ(indexed_column->size(), 3); - ASSERT_EQ((*indexed_column)[0].get(), 1); - ASSERT_EQ((*indexed_column)[1].get(), 3); - ASSERT_EQ((*indexed_column)[2].get(), 2); + ASSERT_EQ((*indexed_column)[0].safeGet(), 1); + ASSERT_EQ((*indexed_column)[1].safeGet(), 3); + ASSERT_EQ((*indexed_column)[2].safeGet(), 2); } TEST(ColumnVariant, PermuteGeneral) @@ -603,9 +603,9 @@ TEST(ColumnVariant, PermuteGeneral) permutation.push_back(5); auto permuted_column = column->permute(permutation, 4); ASSERT_EQ(permuted_column->size(), 4); - ASSERT_EQ((*permuted_column)[0].get(), 43); - ASSERT_EQ((*permuted_column)[1].get(), "World"); - ASSERT_EQ((*permuted_column)[2].get(), "Hello"); + ASSERT_EQ((*permuted_column)[0].safeGet(), 43); + ASSERT_EQ((*permuted_column)[1].safeGet(), "World"); + ASSERT_EQ((*permuted_column)[2].safeGet(), "Hello"); ASSERT_TRUE((*permuted_column)[3].isNull()); } @@ -618,12 +618,12 @@ TEST(ColumnVariant, ReplicateOneColumnNoNull) offsets.push_back(6); auto replicated_column = column->replicate(offsets); ASSERT_EQ(replicated_column->size(), 6); - ASSERT_EQ((*replicated_column)[0].get(), 1); - ASSERT_EQ((*replicated_column)[1].get(), 1); - ASSERT_EQ((*replicated_column)[2].get(), 1); - ASSERT_EQ((*replicated_column)[3].get(), 2); - ASSERT_EQ((*replicated_column)[4].get(), 2); - ASSERT_EQ((*replicated_column)[5].get(), 2); + ASSERT_EQ((*replicated_column)[0].safeGet(), 1); + ASSERT_EQ((*replicated_column)[1].safeGet(), 1); + ASSERT_EQ((*replicated_column)[2].safeGet(), 1); + ASSERT_EQ((*replicated_column)[3].safeGet(), 2); + ASSERT_EQ((*replicated_column)[4].safeGet(), 2); + ASSERT_EQ((*replicated_column)[5].safeGet(), 2); } TEST(ColumnVariant, ReplicateGeneral) @@ -637,9 +637,9 @@ TEST(ColumnVariant, ReplicateGeneral) offsets.push_back(7); auto replicated_column = column->replicate(offsets); ASSERT_EQ(replicated_column->size(), 7); - ASSERT_EQ((*replicated_column)[0].get(), 42); - ASSERT_EQ((*replicated_column)[1].get(), "Hello"); - ASSERT_EQ((*replicated_column)[2].get(), "Hello"); + ASSERT_EQ((*replicated_column)[0].safeGet(), 42); + ASSERT_EQ((*replicated_column)[1].safeGet(), "Hello"); + ASSERT_EQ((*replicated_column)[2].safeGet(), "Hello"); ASSERT_TRUE((*replicated_column)[3].isNull()); ASSERT_TRUE((*replicated_column)[4].isNull()); ASSERT_TRUE((*replicated_column)[5].isNull()); @@ -657,13 +657,13 @@ TEST(ColumnVariant, ScatterOneColumnNoNulls) selector.push_back(1); auto columns = column->scatter(3, selector); ASSERT_EQ(columns[0]->size(), 2); - ASSERT_EQ((*columns[0])[0].get(), 0); - ASSERT_EQ((*columns[0])[1].get(), 3); + ASSERT_EQ((*columns[0])[0].safeGet(), 0); + ASSERT_EQ((*columns[0])[1].safeGet(), 3); ASSERT_EQ(columns[1]->size(), 2); - ASSERT_EQ((*columns[1])[0].get(), 1); - ASSERT_EQ((*columns[1])[1].get(), 4); + ASSERT_EQ((*columns[1])[0].safeGet(), 1); + ASSERT_EQ((*columns[1])[1].safeGet(), 4); ASSERT_EQ(columns[2]->size(), 1); - ASSERT_EQ((*columns[2])[0].get(), 2); + ASSERT_EQ((*columns[2])[0].safeGet(), 2); } TEST(ColumnVariant, ScatterGeneral) @@ -680,12 +680,12 @@ TEST(ColumnVariant, ScatterGeneral) auto columns = column->scatter(3, selector); ASSERT_EQ(columns[0]->size(), 3); - ASSERT_EQ((*columns[0])[0].get(), 42); - ASSERT_EQ((*columns[0])[1].get(), "Hello"); - ASSERT_EQ((*columns[0])[2].get(), 43); + ASSERT_EQ((*columns[0])[0].safeGet(), 42); + ASSERT_EQ((*columns[0])[1].safeGet(), "Hello"); + ASSERT_EQ((*columns[0])[2].safeGet(), 43); ASSERT_EQ(columns[1]->size(), 2); - ASSERT_EQ((*columns[1])[0].get(), "World"); - ASSERT_EQ((*columns[1])[1].get(), 44); + ASSERT_EQ((*columns[1])[0].safeGet(), "World"); + ASSERT_EQ((*columns[1])[1].safeGet(), 44); ASSERT_EQ(columns[2]->size(), 2); ASSERT_TRUE((*columns[2])[0].isNull()); ASSERT_TRUE((*columns[2])[1].isNull()); diff --git a/src/Columns/tests/gtest_low_cardinality.cpp b/src/Columns/tests/gtest_low_cardinality.cpp index 5e01279b7df..ce16d2cadb1 100644 --- a/src/Columns/tests/gtest_low_cardinality.cpp +++ b/src/Columns/tests/gtest_low_cardinality.cpp @@ -20,13 +20,13 @@ void testLowCardinalityNumberInsert(const DataTypePtr & data_type) Field value; column->get(0, value); - ASSERT_EQ(value.get(), 15); + ASSERT_EQ(value.safeGet(), 15); column->get(1, value); - ASSERT_EQ(value.get(), 20); + ASSERT_EQ(value.safeGet(), 20); column->get(2, value); - ASSERT_EQ(value.get(), 25); + ASSERT_EQ(value.safeGet(), 25); } TEST(ColumnLowCardinality, Insert) diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h index 9f80ff727b4..bcbcc36c67a 100644 --- a/src/Common/CollectionOfDerived.h +++ b/src/Common/CollectionOfDerived.h @@ -168,7 +168,7 @@ private: records.emplace(it, type_idx, item); } - Records::const_iterator getImpl(std::type_index type_idx) const + typename Records::const_iterator getImpl(std::type_index type_idx) const { auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx); diff --git a/src/Common/FieldBinaryEncoding.cpp b/src/Common/FieldBinaryEncoding.cpp index 6c1a8496fe6..23263c988c3 100644 --- a/src/Common/FieldBinaryEncoding.cpp +++ b/src/Common/FieldBinaryEncoding.cpp @@ -208,7 +208,7 @@ void FieldVisitorEncodeBinary::operator() (const Map & x, WriteBuffer & buf) con writeVarUInt(size, buf); for (size_t i = 0; i < size; ++i) { - const Tuple & key_and_value = x[i].get(); + const Tuple & key_and_value = x[i].safeGet(); Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[0]); Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[1]); } diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index b825f188586..af9503ac046 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const return x != 0; } -bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get(); return x != 0; } +bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.safeGet(); return x != 0; } bool FieldVisitorSum::operator() (Null &) const { diff --git a/src/Common/FieldVisitorSum.h b/src/Common/FieldVisitorSum.h index cbb4c4a1de3..d28676b5093 100644 --- a/src/Common/FieldVisitorSum.h +++ b/src/Common/FieldVisitorSum.h @@ -37,7 +37,7 @@ public: template bool operator() (DecimalField & x) const { - x += rhs.get>(); + x += rhs.safeGet>(); return x.getValue() != T(0); } diff --git a/src/Common/FieldVisitorToString.cpp b/src/Common/FieldVisitorToString.cpp index c4cb4266418..2148bac20d1 100644 --- a/src/Common/FieldVisitorToString.cpp +++ b/src/Common/FieldVisitorToString.cpp @@ -172,7 +172,7 @@ String FieldVisitorToString::operator() (const Object & x) const String convertFieldToString(const Field & field) { if (field.getType() == Field::Types::Which::String) - return field.get(); + return field.safeGet(); return applyVisitor(FieldVisitorToString(), field); } diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index a26797a687a..92621db5558 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -297,7 +297,7 @@ public: } /// Only inserts the value if key isn't already present - void ALWAYS_INLINE insertIfNotPresent(const Key & x, const Cell::Mapped & value) + void ALWAYS_INLINE insertIfNotPresent(const Key & x, const typename Cell::Mapped & value) { LookupResult it; bool inserted; diff --git a/src/Common/examples/arena_with_free_lists.cpp b/src/Common/examples/arena_with_free_lists.cpp index 6793d567aca..3a1304e2d94 100644 --- a/src/Common/examples/arena_with_free_lists.cpp +++ b/src/Common/examples/arena_with_free_lists.cpp @@ -174,19 +174,19 @@ struct Dictionary { switch (attribute.type) { - case AttributeUnderlyingTypeTest::UInt8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::UInt16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::UInt32: std::get>(attribute.arrays)[idx] = static_cast(value.get()); break; - case AttributeUnderlyingTypeTest::UInt64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Int8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Int16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Int32: std::get>(attribute.arrays)[idx] = static_cast(value.get()); break; - case AttributeUnderlyingTypeTest::Int64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingTypeTest::Float32: std::get>(attribute.arrays)[idx] = static_cast(value.get()); break; - case AttributeUnderlyingTypeTest::Float64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingTypeTest::UInt8: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::UInt16: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::UInt32: std::get>(attribute.arrays)[idx] = static_cast(value.safeGet()); break; + case AttributeUnderlyingTypeTest::UInt64: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Int8: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Int16: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Int32: std::get>(attribute.arrays)[idx] = static_cast(value.safeGet()); break; + case AttributeUnderlyingTypeTest::Int64: std::get>(attribute.arrays)[idx] = value.safeGet(); break; + case AttributeUnderlyingTypeTest::Float32: std::get>(attribute.arrays)[idx] = static_cast(value.safeGet()); break; + case AttributeUnderlyingTypeTest::Float64: std::get>(attribute.arrays)[idx] = value.safeGet(); break; case AttributeUnderlyingTypeTest::String: { - const auto & string = value.get(); + const auto & string = value.safeGet(); auto & string_ref = std::get>(attribute.arrays)[idx]; const auto & null_value_ref = std::get(attribute.null_values); diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index af681cd5639..c2bcf6ec651 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -49,7 +49,7 @@ ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context) { initReadBuffer(); initSampleBlock(); - auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").get()); + auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").safeGet()); auto data = std::make_unique(); data->pipe = std::make_unique(); diff --git a/src/Core/Field.h b/src/Core/Field.h index f1bb4a72b0d..689ac38a235 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -457,15 +457,6 @@ public: std::string_view getTypeName() const; bool isNull() const { return which == Types::Null; } - template - NearestFieldType> & get(); - - template - const auto & get() const - { - auto * mutable_this = const_cast *>(this); - return mutable_this->get(); - } bool isNegativeInfinity() const { return which == Types::Null && get().isNegativeInfinity(); } bool isPositiveInfinity() const { return which == Types::Null && get().isPositiveInfinity(); } @@ -681,6 +672,25 @@ private: Types::Which which; + /// This function is prone to type punning and should never be used outside of Field class, + /// whenever it is used within this class the stored type should be checked in advance. + template + NearestFieldType> & get() + { + // Before storing the value in the Field, we static_cast it to the field + // storage type, so here we return the value of storage type as well. + // Otherwise, it is easy to make a mistake of reinterpret_casting the stored + // value to a different and incompatible type. + // For example, a Float32 value is stored as Float64, and it is incorrect to + // return a reference to this value as Float32. + return *reinterpret_cast>*>(&storage); + } + + template + NearestFieldType> & get() const + { + return const_cast(this)->get(); + } /// Assuming there was no allocated state or it was deallocated (see destroy). template @@ -859,55 +869,18 @@ constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t) || t == Field::Types::UInt64; } -constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t) -{ - return t == Field::Types::Int64 - || t == Field::Types::UInt64 - || t == Field::Types::Bool; -} - -// Field value getter with type checking in debug builds. -template -NearestFieldType> & Field::get() -{ - // Before storing the value in the Field, we static_cast it to the field - // storage type, so here we return the value of storage type as well. - // Otherwise, it is easy to make a mistake of reinterpret_casting the stored - // value to a different and incompatible type. - // For example, a Float32 value is stored as Float64, and it is incorrect to - // return a reference to this value as Float32. - using StoredType = NearestFieldType>; - -#ifndef NDEBUG - // Disregard signedness when converting between int64 types. - constexpr Field::Types::Which target = TypeToEnum::value; - if (target != which - && (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)) && target != Field::Types::IPv4) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid Field get from type {} to type {}", which, target); -#endif - - StoredType * MAY_ALIAS ptr = reinterpret_cast(&storage); - - return *ptr; -} - - template auto & Field::safeGet() { const Types::Which target = TypeToEnum>>::value; - /// We allow converting int64 <-> uint64, int64 <-> bool, uint64 <-> bool in safeGet(). - if (target != which - && (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which))) - throw Exception(ErrorCodes::BAD_GET, - "Bad get: has {}, requested {}", getTypeName(), target); + /// bool is stored as uint64, will be returned as UInt64 when requested as bool or UInt64, as Int64 when requested as Int64 + if (target != which && !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64))) + throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); return get(); } - template requires not_field_or_bool_or_stringlike Field::Field(T && rhs) diff --git a/src/Core/Range.cpp b/src/Core/Range.cpp index 956b96653a1..1a5ce1e012e 100644 --- a/src/Core/Range.cpp +++ b/src/Core/Range.cpp @@ -62,27 +62,27 @@ void Range::shrinkToIncludedIfPossible() { if (left.isExplicit() && !left_included) { - if (left.getType() == Field::Types::UInt64 && left.get() != std::numeric_limits::max()) + if (left.getType() == Field::Types::UInt64 && left.safeGet() != std::numeric_limits::max()) { - ++left.get(); + ++left.safeGet(); left_included = true; } - if (left.getType() == Field::Types::Int64 && left.get() != std::numeric_limits::max()) + if (left.getType() == Field::Types::Int64 && left.safeGet() != std::numeric_limits::max()) { - ++left.get(); + ++left.safeGet(); left_included = true; } } if (right.isExplicit() && !right_included) { - if (right.getType() == Field::Types::UInt64 && right.get() != std::numeric_limits::min()) + if (right.getType() == Field::Types::UInt64 && right.safeGet() != std::numeric_limits::min()) { - --right.get(); + --right.safeGet(); right_included = true; } - if (right.getType() == Field::Types::Int64 && right.get() != std::numeric_limits::min()) + if (right.getType() == Field::Types::Int64 && right.safeGet() != std::numeric_limits::min()) { - --right.get(); + --right.safeGet(); right_included = true; } } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9c9c9c1db00..45bd2b9eb42 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -118,7 +118,7 @@ void Settings::set(std::string_view name, const Field & value) { if (value.getType() != Field::Types::Which::String) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type of value for setting 'compatibility'. Expected String, got {}", value.getTypeName()); - applyCompatibilitySetting(value.get()); + applyCompatibilitySetting(value.safeGet()); } /// If we change setting that was changed by compatibility setting before /// we should remove it from settings_changed_by_compatibility_setting, diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 7d094e2a107..bb2ef58b214 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -53,29 +53,29 @@ namespace { if (f.getType() == Field::Types::String) { - return stringToNumber(f.get()); + return stringToNumber(f.safeGet()); } else if (f.getType() == Field::Types::UInt64) { T result; - if (!accurate::convertNumeric(f.get(), result)) + if (!accurate::convertNumeric(f.safeGet(), result)) throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); return result; } else if (f.getType() == Field::Types::Int64) { T result; - if (!accurate::convertNumeric(f.get(), result)) + if (!accurate::convertNumeric(f.safeGet(), result)) throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name())); return result; } else if (f.getType() == Field::Types::Bool) { - return T(f.get()); + return T(f.safeGet()); } else if (f.getType() == Field::Types::Float64) { - Float64 x = f.get(); + Float64 x = f.safeGet(); if constexpr (std::is_floating_point_v) { return T(x); @@ -120,7 +120,7 @@ namespace if (f.getType() == Field::Types::String) { /// Allow to parse Map from string field. For the convenience. - const auto & str = f.get(); + const auto & str = f.safeGet(); return stringToMap(str); } @@ -218,7 +218,7 @@ namespace UInt64 fieldToMaxThreads(const Field & f) { if (f.getType() == Field::Types::String) - return stringToMaxThreads(f.get()); + return stringToMaxThreads(f.safeGet()); else return fieldToNumber(f); } diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 5541cc19653..3127a5ef36d 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -100,7 +100,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) return current_value; }; - UInt64 max_threads = get_current_value("max_threads").get(); + UInt64 max_threads = get_current_value("max_threads").safeGet(); UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores(); if (max_threads > max_threads_max_value) { @@ -120,7 +120,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) "input_format_parquet_max_block_size"}; for (auto const & setting : block_rows_settings) { - if (auto block_size = get_current_value(setting).get(); + if (auto block_size = get_current_value(setting).safeGet(); block_size > max_sane_block_rows_size) { if (log) @@ -129,7 +129,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log) } } - if (auto max_block_size = get_current_value("max_block_size").get(); max_block_size == 0) + if (auto max_block_size = get_current_value("max_block_size").safeGet(); max_block_size == 0) { if (log) LOG_WARNING(log, "Sanity check: 'max_block_size' cannot be 0. Set to default value {}", DEFAULT_BLOCK_SIZE); diff --git a/src/Core/examples/field.cpp b/src/Core/examples/field.cpp index 110e11d0cb1..3064290e127 100644 --- a/src/Core/examples/field.cpp +++ b/src/Core/examples/field.cpp @@ -37,7 +37,7 @@ int main(int argc, char ** argv) std::cerr << applyVisitor(to_string, field) << std::endl; } - field.get().push_back(field); + field.safeGet().push_back(field); std::cerr << applyVisitor(to_string, field) << std::endl; std::cerr << (field < field2) << std::endl; diff --git a/src/Core/tests/gtest_field.cpp b/src/Core/tests/gtest_field.cpp index 5585442d835..7e778be9575 100644 --- a/src/Core/tests/gtest_field.cpp +++ b/src/Core/tests/gtest_field.cpp @@ -8,31 +8,31 @@ GTEST_TEST(Field, FromBool) { Field f{false}; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 0); - ASSERT_EQ(f.get(), false); + ASSERT_EQ(f.safeGet(), 0); + ASSERT_EQ(f.safeGet(), false); } { Field f{true}; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 1); - ASSERT_EQ(f.get(), true); + ASSERT_EQ(f.safeGet(), 1); + ASSERT_EQ(f.safeGet(), true); } { Field f; f = false; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 0); - ASSERT_EQ(f.get(), false); + ASSERT_EQ(f.safeGet(), 0); + ASSERT_EQ(f.safeGet(), false); } { Field f; f = true; ASSERT_EQ(f.getType(), Field::Types::Bool); - ASSERT_EQ(f.get(), 1); - ASSERT_EQ(f.get(), true); + ASSERT_EQ(f.safeGet(), 1); + ASSERT_EQ(f.safeGet(), true); } } @@ -42,15 +42,15 @@ GTEST_TEST(Field, Move) Field f; f = Field{String{"Hello, world (1)"}}; - ASSERT_EQ(f.get(), "Hello, world (1)"); + ASSERT_EQ(f.safeGet(), "Hello, world (1)"); f = Field{String{"Hello, world (2)"}}; - ASSERT_EQ(f.get(), "Hello, world (2)"); + ASSERT_EQ(f.safeGet(), "Hello, world (2)"); f = Field{Array{Field{String{"Hello, world (3)"}}}}; - ASSERT_EQ(f.get()[0].get(), "Hello, world (3)"); + ASSERT_EQ(f.safeGet()[0].safeGet(), "Hello, world (3)"); f = String{"Hello, world (4)"}; - ASSERT_EQ(f.get(), "Hello, world (4)"); + ASSERT_EQ(f.safeGet(), "Hello, world (4)"); f = Array{Field{String{"Hello, world (5)"}}}; - ASSERT_EQ(f.get()[0].get(), "Hello, world (5)"); + ASSERT_EQ(f.safeGet()[0].safeGet(), "Hello, world (5)"); f = Array{String{"Hello, world (6)"}}; - ASSERT_EQ(f.get()[0].get(), "Hello, world (6)"); + ASSERT_EQ(f.safeGet()[0].safeGet(), "Hello, world (6)"); } diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 09175617bf1..dbb713bc382 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -119,7 +119,7 @@ MutableColumnPtr DataTypeAggregateFunction::createColumn() const Field DataTypeAggregateFunction::getDefault() const { Field field = AggregateFunctionStateData(); - field.get().name = getName(); + field.safeGet().name = getName(); AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData()); AggregateDataPtr place = place_buffer.data(); @@ -128,7 +128,7 @@ Field DataTypeAggregateFunction::getDefault() const try { - WriteBufferFromString buffer_from_field(field.get().data); + WriteBufferFromString buffer_from_field(field.safeGet().data); function->serialize(place, buffer_from_field, version); } catch (...) diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index a1b1f8325f0..e0430272479 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -71,10 +71,10 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() == 0 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS) + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.safeGet() == 0 || literal->value.safeGet() > ColumnVariant::MAX_NESTED_COLUMNS) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255"); - return std::make_shared(literal->value.get()); + return std::make_shared(literal->value.safeGet()); } void registerDataTypeDynamic(DataTypeFactory & factory) diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index 08e0c0d2045..b9a5a1a5a68 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -122,12 +122,12 @@ Field DataTypeEnum::castToName(const Field & value_or_name) const { if (value_or_name.getType() == Field::Types::String) { - this->getValue(value_or_name.get()); /// Check correctness - return value_or_name.get(); + this->getValue(value_or_name.safeGet()); /// Check correctness + return value_or_name.safeGet(); } else if (value_or_name.getType() == Field::Types::Int64) { - Int64 value = value_or_name.get(); + Int64 value = value_or_name.safeGet(); checkOverflow(value); return this->getNameForValue(static_cast(value)).toString(); } @@ -141,12 +141,12 @@ Field DataTypeEnum::castToValue(const Field & value_or_name) const { if (value_or_name.getType() == Field::Types::String) { - return this->getValue(value_or_name.get()); + return this->getValue(value_or_name.safeGet()); } else if (value_or_name.getType() == Field::Types::Int64 || value_or_name.getType() == Field::Types::UInt64) { - Int64 value = value_or_name.get(); + Int64 value = value_or_name.safeGet(); checkOverflow(value); this->getNameForValue(static_cast(value)); /// Check correctness return value; @@ -220,7 +220,7 @@ static void autoAssignNumberForEnum(const ASTPtr & arguments) "Elements of Enum data type must be of form: " "'name' = number or 'name', where name is string literal and number is an integer"); - literal_child_assign_num = value_literal->value.get(); + literal_child_assign_num = value_literal->value.safeGet(); } assign_number_child.emplace_back(child); } @@ -269,8 +269,8 @@ static DataTypePtr createExact(const ASTPtr & arguments) "Elements of Enum data type must be of form: " "'name' = number or 'name', where name is string literal and number is an integer"); - const String & field_name = name_literal->value.get(); - const auto value = value_literal->value.get(); + const String & field_name = name_literal->value.safeGet(); + const auto value = value_literal->value.safeGet(); if (value > std::numeric_limits::max() || value < std::numeric_limits::min()) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value {} for element '{}' exceeds range of {}", @@ -302,7 +302,7 @@ static DataTypePtr create(const ASTPtr & arguments) "Elements of Enum data type must be of form: " "'name' = number or 'name', where name is string literal and number is an integer"); - Int64 value = value_literal->value.get(); + Int64 value = value_literal->value.safeGet(); if (value > std::numeric_limits::max() || value < std::numeric_limits::min()) return createExact(arguments); diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index 080ff8826a5..63d5245287f 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -51,11 +51,11 @@ static DataTypePtr create(const ASTPtr & arguments) "FixedString data type family must have exactly one argument - size in bytes"); const auto * argument = arguments->children[0]->as(); - if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get() == 0) + if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.safeGet() == 0) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "FixedString data type family must have a number (positive integer) as its argument"); - return std::make_shared(argument->value.get()); + return std::make_shared(argument->value.safeGet()); } diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 91b9bfcb2a5..5fd69688194 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -69,7 +69,7 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Object data type family must have a const string as its schema name parameter"); - return std::make_shared(literal->value.get(), is_nullable); + return std::make_shared(literal->value.safeGet(), is_nullable); } void registerDataTypeObject(DataTypeFactory & factory) diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index a427fd0717a..1d8f7711de1 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -80,14 +80,14 @@ static DataTypePtr create(const ASTPtr & arguments) const auto * precision_arg = arguments->children[0]->as(); if (!precision_arg || precision_arg->value.getType() != Field::Types::UInt64) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument precision is invalid"); - precision = precision_arg->value.get(); + precision = precision_arg->value.safeGet(); if (arguments->children.size() == 2) { const auto * scale_arg = arguments->children[1]->as(); if (!scale_arg || !isInt64OrUInt64FieldType(scale_arg->value.getType())) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal argument scale is invalid"); - scale = scale_arg->value.get(); + scale = scale_arg->value.safeGet(); } } @@ -107,7 +107,7 @@ static DataTypePtr createExact(const ASTPtr & arguments) "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); UInt64 precision = DecimalUtils::max_precision; - UInt64 scale = scale_arg->value.get(); + UInt64 scale = scale_arg->value.safeGet(); return createDecimal(precision, scale); } diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 1d525e5987f..a7f2ff83873 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -104,7 +104,7 @@ Array createEmptyArrayField(size_t num_dimensions) for (size_t i = 1; i < num_dimensions; ++i) { current_array->push_back(Array()); - current_array = ¤t_array->back().get(); + current_array = ¤t_array->back().safeGet(); } return array; diff --git a/src/DataTypes/Serializations/JSONDataParser.cpp b/src/DataTypes/Serializations/JSONDataParser.cpp index 56641424396..0f74815f5b4 100644 --- a/src/DataTypes/Serializations/JSONDataParser.cpp +++ b/src/DataTypes/Serializations/JSONDataParser.cpp @@ -131,7 +131,7 @@ void JSONDataParser::traverseArrayElement(const Element & element, P auto nested_hash = getHashOfNestedPath(paths[i], values[i]); if (nested_hash) { - size_t array_size = values[i].template get().size(); + size_t array_size = values[i].template safeGet().size(); auto & current_nested_sizes = ctx.nested_sizes_by_path[*nested_hash]; if (current_nested_sizes.size() == ctx.current_size) @@ -154,7 +154,7 @@ void JSONDataParser::traverseArrayElement(const Element & element, P auto nested_hash = getHashOfNestedPath(paths[i], values[i]); if (nested_hash) { - size_t array_size = values[i].template get().size(); + size_t array_size = values[i].template safeGet().size(); auto & current_nested_sizes = ctx.nested_sizes_by_path[*nested_hash]; if (current_nested_sizes.empty()) diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 55f7641e058..41b198890e4 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -16,14 +16,14 @@ namespace DB void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - const AggregateFunctionStateData & state = field.get(); + const AggregateFunctionStateData & state = field.safeGet(); writeBinary(state.data, ostr); } void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { field = AggregateFunctionStateData(); - AggregateFunctionStateData & s = field.get(); + AggregateFunctionStateData & s = field.safeGet(); readBinary(s.data, istr); s.name = type_name; } diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index b7d43332085..0a9c4529e23 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -29,7 +29,7 @@ static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40; void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const Array & a = field.get(); + const Array & a = field.safeGet(); writeVarUInt(a.size(), ostr); for (const auto & i : a) { @@ -51,7 +51,7 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, con settings.binary.max_binary_string_size); field = Array(); - Array & arr = field.get(); + Array & arr = field.safeGet(); arr.reserve(size); for (size_t i = 0; i < size; ++i) nested->deserializeBinary(arr.emplace_back(), istr, settings); diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index 49dc042e872..8927f949368 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -13,7 +13,7 @@ namespace DB template void SerializationDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - FieldType x = field.get>(); + FieldType x = field.safeGet>(); writeBinaryLittleEndian(x, ostr); } diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index f919dc16d33..688c71792fa 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -28,7 +28,7 @@ static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30; void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - const String & s = field.get(); + const String & s = field.safeGet(); ostr.write(s.data(), std::min(s.size(), n)); if (s.size() < n) for (size_t i = s.size(); i < n; ++i) @@ -39,7 +39,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { field = String(); - String & s = field.get(); + String & s = field.safeGet(); s.resize(n); istr.readStrict(s.data(), n); } diff --git a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp index dfcd24aff58..c1beceb4533 100644 --- a/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp +++ b/src/DataTypes/Serializations/SerializationIPv4andIPv6.cpp @@ -125,7 +125,7 @@ bool SerializationIP::tryDeserializeTextCSV(DB::IColumn & column, DB::ReadB template void SerializationIP::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - IPv x = field.get(); + IPv x = field.safeGet(); if constexpr (std::is_same_v) writeBinary(x, ostr); else diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 0bef3c7d79d..c722b3ac7a1 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -40,7 +40,7 @@ static IColumn & extractNestedColumn(IColumn & column) void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & map = field.get(); + const auto & map = field.safeGet(); writeVarUInt(map.size(), ostr); for (const auto & elem : map) { @@ -63,7 +63,7 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const size, settings.binary.max_binary_string_size); field = Map(); - Map & map = field.get(); + Map & map = field.safeGet(); map.reserve(size); for (size_t i = 0; i < size; ++i) { diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index bdb4dfc6735..bfc13af8ca3 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -169,7 +169,7 @@ template void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 - typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); + typename ColumnVector::ValueType x = static_cast::ValueType>(field.safeGet()); writeBinaryLittleEndian(x, ostr); } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 9e523d0d745..ac5d4e3e128 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const String & s = field.get(); + const String & s = field.safeGet(); if (settings.binary.max_binary_string_size && s.size() > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, @@ -59,7 +59,7 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, co settings.binary.max_binary_string_size); field = String(); - String & s = field.get(); + String & s = field.safeGet(); s.resize(size); istr.readStrict(s.data(), size); } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 7a5227ca752..594a23ab507 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -34,7 +34,7 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_ void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto & tuple = field.get(); + const auto & tuple = field.safeGet(); for (size_t element_index = 0; element_index < elems.size(); ++element_index) { const auto & serialization = elems[element_index]; @@ -47,7 +47,7 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr, con const size_t size = elems.size(); field = Tuple(); - Tuple & tuple = field.get(); + Tuple & tuple = field.safeGet(); tuple.reserve(size); for (size_t i = 0; i < size; ++i) elems[i]->deserializeBinary(tuple.emplace_back(), istr, settings); diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 58178a896dc..f18466ad8ad 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -137,7 +137,7 @@ bool SerializationUUID::tryDeserializeTextCSV(IColumn & column, ReadBuffer & ist void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { - UUID x = field.get(); + UUID x = field.safeGet(); writeBinaryLittleEndian(x, ostr); } diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 84a52d4affb..9a632bd381b 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -55,7 +55,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume } } - return argument->value.get(); + return argument->value.safeGet(); } static DataTypePtr create(const ASTPtr & arguments) diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 67bce915168..b91aa84ecd3 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -183,7 +183,7 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction if (name->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(name->value.safeGet()); if (!maybe_qualified_name) return; @@ -194,7 +194,7 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction if (literal->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.safeGet()); /// Just return if name if invalid if (!maybe_qualified_name) return; diff --git a/src/Databases/DDLRenamingVisitor.cpp b/src/Databases/DDLRenamingVisitor.cpp index 38e100e2470..7556223b30e 100644 --- a/src/Databases/DDLRenamingVisitor.cpp +++ b/src/Databases/DDLRenamingVisitor.cpp @@ -180,7 +180,7 @@ namespace if (database_name_field && table_name_field) { - QualifiedTableName qualified_name{database_name_field->get(), table_name_field->get()}; + QualifiedTableName qualified_name{database_name_field->safeGet(), table_name_field->safeGet()}; if (!qualified_name.database.empty() && !qualified_name.table.empty()) { auto new_qualified_name = data.renaming_map.getNewTableName(qualified_name); @@ -207,7 +207,7 @@ namespace if (literal->value.getType() != Field::Types::String) return; - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.safeGet()); /// Just return if name if invalid if (!maybe_qualified_name || maybe_qualified_name->database.empty() || maybe_qualified_name->table.empty()) return; @@ -247,7 +247,7 @@ namespace if (!literal || (literal->value.getType() != Field::Types::String)) return; - auto database_name = literal->value.get(); + auto database_name = literal->value.safeGet(); if (database_name.empty()) return; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f127ccbc224..602bb29aebc 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -802,8 +802,8 @@ void DatabaseReplicated::checkTableEngine(const ASTCreateQuery & query, ASTStora if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String) return; - String maybe_path = arg1->value.get(); - String maybe_replica = arg2->value.get(); + String maybe_path = arg1->value.safeGet(); + String maybe_replica = arg2->value.safeGet(); /// Looks like it's ReplicatedMergeTree with explicit zookeeper_path and replica_name arguments. /// Let's ensure that some macros are used. diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 04b4070d5af..1364e9ae2b2 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -736,11 +736,11 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) - casted_column->insertValue(static_cast(value.template get())); + casted_column->insertValue(static_cast(value.template safeGet())); } }; @@ -776,17 +776,17 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { if (value.getType() == Field::Types::UInt64) - casted_int32_column->insertValue(static_cast(value.get())); + casted_int32_column->insertValue(static_cast(value.safeGet())); else if (value.getType() == Field::Types::Int64) { /// For MYSQL_TYPE_INT24 - const Int32 & num = static_cast(value.get()); + const Int32 & num = static_cast(value.safeGet()); casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); } else @@ -798,7 +798,7 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) @@ -812,12 +812,12 @@ static void writeFieldsToColumn( { for (size_t index = 0; index < rows_data.size(); ++index) { - const Tuple & row_data = rows_data[index].get(); + const Tuple & row_data = rows_data[index].safeGet(); const Field & value = row_data[column_index]; if (write_data_to_null_map(value, index)) { - const String & data = value.get(); + const String & data = value.safeGet(); casted_fixed_string_column->insertData(data.data(), data.size()); } } @@ -864,7 +864,7 @@ static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t { writeable_rows_mask[index + 1] = true; writeable_rows_mask[index] = differenceSortingKeys( - rows_data[index].get(), rows_data[index + 1].get(), sorting_columns_index); + rows_data[index].safeGet(), rows_data[index + 1].safeGet(), sorting_columns_index); } for (size_t column = 0; column < buffer.columns() - 2; ++column) diff --git a/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp index 11299c5b8b1..6f1ba26ee33 100644 --- a/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp +++ b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp @@ -281,12 +281,12 @@ static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false) ASSERT_EQ(write_event->table, "a"); ASSERT_EQ(write_event->rows.size(), 1); ASSERT_EQ(write_event->rows[0].getType(), Field::Types::Tuple); - auto row_data = write_event->rows[0].get(); + auto row_data = write_event->rows[0].safeGet(); ASSERT_EQ(row_data.size(), 4u); - ASSERT_EQ(row_data[0].get(), 1u); - ASSERT_EQ(row_data[1].get(), 1u); - ASSERT_EQ(row_data[2].get(), 1u); - ASSERT_EQ(row_data[3].get(), 1u); + ASSERT_EQ(row_data[0].safeGet(), 1u); + ASSERT_EQ(row_data[1].safeGet(), 1u); + ASSERT_EQ(row_data[2].safeGet(), 1u); + ASSERT_EQ(row_data[3].safeGet(), 1u); ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); ++count; @@ -342,18 +342,18 @@ static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false) ASSERT_EQ(update_event->table, "a"); ASSERT_EQ(update_event->rows.size(), 2); ASSERT_EQ(update_event->rows[0].getType(), Field::Types::Tuple); - row_data = update_event->rows[0].get(); + row_data = update_event->rows[0].safeGet(); ASSERT_EQ(row_data.size(), 4u); - ASSERT_EQ(row_data[0].get(), 1u); - ASSERT_EQ(row_data[1].get(), 1u); - ASSERT_EQ(row_data[2].get(), 1u); - ASSERT_EQ(row_data[3].get(), 1u); - row_data = update_event->rows[1].get(); + ASSERT_EQ(row_data[0].safeGet(), 1u); + ASSERT_EQ(row_data[1].safeGet(), 1u); + ASSERT_EQ(row_data[2].safeGet(), 1u); + ASSERT_EQ(row_data[3].safeGet(), 1u); + row_data = update_event->rows[1].safeGet(); ASSERT_EQ(row_data.size(), 4u); - ASSERT_EQ(row_data[0].get(), 1u); - ASSERT_EQ(row_data[1].get(), 2u); - ASSERT_EQ(row_data[2].get(), 1u); - ASSERT_EQ(row_data[3].get(), 1u); + ASSERT_EQ(row_data[0].safeGet(), 1u); + ASSERT_EQ(row_data[1].safeGet(), 2u); + ASSERT_EQ(row_data[2].safeGet(), 1u); + ASSERT_EQ(row_data[3].safeGet(), 1u); ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); ++count; diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index 47f99bd1093..781822533e9 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -395,13 +395,13 @@ private: } else if constexpr (std::is_same_v) { - const String & string_value = column_value.get(); + const String & string_value = column_value.safeGet(); StringRef inserted_value = copyStringInArena(arena, string_value); container.back() = inserted_value; } else { - container.back() = static_cast(column_value.get()); + container.back() = static_cast(column_value.safeGet()); } }); } @@ -441,7 +441,7 @@ private: } else if constexpr (std::is_same_v) { - const String & string_value = column_value.get(); + const String & string_value = column_value.safeGet(); StringRef inserted_value = copyStringInArena(arena, string_value); if (!cell_was_default) @@ -454,7 +454,7 @@ private: } else { - container[index_to_use] = static_cast(column_value.get()); + container[index_to_use] = static_cast(column_value.safeGet()); } }); } @@ -651,12 +651,12 @@ private: } else if constexpr (std::is_same_v) { - auto & value = default_value.get(); + auto & value = default_value.safeGet(); value_setter(value); } else { - value_setter(default_value.get()); + value_setter(default_value.safeGet()); } } else diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 64fc05e99ab..43fd39640c3 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -345,7 +345,7 @@ public: if (attribute_default_value.isNull()) default_value_is_null = true; else - default_value = static_cast(attribute_default_value.get()); + default_value = static_cast(attribute_default_value.safeGet()); } else { @@ -377,7 +377,7 @@ public: if constexpr (std::is_same_v) { Field field = (*default_values_column)[row]; - return field.get(); + return field.safeGet(); } else if constexpr (std::is_same_v) return default_values_column->getDataAt(row); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 999160226d9..b0233766741 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -245,7 +245,7 @@ ColumnPtr FlatDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr & std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const ContainerType & parent_keys = std::get>(hierarchical_attribute.container); @@ -300,7 +300,7 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy( std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const ContainerType & parent_keys = std::get>(hierarchical_attribute.container); @@ -701,7 +701,7 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, return; } - auto & attribute_value = value.get(); + auto & attribute_value = value.safeGet(); auto & container = std::get>(attribute.container); loaded_keys[key] = true; diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index d7d50dfb0a6..8768be8e5ec 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -240,7 +240,7 @@ ColumnPtr HashedArrayDictionary::getHierarchy(Colu std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); auto is_key_valid_func = [&, this](auto & key) @@ -313,7 +313,7 @@ ColumnUInt8::Ptr HashedArrayDictionary::isInHierar std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); auto is_key_valid_func = [&](auto & key) @@ -581,13 +581,13 @@ void HashedArrayDictionary::blockToAttributes(cons if constexpr (std::is_same_v) { - String & value_to_insert = column_value_to_insert.get(); + String & value_to_insert = column_value_to_insert.safeGet(); StringRef string_in_arena_reference = copyStringInArena(*string_arenas[shard], value_to_insert); attribute_container.back() = string_in_arena_reference; } else { - auto value_to_insert = static_cast(column_value_to_insert.get()); + auto value_to_insert = static_cast(column_value_to_insert.safeGet()); attribute_container.back() = value_to_insert; } }; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3a2b61e5149..7e935fe4855 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -636,7 +636,7 @@ ColumnPtr HashedDictionary::getHierarchy(C std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const CollectionsHolder & child_key_to_parent_key_maps = std::get>(hierarchical_attribute.containers); @@ -710,7 +710,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHie std::optional null_value; if (!dictionary_attribute.null_value.isNull()) - null_value = dictionary_attribute.null_value.get(); + null_value = dictionary_attribute.null_value.safeGet(); const CollectionsHolder & child_key_to_parent_key_maps = std::get>(hierarchical_attribute.containers); @@ -1004,13 +1004,13 @@ void HashedDictionary::blockToAttributes(c if constexpr (std::is_same_v) { - String & value_to_insert = column_value_to_insert.get(); + String & value_to_insert = column_value_to_insert.safeGet(); StringRef arena_value = copyStringInArena(*string_arenas[shard], value_to_insert); container.insert({key, arena_value}); } else { - auto value_to_insert = static_cast(column_value_to_insert.get()); + auto value_to_insert = static_cast(column_value_to_insert.safeGet()); container.insert({key, value_to_insert}); } diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index e1119982a34..de532ade26d 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -50,7 +50,7 @@ namespace std::optional null_value; if (!hierarchical_attribute.null_value.isNull()) - null_value = hierarchical_attribute.null_value.get(); + null_value = hierarchical_attribute.null_value.safeGet(); ColumnPtr key_to_request_column = ColumnVector::create(); auto * key_to_request_column_typed = static_cast *>(key_to_request_column->assumeMutable().get()); @@ -190,7 +190,7 @@ ColumnPtr getKeysHierarchyDefaultImplementation( std::optional null_value; if (!hierarchical_attribute.null_value.isNull()) - null_value = hierarchical_attribute.null_value.get(); + null_value = hierarchical_attribute.null_value.safeGet(); auto get_parent_key_func = [&](auto & key) { @@ -252,7 +252,7 @@ ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation( std::optional null_value; if (!hierarchical_attribute.null_value.isNull()) - null_value = hierarchical_attribute.null_value.get(); + null_value = hierarchical_attribute.null_value.safeGet(); auto get_parent_key_func = [&](auto & key) { diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 41fafcc162b..4f9e991752f 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -613,14 +613,14 @@ void IPAddressDictionary::calculateBytesAllocated() template void IPAddressDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.isNull() ? T{} : T(null_value.get()); + attribute.null_values = null_value.isNull() ? T{} : T(null_value.safeGet()); attribute.maps.emplace>(); } template <> void IPAddressDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { - attribute.null_values = null_value.isNull() ? String() : null_value.get(); + attribute.null_values = null_value.isNull() ? String() : null_value.safeGet(); attribute.maps.emplace>(); attribute.string_arena = std::make_unique(); } @@ -976,13 +976,13 @@ void IPAddressDictionary::setAttributeValue(Attribute & attribute, const Field & if constexpr (std::is_same_v) { - const auto & string = value.get(); + const auto & string = value.safeGet(); const auto * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); setAttributeValueImpl(attribute, StringRef{string_in_arena, string.size()}); } else { - setAttributeValueImpl(attribute, static_cast(value.get())); + setAttributeValueImpl(attribute, static_cast(value.safeGet())); } }; diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 46910fa9f6a..c30a6f90e44 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -1,7 +1,6 @@ #include "MongoDBDictionarySource.h" #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include "registerDictionaries.h" #include #include @@ -233,7 +232,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con } case AttributeUnderlyingType::String: { - String loaded_str((*key_columns[attribute_index])[row_idx].get()); + String loaded_str((*key_columns[attribute_index])[row_idx].safeGet()); /// Convert string to ObjectID if (key_attribute.is_object_id) { diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index dfc920623e3..ff29ca1f6b8 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -141,7 +141,7 @@ ColumnPtr IPolygonDictionary::getColumn( { getItemsShortCircuitImpl( requested_key_points, - [&](size_t row) { return (*attribute_values_column)[row].get(); }, + [&](size_t row) { return (*attribute_values_column)[row].safeGet(); }, [&](Array & value) { result_column_typed.insert(value); }, default_mask.value()); } @@ -149,7 +149,7 @@ ColumnPtr IPolygonDictionary::getColumn( { getItemsImpl( requested_key_points, - [&](size_t row) { return (*attribute_values_column)[row].get(); }, + [&](size_t row) { return (*attribute_values_column)[row].safeGet(); }, [&](Array & value) { result_column_typed.insert(value); }, default_value_provider.value()); } @@ -432,16 +432,16 @@ void IPolygonDictionary::getItemsImpl( } else if constexpr (std::is_same_v) { - set_value(default_value.get()); + set_value(default_value.safeGet()); } else if constexpr (std::is_same_v) { - auto default_value_string = default_value.get(); + auto default_value_string = default_value.safeGet(); set_value(default_value_string); } else { - set_value(default_value.get>()); + set_value(default_value.safeGet>()); } } } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index fc6c98990d0..c264b480bcb 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -906,13 +906,13 @@ void RangeHashedDictionary::setAttributeValue(Attribute & a if constexpr (std::is_same_v) { - const auto & string = value.get(); + const auto & string = value.safeGet(); StringRef string_ref = copyStringInArena(string_arena, string); value_to_insert = string_ref; } else { - value_to_insert = static_cast(value.get()); + value_to_insert = static_cast(value.safeGet()); } container.back() = value_to_insert; diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 1736cdff306..9db639a0ca4 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -1,7 +1,6 @@ #include "RedisDictionarySource.h" #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include "registerDictionaries.h" #include #include @@ -160,7 +159,7 @@ namespace DB if (isInteger(type)) key << DB::toString(key_columns[i]->get64(row)); else if (isString(type)) - key << (*key_columns[i])[row].get(); + key << (*key_columns[i])[row].safeGet(); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of key in Redis dictionary"); } diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index fd43f31a009..67bae372f87 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -113,7 +113,7 @@ std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr FlattenDiskConfigurationVisitor::Data data{context, attach}; FlattenDiskConfigurationVisitor{data}.visit(ast); - auto disk_name = assert_cast(*ast).value.get(); + auto disk_name = assert_cast(*ast).value.safeGet(); LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); return disk_name; } diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a7bd398cdaa..5dbafe48c3a 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1196,7 +1196,7 @@ struct ToYearImpl { if (point.getType() != Field::Types::UInt64) return std::nullopt; - auto year = point.get(); + auto year = point.safeGet(); if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt; const DateLUTImpl & date_lut = DateLUT::instance("UTC"); @@ -2001,7 +2001,7 @@ struct ToYYYYMMImpl { if (point.getType() != Field::Types::UInt64) return std::nullopt; - auto year_month = point.get(); + auto year_month = point.safeGet(); auto year = year_month / 100; auto month = year_month % 100; diff --git a/src/Functions/FunctionsConsistentHashing.h b/src/Functions/FunctionsConsistentHashing.h index 306b6395dc5..210bb69e16d 100644 --- a/src/Functions/FunctionsConsistentHashing.h +++ b/src/Functions/FunctionsConsistentHashing.h @@ -101,9 +101,9 @@ private: BucketsType num_buckets; if (buckets_field.getType() == Field::Types::Int64) - num_buckets = checkBucketsRange(buckets_field.get()); + num_buckets = checkBucketsRange(buckets_field.safeGet()); else if (buckets_field.getType() == Field::Types::UInt64) - num_buckets = checkBucketsRange(buckets_field.get()); + num_buckets = checkBucketsRange(buckets_field.safeGet()); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of the second argument of function {}", diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..0760b929652 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -117,7 +117,7 @@ UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) Field field; named_column.column->get(0, field); - return static_cast(field.get()); + return static_cast(field.safeGet()); } @@ -2604,8 +2604,8 @@ struct ToNumberMonotonicity if (left.isNull() || right.isNull()) return {}; - Float64 left_float = left.get(); - Float64 right_float = right.get(); + Float64 left_float = left.safeGet(); + Float64 right_float = right.safeGet(); if (left_float >= static_cast(std::numeric_limits::min()) && left_float <= static_cast(std::numeric_limits::max()) @@ -2633,11 +2633,11 @@ struct ToNumberMonotonicity const bool left_in_first_half = left.isNull() ? from_is_unsigned - : (left.get() >= 0); + : (left.safeGet() >= 0); const bool right_in_first_half = right.isNull() ? !from_is_unsigned - : (right.get() >= 0); + : (right.safeGet() >= 0); /// Size of type is the same. if (size_of_from == size_of_to) @@ -2675,7 +2675,7 @@ struct ToNumberMonotonicity return {}; /// Function cannot be monotonic when left and right are not on the same ranges. - if (divideByRangeOfType(left.get()) != divideByRangeOfType(right.get())) + if (divideByRangeOfType(left.safeGet()) != divideByRangeOfType(right.safeGet())) return {}; if (to_is_unsigned) @@ -2683,7 +2683,7 @@ struct ToNumberMonotonicity else { // If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly. - const bool is_monotonic = (T(left.get()) >= 0) == (T(right.get()) >= 0); + const bool is_monotonic = (T(left.safeGet()) >= 0) == (T(right.safeGet()) >= 0); return { .is_monotonic = is_monotonic }; } @@ -2707,13 +2707,13 @@ struct ToDateMonotonicity } else if ( ((left.getType() == Field::Types::UInt64 || left.isNull()) && (right.getType() == Field::Types::UInt64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) || ((left.getType() == Field::Types::Int64 || left.isNull()) && (right.getType() == Field::Types::Int64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) || (( (left.getType() == Field::Types::Float64 || left.isNull()) && (right.getType() == Field::Types::Float64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF)))) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF)))) || !isNativeNumber(type)) { return {}; @@ -2768,16 +2768,16 @@ struct ToStringMonotonicity if (left.getType() == Field::Types::UInt64 && right.getType() == Field::Types::UInt64) { - return (left.get() == 0 && right.get() == 0) - || (floor(log10(left.get())) == floor(log10(right.get()))) + return (left.safeGet() == 0 && right.safeGet() == 0) + || (floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) ? positive : not_monotonic; } if (left.getType() == Field::Types::Int64 && right.getType() == Field::Types::Int64) { - return (left.get() == 0 && right.get() == 0) - || (left.get() > 0 && right.get() > 0 && floor(log10(left.get())) == floor(log10(right.get()))) + return (left.safeGet() == 0 && right.safeGet() == 0) + || (left.safeGet() > 0 && right.safeGet() > 0 && floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) ? positive : not_monotonic; } @@ -4673,7 +4673,7 @@ private: return [function_name] ( ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) { - using ColumnEnumType = EnumType::ColumnType; + using ColumnEnumType = typename EnumType::ColumnType; const auto & first_col = arguments.front().column.get(); const auto & first_type = arguments.front().type.get(); diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index c35df8ba72d..1eaf0d1609a 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -243,7 +243,7 @@ private: } case MoveType::Index: { - Int64 index = (*arguments[j + 1].column)[row].get(); + Int64 index = (*arguments[j + 1].column)[row].safeGet(); if (!moveToElementByIndex(res_element, static_cast(index), key)) return false; break; diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 65d7473b945..ff0cff09c9e 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -701,11 +701,11 @@ ColumnPtr FunctionAnyArityLogical::getConstantResultForNonConstArgum bool constant_value_bool = false; if (field_type == Field::Types::Float64) - constant_value_bool = static_cast(constant_field_value.get()); + constant_value_bool = static_cast(constant_field_value.safeGet()); else if (field_type == Field::Types::Int64) - constant_value_bool = static_cast(constant_field_value.get()); + constant_value_bool = static_cast(constant_field_value.safeGet()); else if (field_type == Field::Types::UInt64) - constant_value_bool = static_cast(constant_field_value.get()); + constant_value_bool = static_cast(constant_field_value.safeGet()); has_true_constant = has_true_constant || constant_value_bool; has_false_constant = has_false_constant || !constant_value_bool; diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 6b65a5feaec..ed7fe1a5de1 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -500,7 +500,7 @@ inline Scale getScaleArg(const ColumnConst* scale_col) { const auto & scale_field = scale_col->getField(); - Int64 scale64 = scale_field.get(); + Int64 scale64 = scale_field.safeGet(); validateScale(scale64); return scale64; @@ -632,7 +632,7 @@ public: Scale raw_scale = scale64; DecimalRoundingImpl::applyOne(value_col_typed->getElement(i), value_col_typed->getScale(), - reinterpret_cast::NativeT&>(col_res->getElement(i)), raw_scale); + reinterpret_cast::NativeT&>(col_res->getElement(i)), raw_scale); } } } @@ -854,7 +854,7 @@ private: using ValueType = typename Container::value_type; std::vector boundary_values(boundaries.size()); for (size_t i = 0; i < boundaries.size(); ++i) - boundary_values[i] = static_cast(boundaries[i].get()); + boundary_values[i] = static_cast(boundaries[i].safeGet()); ::sort(boundary_values.begin(), boundary_values.end()); boundary_values.erase(std::unique(boundary_values.begin(), boundary_values.end()), boundary_values.end()); diff --git a/src/Functions/IFunctionCustomWeek.h b/src/Functions/IFunctionCustomWeek.h index 51542c9cab1..ba0baa35819 100644 --- a/src/Functions/IFunctionCustomWeek.h +++ b/src/Functions/IFunctionCustomWeek.h @@ -50,15 +50,15 @@ public: if (checkAndGetDataType(&type)) { - return Transform::FactorTransform::execute(UInt16(left.get()), date_lut) - == Transform::FactorTransform::execute(UInt16(right.get()), date_lut) + return Transform::FactorTransform::execute(UInt16(left.safeGet()), date_lut) + == Transform::FactorTransform::execute(UInt16(right.safeGet()), date_lut) ? is_monotonic : is_not_monotonic; } else { - return Transform::FactorTransform::execute(UInt32(left.get()), date_lut) - == Transform::FactorTransform::execute(UInt32(right.get()), date_lut) + return Transform::FactorTransform::execute(UInt32(left.safeGet()), date_lut) + == Transform::FactorTransform::execute(UInt32(right.safeGet()), date_lut) ? is_monotonic : is_not_monotonic; } diff --git a/src/Functions/IFunctionDateOrDateTime.h b/src/Functions/IFunctionDateOrDateTime.h index 762b79bfafc..899aa2c305d 100644 --- a/src/Functions/IFunctionDateOrDateTime.h +++ b/src/Functions/IFunctionDateOrDateTime.h @@ -72,30 +72,30 @@ public: if (checkAndGetDataType(type_ptr)) { - return Transform::FactorTransform::execute(UInt16(left.get()), *date_lut) - == Transform::FactorTransform::execute(UInt16(right.get()), *date_lut) + return Transform::FactorTransform::execute(UInt16(left.safeGet()), *date_lut) + == Transform::FactorTransform::execute(UInt16(right.safeGet()), *date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(type_ptr)) { - return Transform::FactorTransform::execute(Int32(left.get()), *date_lut) - == Transform::FactorTransform::execute(Int32(right.get()), *date_lut) + return Transform::FactorTransform::execute(Int32(left.safeGet()), *date_lut) + == Transform::FactorTransform::execute(Int32(right.safeGet()), *date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(type_ptr)) { - return Transform::FactorTransform::execute(UInt32(left.get()), *date_lut) - == Transform::FactorTransform::execute(UInt32(right.get()), *date_lut) + return Transform::FactorTransform::execute(UInt32(left.safeGet()), *date_lut) + == Transform::FactorTransform::execute(UInt32(right.safeGet()), *date_lut) ? is_monotonic : is_not_monotonic; } else { assert(checkAndGetDataType(type_ptr)); - const auto & left_date_time = left.get(); + const auto & left_date_time = left.safeGet(); TransformDateTime64 transformer_left(left_date_time.getScale()); - const auto & right_date_time = right.get(); + const auto & right_date_time = right.safeGet(); TransformDateTime64 transformer_right(right_date_time.getScale()); return transformer_left.execute(left_date_time.getValue(), *date_lut) diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp index fb74018b330..84ac0ff08f3 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathRange.cpp @@ -46,7 +46,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte { return false; } - range_indices.first = static_cast(number_ptr->as()->value.get()); + range_indices.first = static_cast(number_ptr->as()->value.safeGet()); if (pos->type == TokenType::Comma || pos->type == TokenType::ClosingSquareBracket) { @@ -63,7 +63,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte { return false; } - range_indices.second = static_cast(number_ptr->as()->value.get()); + range_indices.second = static_cast(number_ptr->as()->value.safeGet()); } else { diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index 3e9c8fba215..c1a2fb6be2d 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -75,7 +75,7 @@ struct MultiMatchAllIndicesImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 20b2150048b..ce6a054c064 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -89,7 +89,7 @@ struct MultiMatchAnyImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); diff --git a/src/Functions/MultiSearchAllPositionsImpl.h b/src/Functions/MultiSearchAllPositionsImpl.h index cfe60e51bcd..6c2cd215638 100644 --- a/src/Functions/MultiSearchAllPositionsImpl.h +++ b/src/Functions/MultiSearchAllPositionsImpl.h @@ -33,7 +33,7 @@ struct MultiSearchAllPositionsImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 36a5fd514d9..3c6ec8ead44 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -44,7 +44,7 @@ struct MultiSearchFirstIndexImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto searcher = Impl::createMultiSearcherInBigHaystack(needles); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index ccdd82a0ee5..b5c68ad664d 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -44,7 +44,7 @@ struct MultiSearchFirstPositionImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 { diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 467cc96a95f..3eb8e6fb627 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -44,7 +44,7 @@ struct MultiSearchImpl std::vector needles; needles.reserve(needles_arr.size()); for (const auto & needle : needles_arr) - needles.emplace_back(needle.get()); + needles.emplace_back(needle.safeGet()); auto searcher = Impl::createMultiSearcherInBigHaystack(needles); diff --git a/src/Functions/URL/cutURLParameter.cpp b/src/Functions/URL/cutURLParameter.cpp index 3ab9cad1ea7..4439e79e962 100644 --- a/src/Functions/URL/cutURLParameter.cpp +++ b/src/Functions/URL/cutURLParameter.cpp @@ -156,7 +156,7 @@ public: for (size_t j = 0; j < num_needles; ++j) { auto field = col_needle_const_array->getData()[j]; - cutURL(res_data, field.get(), res_offset, cur_res_offset); + cutURL(res_data, field.safeGet(), res_offset, cur_res_offset); } } else diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 227b29d5d9f..81f3f97979b 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -904,10 +904,10 @@ ColumnPtr FunctionArrayElement::executeNumberConst( return nullptr; if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { ArrayElementNumImpl::template vectorConst( - col_nested->getData(), col_array->getOffsets(), index.get() - 1, col_res_vec->getData(), builder); + col_nested->getData(), col_array->getOffsets(), index.safeGet() - 1, col_res_vec->getData(), builder); } else if (index.getType() == Field::Types::Int64) { @@ -972,14 +972,14 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument auto col_res = ColumnString::create(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { if (builder) ArrayElementStringImpl::vectorConst( col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - index.get() - 1, + index.safeGet() - 1, col_res->getChars(), col_res->getOffsets(), builder); @@ -988,7 +988,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - index.get() - 1, + index.safeGet() - 1, col_res->getChars(), col_res->getOffsets(), builder); @@ -1000,7 +1000,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), col_res->getChars(), col_res->getOffsets(), builder); @@ -1009,7 +1009,7 @@ FunctionArrayElement::executeStringConst(const ColumnsWithTypeAndName & argument col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), col_res->getChars(), col_res->getOffsets(), builder); @@ -1046,7 +1046,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( auto res_offsets = ColumnArray::ColumnOffsets::create(); auto res_string_null_map = col_nullable ? ColumnUInt8::create() : nullptr; if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { if (col_nullable) ArrayElementArrayStringImpl::vectorConst( @@ -1055,7 +1055,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), &string_null_map->getData(), - index.get() - 1, + index.safeGet() - 1, res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1068,7 +1068,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), nullptr, - index.get() - 1, + index.safeGet() - 1, res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1084,7 +1084,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), &string_null_map->getData(), - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1097,7 +1097,7 @@ ColumnPtr FunctionArrayElement::executeArrayStringConst( col_nested_array->getOffsets(), col_nested_elem->getOffsets(), nullptr, - -(UInt64(index.get()) + 1), + -(UInt64(index.safeGet()) + 1), res_string->getChars(), res_offsets->getData(), res_string->getOffsets(), @@ -1153,7 +1153,7 @@ ColumnPtr FunctionArrayElement::executeArrayNumberConst( auto & res_offsets = res_array->getOffsets(); NullMap * res_null_map = res_nullable ? &res_nullable->getNullMapData() : nullptr; - if (index.getType() == Field::Types::UInt64 || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + if (index.getType() == Field::Types::UInt64 || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) { if (col_nullable) ArrayElementArrayNumImpl::template vectorConst( @@ -1161,7 +1161,7 @@ ColumnPtr FunctionArrayElement::executeArrayNumberConst( col_array->getOffsets(), col_nested_array->getOffsets(), null_map, - index.get() - 1, + index.safeGet() - 1, res_data->getData(), res_offsets, res_null_map, @@ -1172,7 +1172,7 @@ ColumnPtr FunctionArrayElement::executeArrayNumberConst( col_array->getOffsets(), col_nested_array->getOffsets(), null_map, - index.get() - 1, + index.safeGet() - 1, res_data->getData(), res_offsets, res_null_map, @@ -1392,12 +1392,12 @@ ColumnPtr FunctionArrayElement::executeGenericConst( auto col_res = col_nested.cloneEmpty(); if (index.getType() == Field::Types::UInt64 - || (index.getType() == Field::Types::Int64 && index.get() >= 0)) + || (index.getType() == Field::Types::Int64 && index.safeGet() >= 0)) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), index.get() - 1, *col_res, builder); + col_nested, col_array->getOffsets(), index.safeGet() - 1, *col_res, builder); else if (index.getType() == Field::Types::Int64) ArrayElementGenericImpl::vectorConst( - col_nested, col_array->getOffsets(), -(static_cast(index.get() + 1)), *col_res, builder); + col_nested, col_array->getOffsets(), -(static_cast(index.safeGet() + 1)), *col_res, builder); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal type of array index"); @@ -1789,7 +1789,7 @@ bool FunctionArrayElement::matchKeyToIndexStringConst( using DataColumn = std::decay_t; if (index.getType() != Field::Types::String) return false; - MatcherStringConst matcher{data_column, index.get()}; + MatcherStringConst matcher{data_column, index.safeGet()}; executeMatchKeyToIndex(offsets, matched_idxs, matcher); return true; }); diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index 86797cb5db0..614b01c2ac8 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -237,7 +237,7 @@ private: } arg.val_column->get(offset + j, temp_val); - ValType value = temp_val.get(); + ValType value = temp_val.safeGet(); if constexpr (op_type == OpTypes::ADD) { diff --git a/src/Functions/getClientHTTPHeader.cpp b/src/Functions/getClientHTTPHeader.cpp index 140f39d03b8..50a6275fc82 100644 --- a/src/Functions/getClientHTTPHeader.cpp +++ b/src/Functions/getClientHTTPHeader.cpp @@ -58,7 +58,7 @@ public: { Field header; source->get(row, header); - if (auto it = client_info.http_headers.find(header.get()); it != client_info.http_headers.end()) + if (auto it = client_info.http_headers.find(header.safeGet()); it != client_info.http_headers.end()) result->insert(it->second); else result->insertDefault(); diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index c4b675fcf6c..14b8b70b22c 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -200,7 +200,7 @@ public: if (value.isNull()) continue; - if (value.get() == 0) + if (value.safeGet() == 0) continue; instruction.condition_always_true = true; diff --git a/src/Functions/nested.cpp b/src/Functions/nested.cpp index bdaf57d65c9..85c342b5e7c 100644 --- a/src/Functions/nested.cpp +++ b/src/Functions/nested.cpp @@ -145,7 +145,7 @@ private: if (nested_names_field.getType() != Field::Types::Array) return {}; - const auto & nested_names_array = nested_names_field.get(); + const auto & nested_names_array = nested_names_field.safeGet(); Names nested_names; nested_names.reserve(nested_names_array.size()); @@ -155,7 +155,7 @@ private: if (nested_name_field.getType() != Field::Types::String) return {}; - nested_names.push_back(nested_name_field.get()); + nested_names.push_back(nested_name_field.safeGet()); } return nested_names; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index a794cdbcf05..59040bf1fea 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -294,21 +294,21 @@ void RequestSettings::finishInit(const DB::Settings & settings, bool validate_se /// to avoid losing token bucket state on every config reload, /// which could lead to exceeding limit for short time. /// But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = isChanged("max_get_rps") ? get("max_get_rps").get() : settings.s3_max_get_rps) + if (UInt64 max_get_rps = isChanged("max_get_rps") ? get("max_get_rps").safeGet() : settings.s3_max_get_rps) { size_t default_max_get_burst = settings.s3_max_get_burst ? settings.s3_max_get_burst : (Throttler::default_burst_seconds * max_get_rps); - size_t max_get_burst = isChanged("max_get_burts") ? get("max_get_burst").get() : default_max_get_burst; + size_t max_get_burst = isChanged("max_get_burts") ? get("max_get_burst").safeGet() : default_max_get_burst; get_request_throttler = std::make_shared(max_get_rps, max_get_burst); } - if (UInt64 max_put_rps = isChanged("max_put_rps") ? get("max_put_rps").get() : settings.s3_max_put_rps) + if (UInt64 max_put_rps = isChanged("max_put_rps") ? get("max_put_rps").safeGet() : settings.s3_max_put_rps) { size_t default_max_put_burst = settings.s3_max_put_burst ? settings.s3_max_put_burst : (Throttler::default_burst_seconds * max_put_rps); - size_t max_put_burst = isChanged("max_put_burts") ? get("max_put_burst").get() : default_max_put_burst; + size_t max_put_burst = isChanged("max_put_burts") ? get("max_put_burst").safeGet() : default_max_put_burst; put_request_throttler = std::make_shared(max_put_rps, max_put_burst); } } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e1b7e92ee5d..368eb8174f0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -131,7 +131,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Invalid type in set. Expected tuple, got {}", String(value.getTypeName())); - const auto & tuple = value.template get(); + const auto & tuple = value.template safeGet(); size_t tuple_size = tuple.size(); if (tuple_size != columns_num) throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect size of tuple in set: {} instead of {}", @@ -233,7 +233,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co "Invalid type of set. Expected tuple, got {}", function_result.getTypeName()); - tuple = &function_result.get(); + tuple = &function_result.safeGet(); } /// Tuple can be represented as a literal in AST. @@ -246,7 +246,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co "Invalid type in set. Expected tuple, got {}", literal->value.getTypeName()); - tuple = &literal->value.get(); + tuple = &literal->value.safeGet(); } assert(tuple || func); @@ -332,14 +332,14 @@ Block createBlockForSet( if (type_index == TypeIndex::Tuple) { const DataTypes & value_types = assert_cast(right_arg_type.get())->getElements(); - block = createBlockFromCollection(right_arg_value.get(), value_types, set_element_types, tranform_null_in); + block = createBlockFromCollection(right_arg_value.safeGet(), value_types, set_element_types, tranform_null_in); } else if (type_index == TypeIndex::Array) { const auto* right_arg_array_type = assert_cast(right_arg_type.get()); - size_t right_arg_array_size = right_arg_value.get().size(); + size_t right_arg_array_size = right_arg_value.safeGet().size(); DataTypes value_types(right_arg_array_size, right_arg_array_type->getNestedType()); - block = createBlockFromCollection(right_arg_value.get(), value_types, set_element_types, tranform_null_in); + block = createBlockFromCollection(right_arg_value.safeGet(), value_types, set_element_types, tranform_null_in); } else throw_unsupported_type(right_arg_type); diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 356bffa75e9..302a5e55c53 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -201,7 +201,7 @@ private: if (literal_value.getType() != Field::Types::String) continue; - auto dictionary_name = literal_value.get(); + auto dictionary_name = literal_value.safeGet(); auto qualified_dictionary_name = context->getExternalDictionariesLoader().qualifyDictionaryNameWithDatabase(dictionary_name, context); literal_value = qualified_dictionary_name.getFullName(); } diff --git a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp index 4f06f345b96..b9f7f37b338 100644 --- a/src/Interpreters/ComparisonTupleEliminationVisitor.cpp +++ b/src/Interpreters/ComparisonTupleEliminationVisitor.cpp @@ -22,7 +22,7 @@ ASTs splitTuple(const ASTPtr & node) if (const auto * literal = node->as(); literal && literal->value.getType() == Field::Types::Tuple) { ASTs result; - const auto & tuple = literal->value.get(); + const auto & tuple = literal->value.safeGet(); for (const auto & child : tuple) result.emplace_back(std::make_shared(child)); return result; diff --git a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp index 084bb0a1bb9..220355e0741 100644 --- a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp +++ b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp @@ -45,7 +45,7 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) if (!identifier || !literal || literal->value.getType() != Field::Types::String) continue; - String regexp = likePatternToRegexp(literal->value.get()); + String regexp = likePatternToRegexp(literal->value.safeGet()); /// Case insensitive. Works with UTF-8 as well. if (is_ilike) regexp = "(?i)" + regexp; @@ -61,7 +61,7 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) match->arguments->children.push_back(it->second); unique_elems.push_back(std::move(match)); } - it->second->value.get().push_back(regexp); + it->second->value.safeGet().push_back(regexp); } } diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.cpp b/src/Interpreters/ConvertStringsToEnumVisitor.cpp index 7cc95dc521b..d35baa92900 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.cpp +++ b/src/Interpreters/ConvertStringsToEnumVisitor.cpp @@ -33,8 +33,8 @@ String makeStringsEnum(const std::set & values) void changeIfArguments(ASTPtr & first, ASTPtr & second) { - String first_value = first->as()->value.get(); - String second_value = second->as()->value.get(); + String first_value = first->as()->value.safeGet(); + String second_value = second->as()->value.safeGet(); std::set values; values.insert(first_value); @@ -59,9 +59,9 @@ void changeTransformArguments(ASTPtr & array_to, ASTPtr & other) { std::set values; - for (const auto & item : array_to->as()->value.get()) - values.insert(item.get()); - values.insert(other->as()->value.get()); + for (const auto & item : array_to->as()->value.safeGet()) + values.insert(item.safeGet()); + values.insert(other->as()->value.safeGet()); String enum_string = makeStringsEnum(values); @@ -168,7 +168,7 @@ void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data if (literal_to->value.getTypeName() != "Array" || literal_other->value.getTypeName() != "String") return; - Array array_to = literal_to->value.get(); + Array array_to = literal_to->value.safeGet(); if (array_to.empty()) return; diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index fa197d59c13..6e08dd5e2cc 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -538,7 +538,7 @@ void DatabaseReplicatedTask::createSyncedNodeIfNeed(const ZooKeeperPtr & zookeep /// Bool type is really weird, sometimes it's Bool and sometimes it's UInt64... assert(value.getType() == Field::Types::Bool || value.getType() == Field::Types::UInt64); - if (!value.get()) + if (!value.safeGet()) return; zookeeper->createIfNotExists(getSyncedNodePath(), ""); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 971f90bd3cd..15c6aba8b62 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -689,7 +689,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( throw Exception(ErrorCodes::LOGICAL_ERROR, "Neither default value expression nor type is provided for a column"); if (col_decl.comment) - column.comment = col_decl.comment->as().value.get(); + column.comment = col_decl.comment->as().value.safeGet(); if (col_decl.codec) { @@ -1875,7 +1875,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, Cont if (has_explicit_zk_path_arg) { - String zk_path = create.storage->engine->arguments->children[0]->as()->value.get(); + String zk_path = create.storage->engine->arguments->children[0]->as()->value.safeGet(); Macros::MacroExpansionInfo info; info.table_id.uuid = create.uuid; info.ignore_unknown = true; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index bedd9cb4a80..c820f999e0c 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -332,7 +332,7 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) if (settings.hasBooleanSetting(change.name)) { - auto value = change.value.get(); + auto value = change.value.safeGet(); if (value > 1) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value {} for setting \"{}\". " "Expected boolean type", value, change.name); @@ -341,7 +341,7 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) } else { - auto value = change.value.get(); + auto value = change.value.safeGet(); settings.setIntegerSetting(change.name, value); } } diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 7eb487ba7b3..2c579f3b468 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -334,7 +334,7 @@ BlockIO InterpreterKillQueryQuery::execute() for (size_t i = 0; i < moves_block.rows(); ++i) { table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()}; - auto task_uuid = task_uuid_col[i].get(); + auto task_uuid = task_uuid_col[i].safeGet(); CancellationCode code = CancellationCode::Unknown; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cb42a8abf9c..0c79f4310ce 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1232,7 +1232,7 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery std::shared_ptr collator; if (order_by_elem.getCollation()) - collator = std::make_shared(order_by_elem.getCollation()->as().value.get()); + collator = std::make_shared(order_by_elem.getCollation()->as().value.safeGet()); if (order_by_elem.with_fill) { diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 57ad5caa4c7..dfffeb437d4 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -220,7 +220,7 @@ bool isStorageTouchedByMutations( Block tmp_block; while (executor.pull(tmp_block)); - auto count = (*block.getByName("count()").column)[0].get(); + auto count = (*block.getByName("count()").column)[0].safeGet(); return count != 0; } diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index dd205ae6508..913f9900b77 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -42,13 +42,13 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTy if (isDateOrDate32(column.type.get())) { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); + start_date_or_date_time = date_lut.dateToString(range.first.safeGet()); + end_date_or_date_time = date_lut.dateToString(range.second.safeGet()); } else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); + start_date_or_date_time = date_lut.timeToString(range.first.safeGet()); + end_date_or_date_time = date_lut.timeToString(range.second.safeGet()); } else [[unlikely]] return {}; diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 48c9988b6fc..e9a663d53b0 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -24,7 +24,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v if (literal->value.getType() == Field::Types::Int64 || literal->value.getType() == Field::Types::UInt64) { - value = literal->value.get(); + value = literal->value.safeGet(); return true; } if (literal->value.getType() == Field::Types::Null) @@ -51,7 +51,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v { if (type_literal->value.getType() == Field::Types::String) { - const auto & type_str = type_literal->value.get(); + const auto & type_str = type_literal->value.safeGet(); if (type_str == "UInt8" || type_str == "Nullable(UInt8)") return tryExtractConstValueFromCondition(expr_list->children.at(0), value); } diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 54515ea072a..6a0522b0676 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -72,7 +72,7 @@ bool shardContains( if (sharding_value.isNull()) return false; - UInt64 value = sharding_value.get(); + UInt64 value = sharding_value.safeGet(); const auto shard_num = data.slots[value % data.slots.size()] + 1; return data.shard_info.shard_num == shard_num; } @@ -120,11 +120,20 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d else if (auto * tuple_literal = right->as(); tuple_literal && tuple_literal->value.getType() == Field::Types::Tuple) { - auto & tuple = tuple_literal->value.get(); - std::erase_if(tuple, [&](auto & child) + auto & tuple = tuple_literal->value.safeGet(); + if (tuple.size() > 1) { - return tuple.size() > 1 && !shardContains(child, name, data); - }); + Tuple new_tuple; + + for (const auto & child : tuple) + if (shardContains(child, name, data)) + new_tuple.emplace_back(std::move(child)); + + if (new_tuple.size() == 0) + new_tuple.emplace_back(std::move(tuple.back())); + + tuple_literal->value = std::move(new_tuple); + } } } @@ -159,7 +168,7 @@ public: { if (isTuple(constant->getResultType())) { - const auto & tuple = constant->getValue().get(); + const auto & tuple = constant->getValue().safeGet(); Tuple new_tuple; new_tuple.reserve(tuple.size()); diff --git a/src/Interpreters/RewriteCountVariantsVisitor.cpp b/src/Interpreters/RewriteCountVariantsVisitor.cpp index 4a541c3765a..272e1ac735f 100644 --- a/src/Interpreters/RewriteCountVariantsVisitor.cpp +++ b/src/Interpreters/RewriteCountVariantsVisitor.cpp @@ -53,7 +53,7 @@ void RewriteCountVariantsVisitor::visit(ASTFunction & func) { if (first_arg_literal->value.getType() == Field::Types::UInt64) { - auto constant = first_arg_literal->value.get(); + auto constant = first_arg_literal->value.safeGet(); if (constant == 1 && !context->getSettingsRef().aggregate_functions_null_for_empty) transform = true; } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index b872eb94fde..6483dd3be48 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -184,7 +184,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) const auto & value = group_exprs[i]->as()->value; if (value.getType() == Field::Types::UInt64) { - auto pos = value.get(); + auto pos = value.safeGet(); if (pos > 0 && pos <= select_query->select()->children.size()) keep_position = true; } diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 31a881001e3..b1e12ff8048 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -94,8 +94,8 @@ void WindowFrame::checkValid() const if (begin_type == BoundaryType::Offset && !((begin_offset.getType() == Field::Types::UInt64 || begin_offset.getType() == Field::Types::Int64) - && begin_offset.get() >= 0 - && begin_offset.get() < INT_MAX)) + && begin_offset.safeGet() >= 0 + && begin_offset.safeGet() < INT_MAX)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", @@ -107,8 +107,8 @@ void WindowFrame::checkValid() const if (end_type == BoundaryType::Offset && !((end_offset.getType() == Field::Types::UInt64 || end_offset.getType() == Field::Types::Int64) - && end_offset.get() >= 0 - && end_offset.get() < INT_MAX)) + && end_offset.safeGet() >= 0 + && end_offset.safeGet() < INT_MAX)) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 1a7c166c6a5..f3957c3b69b 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -57,7 +57,7 @@ template Field convertNumericTypeImpl(const Field & from) { To result; - if (!accurate::convertNumeric(from.get(), result)) + if (!accurate::convertNumeric(from.safeGet(), result)) return {}; return result; } @@ -88,7 +88,7 @@ Field convertNumericType(const Field & from, const IDataType & type) template Field convertIntToDecimalType(const Field & from, const DataTypeDecimal & type) { - From value = from.get(); + From value = from.safeGet(); if (!type.canStoreWhole(value)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Number is too big to place in {}", type.getName()); @@ -100,7 +100,7 @@ Field convertIntToDecimalType(const Field & from, const DataTypeDecimal & typ template Field convertStringToDecimalType(const Field & from, const DataTypeDecimal & type) { - const String & str_value = from.get(); + const String & str_value = from.safeGet(); T value = type.parseFromString(str_value); return DecimalField(value, type.getScale()); } @@ -108,7 +108,7 @@ Field convertStringToDecimalType(const Field & from, const DataTypeDecimal & template Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal & type) { - auto field = from.get>(); + auto field = from.safeGet>(); T value = convertDecimals, DataTypeDecimal>(field.getValue(), field.getScale(), type.getScale()); return DecimalField(value, type.getScale()); } @@ -116,7 +116,7 @@ Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal & template Field convertFloatToDecimalType(const Field & from, const DataTypeDecimal & type) { - From value = from.get(); + From value = from.safeGet(); if (!type.canStoreWhole(value)) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Number is too big to place in {}", type.getName()); @@ -182,24 +182,24 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID /// Conversion between Date and DateTime and vice versa. if (which_type.isDate() && which_from_type.isDateTime()) { - return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get()).toUnderType()); + return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.safeGet()).toUnderType()); } else if (which_type.isDate32() && which_from_type.isDateTime()) { - return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get()).toUnderType()); + return static_cast(static_cast(*from_type_hint).getTimeZone().toDayNum(src.safeGet()).toUnderType()); } else if (which_type.isDateTime() && which_from_type.isDate()) { - return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return static_cast(type).getTimeZone().fromDayNum(DayNum(src.safeGet())); } else if (which_type.isDateTime() && which_from_type.isDate32()) { - return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return static_cast(type).getTimeZone().fromDayNum(DayNum(src.safeGet())); } else if (which_type.isDateTime64() && which_from_type.isDate()) { const auto & date_time64_type = static_cast(type); - const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.get())); + const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.safeGet())); return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), date_time64_type.getScale()); @@ -207,7 +207,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID else if (which_type.isDateTime64() && which_from_type.isDate32()) { const auto & date_time64_type = static_cast(type); - const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.get()))); + const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.safeGet()))); return DecimalField( DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), date_time64_type.getScale()); @@ -253,7 +253,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64) { - const auto & from_type = src.get(); + const auto & from_type = src.safeGet(); const auto & to_type = static_cast(type); const auto scale_from = from_type.getScale(); @@ -318,7 +318,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_from_type.isFixedString() && assert_cast(from_type_hint)->getN() == IPV6_BINARY_LENGTH) { const auto col = type.createColumn(); - ReadBufferFromString in_buffer(src.get()); + ReadBufferFromString in_buffer(src.safeGet()); type.getDefaultSerialization()->deserializeBinary(*col, in_buffer, {}); return (*col)[0]; } @@ -330,7 +330,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isFixedString()) { size_t n = assert_cast(type).getN(); - const auto & src_str = src.get(); + const auto & src_str = src.safeGet(); if (src_str.size() < n) { String src_str_extended = src_str; @@ -347,7 +347,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { if (src.getType() == Field::Types::Array) { - const Array & src_arr = src.get(); + const Array & src_arr = src.safeGet(); size_t src_arr_size = src_arr.size(); const auto & element_type = *(type_array->getNestedType()); @@ -370,7 +370,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { if (src.getType() == Field::Types::Tuple) { - const auto & src_tuple = src.get(); + const auto & src_tuple = src.safeGet(); size_t src_tuple_size = src_tuple.size(); size_t dst_tuple_size = type_tuple->getElements().size(); @@ -415,7 +415,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID const auto & key_type = *type_map->getKeyType(); const auto & value_type = *type_map->getValueType(); - const auto & map = src.get(); + const auto & map = src.safeGet(); size_t map_size = map.size(); Map res(map_size); @@ -424,7 +424,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID for (size_t i = 0; i < map_size; ++i) { - const auto & map_entry = map[i].get(); + const auto & map_entry = map[i].safeGet(); const auto & key = map_entry[0]; const auto & value = map_entry[1]; @@ -453,7 +453,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID "Cannot convert {} to {}", src.getTypeName(), agg_func_type->getName()); - const auto & name = src.get().name; + const auto & name = src.safeGet().name; if (agg_func_type->getName() != name) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert {} to {}", name, agg_func_type->getName()); @@ -468,7 +468,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (src.getType() == Field::Types::Tuple && from_type_tuple && from_type_tuple->haveExplicitNames()) { const auto & names = from_type_tuple->getElementNames(); - const auto & tuple = src.get(); + const auto & tuple = src.safeGet(); if (names.size() != tuple.size()) throw Exception(ErrorCodes::TYPE_MISMATCH, @@ -485,10 +485,10 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (src.getType() == Field::Types::Map) { Object object; - const auto & map = src.get(); + const auto & map = src.safeGet(); for (const auto & element : map) { - const auto & map_entry = element.get(); + const auto & map_entry = element.safeGet(); const auto & key = map_entry[0]; const auto & value = map_entry[1]; @@ -496,7 +496,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert from Map with key of type {} to Object", key.getTypeName()); - object[key.get()] = value; + object[key.safeGet()] = value; } return object; @@ -537,7 +537,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } const auto col = type_to_parse->createColumn(); - ReadBufferFromString in_buffer(src.get()); + ReadBufferFromString in_buffer(src.safeGet()); try { type_to_parse->getDefaultSerialization()->deserializeWholeText(*col, in_buffer, FormatSettings{}); @@ -545,9 +545,9 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID catch (Exception & e) { if (e.code() == ErrorCodes::UNEXPECTED_DATA_AFTER_PARSED_VALUE) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string '{}' to type {}", src.get(), type.getName()); + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string '{}' to type {}", src.safeGet(), type.getName()); - e.addMessage(fmt::format("while converting '{}' to {}", src.get(), type.getName())); + e.addMessage(fmt::format("while converting '{}' to {}", src.safeGet(), type.getName())); throw; } @@ -610,7 +610,7 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t template static bool decimalEqualsFloat(Field field, Float64 float_value) { - auto decimal_field = field.get>(); + auto decimal_field = field.safeGet>(); auto decimal_to_float = DecimalUtils::convertTo(decimal_field.getValue(), decimal_field.getScale()); return decimal_to_float == float_value; } @@ -629,13 +629,13 @@ std::optional convertFieldToTypeStrict(const Field & from_value, const ID { /// Convert back to Float64 and compare if (result_value.getType() == Field::Types::Decimal32) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal64) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal128) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal256) - return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; + return decimalEqualsFloat(result_value, from_value.safeGet()) ? result_value : std::optional{}; throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown decimal type {}", result_value.getTypeName()); } diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 4bfc80af1fe..d4bb0cc2f8a 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -297,7 +297,7 @@ namespace { if (tuple_literal->value.getType() == Field::Types::Tuple) { - const auto & tuple = tuple_literal->value.get(); + const auto & tuple = tuple_literal->value.safeGet(); for (const auto & child : tuple) { const auto dnf = analyzeEquals(identifier, child, expr); @@ -792,7 +792,7 @@ std::optional evaluateExpressionOverConstantCondition(const ASTPtr & nod else if (const auto * literal = node->as()) { // Check if it's always true or false. - if (literal->value.getType() == Field::Types::UInt64 && literal->value.get() == 0) + if (literal->value.getType() == Field::Types::UInt64 && literal->value.safeGet() == 0) return {result}; else return {}; diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index 3d60723a167..ee967f45c74 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -35,11 +35,11 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel if (which == Field::Types::UInt64) { - pos = ast_literal->value.get(); + pos = ast_literal->value.safeGet(); } else if (which == Field::Types::Int64) { - auto value = ast_literal->value.get(); + auto value = ast_literal->value.safeGet(); if (value > 0) pos = value; else diff --git a/src/Interpreters/tests/gtest_comparison_graph.cpp b/src/Interpreters/tests/gtest_comparison_graph.cpp index ac24a8de368..5f93bb983c1 100644 --- a/src/Interpreters/tests/gtest_comparison_graph.cpp +++ b/src/Interpreters/tests/gtest_comparison_graph.cpp @@ -29,7 +29,7 @@ TEST(ComparisonGraph, Bounds) const auto & [lower, strict] = *res; - ASSERT_EQ(lower.get(), 3); + ASSERT_EQ(lower.safeGet(), 3); ASSERT_TRUE(strict); } @@ -39,7 +39,7 @@ TEST(ComparisonGraph, Bounds) const auto & [upper, strict] = *res; - ASSERT_EQ(upper.get(), 7); + ASSERT_EQ(upper.safeGet(), 7); ASSERT_TRUE(strict); } diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index cd9e910d45a..d42728addb7 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -522,7 +522,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (tuple_arguments_valid && lit_right) { if (isInt64OrUInt64FieldType(lit_right->value.getType()) - && lit_right->value.get() >= 0) + && lit_right->value.safeGet() >= 0) { if (frame.need_parens) settings.ostr << '('; diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 8dedc5dc95d..515f4f0cb9f 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -73,8 +73,8 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const /// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) auto type = value.getType(); - if ((type == Field::Types::Array && value.get().size() > min_elements_for_hashing) - || (type == Field::Types::Tuple && value.get().size() > min_elements_for_hashing)) + if ((type == Field::Types::Array && value.safeGet().size() > min_elements_for_hashing) + || (type == Field::Types::Tuple && value.safeGet().size() > min_elements_for_hashing)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); @@ -92,7 +92,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const /// for tons of literals as it creates temporary String. if (value.getType() == Field::Types::String) { - writeQuoted(value.get(), ostr); + writeQuoted(value.safeGet(), ostr); } else { @@ -110,7 +110,7 @@ void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const /// Special case for very large arrays. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) auto type = value.getType(); - if ((type == Field::Types::Array && value.get().size() > min_elements_for_hashing)) + if ((type == Field::Types::Array && value.safeGet().size() > min_elements_for_hashing)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index ddfdbe38903..ddf4e9ecda5 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -114,7 +114,7 @@ namespace T fieldToNumber(const Field & f) { if (f.getType() == Field::Types::String) - return parseWithSizeSuffix(boost::algorithm::trim_copy(f.get())); + return parseWithSizeSuffix(boost::algorithm::trim_copy(f.safeGet())); else return applyVisitor(FieldVisitorConvertToNumber(), f); } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 9927acdcf17..de395d120d7 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -314,7 +314,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ASTPtr ast_uuid; if (!uuid_p.parse(pos, ast_uuid, expected)) return false; - uuid = parseFromString(ast_uuid->as()->value.get()); + uuid = parseFromString(ast_uuid->as()->value.safeGet()); } if (parts.size() == 1) node = std::make_shared(parts[0], std::move(params)); @@ -1626,7 +1626,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (!parser_string_literal.parse(pos, ast_prefix_name, expected)) return false; - column_name_prefix = ast_prefix_name->as().value.get(); + column_name_prefix = ast_prefix_name->as().value.safeGet(); } if (with_open_round_bracket) @@ -1689,7 +1689,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e auto res = std::make_shared(); if (regexp_node) - res->setPattern(regexp_node->as().value.get()); + res->setPattern(regexp_node->as().value.safeGet()); else res->children = std::move(identifiers); res->is_strict = is_strict; @@ -1861,7 +1861,7 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected else { auto regexp_matcher = std::make_shared(); - regexp_matcher->setPattern(regexp_node->as().value.get()); + regexp_matcher->setPattern(regexp_node->as().value.safeGet()); if (!transformers->children.empty()) { @@ -2310,7 +2310,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; - destination_name = ast_space_name->as().value.get(); + destination_name = ast_space_name->as().value.safeGet(); } else if (mode == TTLMode::GROUP_BY) { diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index dbefb0cb966..73fd563faf6 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -517,7 +517,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; - command->move_destination_name = ast_space_name->as().value.get(); + command->move_destination_name = ast_space_name->as().value.safeGet(); } else if (s_move_partition.ignore(pos, expected)) { @@ -545,7 +545,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_space_name, expected)) return false; - command->move_destination_name = ast_space_name->as().value.get(); + command->move_destination_name = ast_space_name->as().value.safeGet(); } } else if (s_add_constraint.ignore(pos, expected)) @@ -638,7 +638,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_from, expected)) return false; - command->from = ast_from->as().value.get(); + command->from = ast_from->as().value.safeGet(); command->type = ASTAlterCommand::FETCH_PARTITION; } else if (s_fetch_part.ignore(pos, expected)) @@ -652,7 +652,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ASTPtr ast_from; if (!parser_string_literal.parse(pos, ast_from, expected)) return false; - command->from = ast_from->as().value.get(); + command->from = ast_from->as().value.safeGet(); command->part = true; command->type = ASTAlterCommand::FETCH_PARTITION; } @@ -680,7 +680,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_with_name, expected)) return false; - command->with_name = ast_with_name->as().value.get(); + command->with_name = ast_with_name->as().value.safeGet(); } } else if (s_unfreeze.ignore(pos, expected)) @@ -707,7 +707,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_string_literal.parse(pos, ast_with_name, expected)) return false; - command->with_name = ast_with_name->as().value.get(); + command->with_name = ast_with_name->as().value.safeGet(); } else { diff --git a/src/Parsers/ParserCheckQuery.cpp b/src/Parsers/ParserCheckQuery.cpp index 42716ba7f2c..33b6a5a1ac2 100644 --- a/src/Parsers/ParserCheckQuery.cpp +++ b/src/Parsers/ParserCheckQuery.cpp @@ -55,7 +55,7 @@ bool ParserCheckQuery::parseCheckTable(Pos & pos, ASTPtr & node, Expected & expe const auto * ast_literal = ast_part_name->as(); if (!ast_literal || ast_literal->value.getType() != Field::Types::String) return false; - query->part_name = ast_literal->value.get(); + query->part_name = ast_literal->value.safeGet(); } if (query->database) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index a592975613b..3621a695272 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -922,7 +922,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->is_create_empty = is_create_empty; if (from_path) - query->attach_from_path = from_path->as().value.get(); + query->attach_from_path = from_path->as().value.safeGet(); return true; } @@ -1431,7 +1431,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ASTPtr ast_uuid; if (!uuid_p.parse(pos, ast_uuid, expected)) return false; - uuid = parseFromString(ast_uuid->as()->value.get()); + uuid = parseFromString(ast_uuid->as()->value.safeGet()); } if (s_on.ignore(pos, expected)) diff --git a/src/Parsers/ParserDictionary.cpp b/src/Parsers/ParserDictionary.cpp index 83a006231d9..ce38d1b54d1 100644 --- a/src/Parsers/ParserDictionary.cpp +++ b/src/Parsers/ParserDictionary.cpp @@ -33,7 +33,7 @@ bool ParserDictionaryLifetime::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (literal.value.getType() != Field::Types::UInt64) return false; - res->max_sec = literal.value.get(); + res->max_sec = literal.value.safeGet(); node = res; return true; } @@ -58,10 +58,10 @@ bool ParserDictionaryLifetime::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; if (pair.first == "min") - res->min_sec = literal->value.get(); + res->min_sec = literal->value.safeGet(); else if (pair.first == "max") { - res->max_sec = literal->value.get(); + res->max_sec = literal->value.safeGet(); initialized_max = true; } else diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 80a28f4803e..ab97b3d0e3b 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -65,7 +65,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (literal_ast->value.getType() == Field::Types::Tuple) { - fields_count = literal_ast->value.get().size(); + fields_count = literal_ast->value.safeGet().size(); } else { diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 0545c3e5568..81b64ab47c6 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -445,7 +445,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ASTPtr ast; if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; - String time_str = ast->as().value.get(); + String time_str = ast->as().value.safeGet(); ReadBufferFromString buf(time_str); time_t time; readDateTimeText(time, buf); @@ -467,7 +467,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & return false; } - res->seconds = seconds->as()->value.get(); + res->seconds = seconds->as()->value.safeGet(); break; } case Type::DROP_FILESYSTEM_CACHE: @@ -538,7 +538,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ASTPtr ast; if (ParserKeyword{Keyword::WITH_NAME}.ignore(pos, expected) && ParserStringLiteral{}.parse(pos, ast, expected)) { - res->backup_name = ast->as().value.get(); + res->backup_name = ast->as().value.safeGet(); } else { @@ -577,7 +577,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; - custom_name = ast->as().value.get(); + custom_name = ast->as().value.safeGet(); } return true; diff --git a/src/Parsers/ParserUndropQuery.cpp b/src/Parsers/ParserUndropQuery.cpp index 07ca8a3b5fd..57da47df70d 100644 --- a/src/Parsers/ParserUndropQuery.cpp +++ b/src/Parsers/ParserUndropQuery.cpp @@ -41,7 +41,7 @@ bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected) ASTPtr ast_uuid; if (!uuid_p.parse(pos, ast_uuid, expected)) return false; - uuid = parseFromString(ast_uuid->as()->value.get()); + uuid = parseFromString(ast_uuid->as()->value.safeGet()); } if (ParserKeyword{Keyword::ON}.ignore(pos, expected)) { diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index a1ba46125a7..af3591750a1 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -56,21 +56,21 @@ TEST(ParserDictionaryDDL, SimpleDictionary) EXPECT_EQ(create->dictionary->source->name, "clickhouse"); auto children = create->dictionary->source->elements->children; EXPECT_EQ(children[0]->as() -> first, "host"); - EXPECT_EQ(children[0]->as()->second->as()->value.get(), "localhost"); + EXPECT_EQ(children[0]->as()->second->as()->value.safeGet(), "localhost"); EXPECT_EQ(children[1]->as()->first, "port"); - EXPECT_EQ(children[1]->as()->second->as()->value.get(), 9000); + EXPECT_EQ(children[1]->as()->second->as()->value.safeGet(), 9000); EXPECT_EQ(children[2]->as()->first, "user"); - EXPECT_EQ(children[2]->as()->second->as()->value.get(), "default"); + EXPECT_EQ(children[2]->as()->second->as()->value.safeGet(), "default"); EXPECT_EQ(children[3]->as()->first, "password"); - EXPECT_EQ(children[3]->as()->second->as()->value.get(), ""); + EXPECT_EQ(children[3]->as()->second->as()->value.safeGet(), ""); EXPECT_EQ(children[4]->as()->first, "db"); - EXPECT_EQ(children[4]->as()->second->as()->value.get(), "test"); + EXPECT_EQ(children[4]->as()->second->as()->value.safeGet(), "test"); EXPECT_EQ(children[5]->as()->first, "table"); - EXPECT_EQ(children[5]->as()->second->as()->value.get(), "table_for_dict"); + EXPECT_EQ(children[5]->as()->second->as()->value.safeGet(), "table_for_dict"); /// layout test auto * layout = create->dictionary->layout; @@ -102,9 +102,9 @@ TEST(ParserDictionaryDDL, SimpleDictionary) EXPECT_EQ(attributes_children[1]->as()->name, "second_column"); EXPECT_EQ(attributes_children[2]->as()->name, "third_column"); - EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.get(), 0); - EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.get(), 1); - EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.get(), 2); + EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.safeGet(), 0); + EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.safeGet(), 1); + EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.safeGet(), 2); EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); @@ -150,8 +150,8 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties) EXPECT_EQ(attributes_children[2]->as()->name, "third_column"); EXPECT_EQ(attributes_children[0]->as()->default_value, nullptr); - EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.get(), 1); - EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.get(), 2); + EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.safeGet(), 1); + EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.safeGet(), 2); EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); @@ -195,9 +195,9 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder) EXPECT_EQ(attributes_children[1]->as()->name, "second_column"); EXPECT_EQ(attributes_children[2]->as()->name, "third_column"); - EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.get(), 100); - EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.get(), 1); - EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.get(), 2); + EXPECT_EQ(attributes_children[0]->as()->default_value->as()->value.safeGet(), 100); + EXPECT_EQ(attributes_children[1]->as()->default_value->as()->value.safeGet(), 1); + EXPECT_EQ(attributes_children[2]->as()->default_value->as()->value.safeGet(), 2); EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); @@ -248,25 +248,25 @@ TEST(ParserDictionaryDDL, NestedSource) auto children = create->dictionary->source->elements->children; EXPECT_EQ(children[0]->as()->first, "host"); - EXPECT_EQ(children[0]->as()->second->as()->value.get(), "localhost"); + EXPECT_EQ(children[0]->as()->second->as()->value.safeGet(), "localhost"); EXPECT_EQ(children[1]->as()->first, "port"); - EXPECT_EQ(children[1]->as()->second->as()->value.get(), 9000); + EXPECT_EQ(children[1]->as()->second->as()->value.safeGet(), 9000); EXPECT_EQ(children[2]->as()->first, "user"); - EXPECT_EQ(children[2]->as()->second->as()->value.get(), "default"); + EXPECT_EQ(children[2]->as()->second->as()->value.safeGet(), "default"); EXPECT_EQ(children[3]->as()->first, "replica"); auto replica = children[3]->as()->second->children; EXPECT_EQ(replica[0]->as()->first, "host"); - EXPECT_EQ(replica[0]->as()->second->as()->value.get(), "127.0.0.1"); + EXPECT_EQ(replica[0]->as()->second->as()->value.safeGet(), "127.0.0.1"); EXPECT_EQ(replica[1]->as()->first, "priority"); - EXPECT_EQ(replica[1]->as()->second->as()->value.get(), 1); + EXPECT_EQ(replica[1]->as()->second->as()->value.safeGet(), 1); EXPECT_EQ(children[4]->as()->first, "password"); - EXPECT_EQ(children[4]->as()->second->as()->value.get(), ""); + EXPECT_EQ(children[4]->as()->second->as()->value.safeGet(), ""); } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 58bf4c1a2fc..30301b242db 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -185,7 +185,7 @@ namespace DB } else { - auto value = static_cast(column[value_i].get>().getValue()); + auto value = static_cast(column[value_i].safeGet>().getValue()); if (need_rescale) { if (common::mulOverflow(value, rescale_multiplier, value)) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 06e8668cd7c..566a036d79c 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -208,20 +208,20 @@ private: /// Do not replace empty array and array of NULLs if (literal->value.getType() == Field::Types::Array) { - const Array & array = literal->value.get(); + const Array & array = literal->value.safeGet(); auto not_null = std::find_if_not(array.begin(), array.end(), [](const auto & elem) { return elem.isNull(); }); if (not_null == array.end()) return true; } else if (literal->value.getType() == Field::Types::Map) { - const Map & map = literal->value.get(); + const Map & map = literal->value.safeGet(); if (map.size() % 2) return false; } else if (literal->value.getType() == Field::Types::Tuple) { - const Tuple & tuple = literal->value.get(); + const Tuple & tuple = literal->value.safeGet(); for (const auto & value : tuple) if (value.isNull()) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 649721f28bf..58bec8120f1 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -262,7 +262,7 @@ convertFieldToORCLiteral(const orc::Type & orc_type, const Field & field, DataTy { case orc::BOOLEAN: { /// May throw exception - auto val = field.get(); + auto val = field.safeGet(); return orc::Literal(val != 0); } case orc::BYTE: @@ -275,7 +275,7 @@ convertFieldToORCLiteral(const orc::Type & orc_type, const Field & field, DataTy /// SELECT * FROM file('t.orc', ORC, 'x UInt8') WHERE x > 10 /// We have to reject this, otherwise it would miss values > 127 (because /// they're treated as negative by ORC). - auto val = field.get(); + auto val = field.safeGet(); return orc::Literal(val); } case orc::FLOAT: diff --git a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp index 3578401a0f8..b43c195f201 100644 --- a/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrometheusTextOutputFormat.cpp @@ -286,10 +286,10 @@ static void columnMapToContainer(const ColumnMap * col_map, size_t row_num, Cont { Field field; col_map->get(row_num, field); - const auto & map_field = field.get(); + const auto & map_field = field.safeGet(); for (const auto & map_element : map_field) { - const auto & map_entry = map_element.get(); + const auto & map_entry = map_element.safeGet(); String entry_key; String entry_value; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index de34a8aa04f..9839f64b947 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -333,7 +333,7 @@ namespace { const DataTypeTuple & type_tuple = static_cast(data_type); - Tuple & tuple_value = value.get(); + Tuple & tuple_value = value.safeGet(); size_t src_tuple_size = tuple_value.size(); size_t dst_tuple_size = type_tuple.getElements().size(); @@ -360,7 +360,7 @@ namespace if (element_type.isNullable()) return; - Array & array_value = value.get(); + Array & array_value = value.safeGet(); size_t array_value_size = array_value.size(); for (size_t i = 0; i < array_value_size; ++i) @@ -378,12 +378,12 @@ namespace const auto & key_type = *type_map.getKeyType(); const auto & value_type = *type_map.getValueType(); - auto & map = value.get(); + auto & map = value.safeGet(); size_t map_size = map.size(); for (size_t i = 0; i < map_size; ++i) { - auto & map_entry = map[i].get(); + auto & map_entry = map[i].safeGet(); auto & entry_key = map_entry[0]; auto & entry_value = map_entry[1]; diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index e2c6371c44f..80c00f91d82 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -127,14 +127,14 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row right(left.size()); for (size_t col_num : desc.key_col_nums) - right[col_num] = (*raw_columns[col_num])[row_number].template get(); + right[col_num] = (*raw_columns[col_num])[row_number].template safeGet(); for (size_t col_num : desc.val_col_nums) - right[col_num] = (*raw_columns[col_num])[row_number].template get(); + right[col_num] = (*raw_columns[col_num])[row_number].template safeGet(); auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field & { - return matrix[i].get()[j]; + return matrix[i].safeGet()[j]; }; auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array @@ -160,7 +160,7 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, auto merge = [&](const Row & matrix) { - size_t rows = matrix[desc.key_col_nums[0]].get().size(); + size_t rows = matrix[desc.key_col_nums[0]].safeGet().size(); for (size_t j = 0; j < rows; ++j) { @@ -190,10 +190,10 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, for (const auto & key_value : merged) { for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.key_col_nums[col_num_index]].get()[row_num] = key_value.first[col_num_index]; + row[desc.key_col_nums[col_num_index]].safeGet()[row_num] = key_value.first[col_num_index]; for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index) - row[desc.val_col_nums[col_num_index]].get()[row_num] = key_value.second[col_num_index]; + row[desc.val_col_nums[col_num_index]].safeGet()[row_num] = key_value.second[col_num_index]; ++row_num; } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 2080e29ceba..596d08845e1 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -119,23 +119,23 @@ using RangesWithStep = std::vector; std::optional steppedRangeFromRange(const Range & r, UInt64 step, UInt64 remainder) { - if ((r.right.get() == 0) && (!r.right_included)) + if ((r.right.safeGet() == 0) && (!r.right_included)) return std::nullopt; - UInt64 begin = (r.left.get() / step) * step; + UInt64 begin = (r.left.safeGet() / step) * step; if (begin > std::numeric_limits::max() - remainder) return std::nullopt; begin += remainder; - while ((r.left_included <= r.left.get()) && (begin <= r.left.get() - r.left_included)) + while ((r.left_included <= r.left.safeGet()) && (begin <= r.left.safeGet() - r.left_included)) { if (std::numeric_limits::max() - step < begin) return std::nullopt; begin += step; } - if ((begin >= r.right_included) && (begin - r.right_included >= r.right.get())) + if ((begin >= r.right_included) && (begin - r.right_included >= r.right.safeGet())) return std::nullopt; - UInt64 right_edge_included = r.right.get() - (1 - r.right_included); + UInt64 right_edge_included = r.right.safeGet() - (1 - r.right_included); return std::optional{RangeWithStep{begin, step, static_cast(right_edge_included - begin) / step + 1}}; } diff --git a/src/Processors/Sources/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp index 5d533a7747e..52be9a6e84a 100644 --- a/src/Processors/Sources/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -219,11 +219,11 @@ namespace read_bytes_size += 8; break; case ValueType::vtEnum8: - assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).get()); + assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).safeGet()); read_bytes_size += assert_cast(column).byteSize(); break; case ValueType::vtEnum16: - assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).get()); + assert_cast(column).insertValue(assert_cast &>(data_type).castToValue(value.data()).safeGet()); read_bytes_size += assert_cast(column).byteSize(); break; case ValueType::vtString: diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 9601f821cc8..95f4a674ebb 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -62,7 +62,7 @@ static FillColumnDescription::StepFunction getStepFunction( case IntervalKind::Kind::NAME: \ return [step, scale, &date_lut](Field & field) { \ field = Add##NAME##sImpl::execute(static_cast(\ - field.get()), static_cast(step), date_lut, utc_time_zone, scale); }; + field.safeGet()), static_cast(step), date_lut, utc_time_zone, scale); }; FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE @@ -139,21 +139,21 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & { if (which.isDate() || which.isDate32()) { - Int64 avg_seconds = descr.fill_step.get() * descr.step_kind->toAvgSeconds(); + Int64 avg_seconds = descr.fill_step.safeGet() * descr.step_kind->toAvgSeconds(); if (std::abs(avg_seconds) < 86400) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds)); } if (which.isDate()) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), DateLUT::instance()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), DateLUT::instance()); else if (which.isDate32()) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), DateLUT::instance()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), DateLUT::instance()); else if (const auto * date_time = checkAndGetDataType(type.get())) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.get(), date_time->getTimeZone()); + descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), date_time->getTimeZone()); else if (const auto * date_time64 = checkAndGetDataType(type.get())) { - const auto & step_dec = descr.fill_step.get &>(); + const auto & step_dec = descr.fill_step.safeGet &>(); Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); @@ -163,7 +163,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & case IntervalKind::Kind::NAME: \ descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ { \ - auto field_decimal = field.get>(); \ + auto field_decimal = field.safeGet>(); \ auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, utc_time_zone, field_decimal.getScale()); \ field = DecimalField(res, field_decimal.getScale()); \ }; \ diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index f76e2d64368..32066adad0d 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -107,7 +107,7 @@ static int compareValuesWithOffset(const IColumn * _compared_column, using ValueType = typename ColumnType::ValueType; // Note that the storage type of offset returned by get<> is different, so // we need to specify the type explicitly. - const ValueType offset = static_cast(_offset.get()); + const ValueType offset = static_cast(_offset.safeGet()); assert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); @@ -162,7 +162,7 @@ static int compareValuesWithOffsetFloat(const IColumn * _compared_column, _compared_column); const auto * reference_column = assert_cast( _reference_column); - const auto offset = _offset.get(); + const auto offset = _offset.safeGet(); chassert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); @@ -631,7 +631,7 @@ void WindowTransform::advanceFrameStartRowsOffset() { // Just recalculate it each time by walking blocks. const auto [moved_row, offset_left] = moveRowNumber(current_row, - window_description.frame.begin_offset.get() + window_description.frame.begin_offset.safeGet() * (window_description.frame.begin_preceding ? -1 : 1)); frame_start = moved_row; @@ -870,7 +870,7 @@ void WindowTransform::advanceFrameEndRowsOffset() // Walk the specified offset from the current row. The "+1" is needed // because the frame_end is a past-the-end pointer. const auto [moved_row, offset_left] = moveRowNumber(current_row, - window_description.frame.end_offset.get() + window_description.frame.end_offset.safeGet() * (window_description.frame.end_preceding ? -1 : 1) + 1); @@ -2192,13 +2192,13 @@ namespace throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant"); auto type_id = argument_types[0]->getTypeId(); if (type_id == TypeIndex::UInt8) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); else if (type_id == TypeIndex::UInt16) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); else if (type_id == TypeIndex::UInt32) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); else if (type_id == TypeIndex::UInt64) - buckets = arg_col[transform->current_row.row].get(); + buckets = arg_col[transform->current_row.row].safeGet(); if (!buckets) { @@ -2490,7 +2490,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction { offset = (*current_block.input_columns[ workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); + transform->current_row.row].safeGet(); /// Either overflow or really negative value, both is not acceptable. if (offset < 0) @@ -2576,7 +2576,7 @@ struct WindowFunctionNthValue final : public WindowFunction Int64 offset = (*current_block.input_columns[ workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); + transform->current_row.row].safeGet(); /// Either overflow or really negative value, both is not acceptable. if (offset <= 0) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7891042bb96..4869819d0d6 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -109,7 +109,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.comment) { const auto & ast_comment = typeid_cast(*ast_col_decl.comment); - command.comment = ast_comment.value.get(); + command.comment = ast_comment.value.safeGet(); } if (ast_col_decl.codec) @@ -167,7 +167,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.comment) { const auto & ast_comment = ast_col_decl.comment->as(); - command.comment.emplace(ast_comment.value.get()); + command.comment.emplace(ast_comment.value.safeGet()); } if (ast_col_decl.ttl) @@ -210,7 +210,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.type = COMMENT_COLUMN; command.column_name = getIdentifierName(command_ast->column); const auto & ast_comment = command_ast->comment->as(); - command.comment = ast_comment.value.get(); + command.comment = ast_comment.value.safeGet(); command.if_exists = command_ast->if_exists; return command; } @@ -220,7 +220,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.ast = command_ast->clone(); command.type = COMMENT_TABLE; const auto & ast_comment = command_ast->comment->as(); - command.comment = ast_comment.value.get(); + command.comment = ast_comment.value.safeGet(); return command; } else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index da749812167..0d724245b49 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -197,7 +197,7 @@ void ColumnDescription::readText(ReadBuffer & buf) } if (col_ast->comment) - comment = col_ast->comment->as().value.get(); + comment = col_ast->comment->as().value.safeGet(); if (col_ast->codec) codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true, true, true); diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 7354243732c..d6a8af3238e 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -268,7 +268,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Float64) { out.function = RPNElement::FUNCTION_FLOAT_LITERAL; - out.float_literal.emplace(const_value.get()); + out.float_literal.emplace(const_value.safeGet()); out.func_name = "Float literal"; return true; } @@ -276,7 +276,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::UInt64) { out.function = RPNElement::FUNCTION_INT_LITERAL; - out.int_literal.emplace(const_value.get()); + out.int_literal.emplace(const_value.safeGet()); out.func_name = "Int literal"; return true; } @@ -284,7 +284,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Int64) { out.function = RPNElement::FUNCTION_INT_LITERAL; - out.int_literal.emplace(const_value.get()); + out.int_literal.emplace(const_value.safeGet()); out.func_name = "Int literal"; return true; } @@ -292,7 +292,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Tuple) { out.function = RPNElement::FUNCTION_LITERAL_TUPLE; - out.tuple_literal = const_value.get(); + out.tuple_literal = const_value.safeGet(); out.func_name = "Tuple literal"; return true; } @@ -300,7 +300,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::Array) { out.function = RPNElement::FUNCTION_LITERAL_ARRAY; - out.array_literal = const_value.get(); + out.array_literal = const_value.safeGet(); out.func_name = "Array literal"; return true; } @@ -308,7 +308,7 @@ bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node if (const_value.getType() == Field::Types::String) { out.function = RPNElement::FUNCTION_STRING_LITERAL; - out.func_name = const_value.get(); + out.func_name = const_value.safeGet(); return true; } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3a44359b537..0db05373e43 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -428,7 +428,7 @@ std::pair IMergeTreeDataPart::getMinMaxDate() const if (storage.minmax_idx_date_column_pos != -1 && minmax_idx->initialized) { const auto & hyperrectangle = minmax_idx->hyperrectangle[storage.minmax_idx_date_column_pos]; - return {DayNum(hyperrectangle.left.get()), DayNum(hyperrectangle.right.get())}; + return {DayNum(hyperrectangle.left.safeGet()), DayNum(hyperrectangle.right.safeGet())}; } else return {}; @@ -444,15 +444,15 @@ std::pair IMergeTreeDataPart::getMinMaxTime() const if (hyperrectangle.left.getType() == Field::Types::UInt64) { assert(hyperrectangle.right.getType() == Field::Types::UInt64); - return {hyperrectangle.left.get(), hyperrectangle.right.get()}; + return {hyperrectangle.left.safeGet(), hyperrectangle.right.safeGet()}; } /// The case of DateTime64 else if (hyperrectangle.left.getType() == Field::Types::Decimal64) { assert(hyperrectangle.right.getType() == Field::Types::Decimal64); - auto left = hyperrectangle.left.get>(); - auto right = hyperrectangle.right.get>(); + auto left = hyperrectangle.left.safeGet>(); + auto right = hyperrectangle.right.safeGet>(); assert(left.getScale() == right.getScale()); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 69bffac9160..a717c08f0a2 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -349,7 +349,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ false); + String prefix = extractFixedPrefixFromLikePattern(value.safeGet(), /*requires_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -370,7 +370,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ true); + String prefix = extractFixedPrefixFromLikePattern(value.safeGet(), /*requires_perfect_prefix*/ true); if (prefix.empty()) return false; @@ -391,7 +391,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = value.get(); + String prefix = value.safeGet(); if (prefix.empty()) return false; @@ -412,7 +412,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - const String & expression = value.get(); + const String & expression = value.safeGet(); /// This optimization can't process alternation - this would require /// a comprehensive parsing of regular expression. @@ -2918,8 +2918,8 @@ BoolMask KeyCondition::checkInHyperrectangle( /// Let's support only the case of 2d, because I'm not confident in other cases. if (num_dimensions == 2) { - UInt64 left = key_range.left.get(); - UInt64 right = key_range.right.get(); + UInt64 left = key_range.left.safeGet(); + UInt64 right = key_range.right.safeGet(); BoolMask mask(false, true); auto hyperrectangle_intersection_callback = [&](std::array, 2> curve_hyperrectangle) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e10f5a0227..830f6f5d52f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5869,7 +5869,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (partition_lit && partition_lit->value.getType() == Field::Types::String) { MergeTreePartInfo::validatePartitionID(partition_ast.value->clone(), format_version); - return partition_lit->value.get(); + return partition_lit->value.safeGet(); } } @@ -5932,7 +5932,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", partition_key_value.getTypeName()); - const Tuple & tuple = partition_key_value.get(); + const Tuple & tuple = partition_key_value.safeGet(); if (tuple.size() != fields_count) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of fields in the partition expression: {}, must be: {}", tuple.size(), fields_count); @@ -6868,7 +6868,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( auto * place = arena.alignedAlloc(size_of_state, align_of_state); func->create(place); if (const AggregateFunctionCount * agg_count = typeid_cast(func.get())) - AggregateFunctionCount::set(place, value.get()); + AggregateFunctionCount::set(place, value.safeGet()); else { auto value_column = func->getArgumentTypes().front()->createColumnConst(1, value)->convertToFullColumnIfConst(); @@ -7510,7 +7510,7 @@ MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & parti if (const auto * partition_lit = partition_ast->as().value->as()) { id = partition_lit->value.getType() == Field::Types::UInt64 - ? toString(partition_lit->value.get()) + ? toString(partition_lit->value.safeGet()) : partition_lit->value.safeGet(); prefixed = true; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index ee3ac4207cc..98f1563546e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -441,8 +441,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( String part_name; if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.get()); - DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.get()); + DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.safeGet()); + DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.safeGet()); const auto & date_lut = DateLUT::serverTimezoneInstance(); diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 497e86334f3..b68e48eeb3a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -343,12 +343,12 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; String distance_function = DEFAULT_DISTANCE_FUNCTION; if (!index.arguments.empty()) - distance_function = index.arguments[0].get(); + distance_function = index.arguments[0].safeGet(); static constexpr auto DEFAULT_TREES = 100uz; UInt64 trees = DEFAULT_TREES; if (index.arguments.size() > 1) - trees = index.arguments[1].get(); + trees = index.arguments[1].safeGet(); return std::make_shared(index, trees, distance_function); } @@ -375,7 +375,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { - String distance_name = index.arguments[0].get(); + String distance_name = index.arguments[0].safeGet(); if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index c6a00751f25..be0ee693e15 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -348,19 +348,19 @@ bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTre { if (const_value.getType() == Field::Types::UInt64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Int64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Float64) { - out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } @@ -692,7 +692,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( const bool is_nullable = actual_type->isNullable(); auto mutable_column = actual_type->createColumn(); - for (const auto & f : value_field.get()) + for (const auto & f : value_field.safeGet()) { if ((f.isNull() && !is_nullable) || f.isDecimal(f.getType())) /// NOLINT(readability-static-accessed-through-instance) return false; @@ -763,7 +763,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( if (which.isTuple() && key_node_function_name == "tuple") { - const Tuple & tuple = value_field.get(); + const Tuple & tuple = value_field.safeGet(); const auto * value_tuple_data_type = typeid_cast(value_type.get()); if (tuple.size() != key_node_function_arguments_size) @@ -952,7 +952,7 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach) { const auto & argument = index.arguments[0]; - if (!attach && (argument.getType() != Field::Types::Float64 || argument.get() < 0 || argument.get() > 1)) + if (!attach && (argument.getType() != Field::Types::Float64 || argument.safeGet() < 0 || argument.safeGet() > 1)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The BloomFilter false positive must be a double number between 0 and 1."); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp index 5b6813d12e3..857b7903588 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilterText.cpp @@ -341,19 +341,19 @@ bool MergeTreeConditionBloomFilterText::extractAtomFromTree(const RPNBuilderTree if (const_value.getType() == Field::Types::UInt64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Int64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Float64) { - out.function = const_value.get() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } @@ -493,7 +493,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - auto value = const_value.get(); + auto value = const_value.safeGet(); if (is_case_insensitive_scenario) std::ranges::transform(value, value.begin(), [](const auto & c) { return static_cast(std::tolower(c)); }); @@ -509,7 +509,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_HAS; out.bloom_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -519,7 +519,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_HAS; out.bloom_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -529,7 +529,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -538,7 +538,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -547,7 +547,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -556,7 +556,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToBloomFilter(value.data(), value.size(), *out.bloom_filter); return true; } @@ -565,7 +565,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToBloomFilter(value.data(), value.size(), *out.bloom_filter, true, false); return true; } @@ -574,7 +574,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.key_column = *key_index; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToBloomFilter(value.data(), value.size(), *out.bloom_filter, false, true); return true; } @@ -589,13 +589,13 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( /// 2d vector is not needed here but is used because already exists for FUNCTION_IN std::vector> bloom_filters; bloom_filters.emplace_back(); - for (const auto & element : const_value.get()) + for (const auto & element : const_value.safeGet()) { if (element.getType() != Field::Types::String) return false; bloom_filters.back().emplace_back(params); - const auto & value = element.get(); + const auto & value = element.safeGet(); if (function_name == "multiSearchAny") { @@ -615,7 +615,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals( out.function = RPNElement::FUNCTION_MATCH; out.bloom_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); String required_substring; bool dummy_is_trivial, dummy_required_substring_is_prefix; std::vector alternatives; @@ -743,11 +743,11 @@ MergeTreeIndexPtr bloomFilterIndexTextCreator( { if (index.type == NgramTokenExtractor::getName()) { - size_t n = index.arguments[0].get(); + size_t n = index.arguments[0].safeGet(); BloomFilterParameters params( - index.arguments[1].get(), - index.arguments[2].get(), - index.arguments[3].get()); + index.arguments[1].safeGet(), + index.arguments[2].safeGet(), + index.arguments[3].safeGet()); auto tokenizer = std::make_unique(n); @@ -756,9 +756,9 @@ MergeTreeIndexPtr bloomFilterIndexTextCreator( else if (index.type == SplitTokenExtractor::getName()) { BloomFilterParameters params( - index.arguments[0].get(), - index.arguments[1].get(), - index.arguments[2].get()); + index.arguments[0].safeGet(), + index.arguments[1].safeGet(), + index.arguments[2].safeGet()); auto tokenizer = std::make_unique(); @@ -815,9 +815,9 @@ void bloomFilterIndexTextValidator(const IndexDescription & index, bool /*attach /// Just validate BloomFilterParameters params( - index.arguments[0].get(), - index.arguments[1].get(), - index.arguments[2].get()); + index.arguments[0].safeGet(), + index.arguments[1].safeGet(), + index.arguments[2].safeGet()); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index cd6af68ebcc..b5c6bb95d37 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -74,7 +74,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr for (auto & gin_filter : gin_filters) { size_serialization->deserializeBinary(field_rows, istr, {}); - size_t filter_size = field_rows.get(); + size_t filter_size = field_rows.safeGet(); gin_filter.getFilter().resize(filter_size); if (filter_size == 0) @@ -379,19 +379,19 @@ bool MergeTreeConditionFullText::traverseAtomAST(const RPNBuilderTreeNode & node /// Check constant like in KeyCondition if (const_value.getType() == Field::Types::UInt64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Int64) { - out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } if (const_value.getType() == Field::Types::Float64) { - out.function = const_value.get() != 0.00 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.safeGet() != 0.00 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } @@ -530,7 +530,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_HAS; out.gin_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -539,7 +539,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_HAS; out.gin_filter = std::make_unique(params); - auto & value = const_value.get(); + auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -549,7 +549,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -558,7 +558,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -567,7 +567,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -576,7 +576,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -585,7 +585,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter); return true; } @@ -594,7 +594,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToGinFilter(value.data(), value.size(), *out.gin_filter, true, false); return true; } @@ -603,7 +603,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.gin_filter = std::make_unique(params); - const auto & value = const_value.get(); + const auto & value = const_value.safeGet(); token_extractor->substringToGinFilter(value.data(), value.size(), *out.gin_filter, false, true); return true; } @@ -615,13 +615,13 @@ bool MergeTreeConditionFullText::traverseASTEquals( /// 2d vector is not needed here but is used because already exists for FUNCTION_IN std::vector gin_filters; gin_filters.emplace_back(); - for (const auto & element : const_value.get()) + for (const auto & element : const_value.safeGet()) { if (element.getType() != Field::Types::String) return false; gin_filters.back().emplace_back(params); - const auto & value = element.get(); + const auto & value = element.safeGet(); token_extractor->substringToGinFilter(value.data(), value.size(), gin_filters.back().back(), false, false); } out.set_gin_filters = std::move(gin_filters); @@ -632,7 +632,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_MATCH; - auto & value = const_value.get(); + auto & value = const_value.safeGet(); String required_substring; bool dummy_is_trivial, dummy_required_substring_is_prefix; std::vector alternatives; @@ -776,8 +776,8 @@ MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition( MergeTreeIndexPtr fullTextIndexCreator( const IndexDescription & index) { - size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); + size_t n = index.arguments.empty() ? 0 : index.arguments[0].safeGet(); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].safeGet(); GinFilterParameters params(n, max_rows); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor @@ -826,12 +826,12 @@ void fullTextIndexValidator(const IndexDescription & index, bool /*attach*/) { if (index.arguments[1].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INCORRECT_QUERY, "The second full text index argument must be UInt64"); - if (index.arguments[1].get() != UNLIMITED_ROWS_PER_POSTINGS_LIST && index.arguments[1].get() < MIN_ROWS_PER_POSTINGS_LIST) + if (index.arguments[1].safeGet() != UNLIMITED_ROWS_PER_POSTINGS_LIST && index.arguments[1].safeGet() < MIN_ROWS_PER_POSTINGS_LIST) throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows per postings list must be no less than {}", MIN_ROWS_PER_POSTINGS_LIST); } /// Just validate - size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows_per_postings_list = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); + size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].safeGet(); + UInt64 max_rows_per_postings_list = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].safeGet(); GinFilterParameters params(ngrams, max_rows_per_postings_list); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index cd8065ecadf..abf3ae56376 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -37,7 +37,7 @@ void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr, Merge Field field_met; const auto & size_type = DataTypePtr(std::make_shared()); size_type->getDefaultSerialization()->deserializeBinary(field_met, istr, {}); - met = field_met.get(); + met = field_met.safeGet(); is_empty = false; } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index a92df4ac72d..fa242fccbc1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -97,7 +97,7 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd Field field_rows; const auto & size_type = DataTypePtr(std::make_shared()); size_type->getDefaultSerialization()->deserializeBinary(field_rows, istr, {}); - size_t rows_to_read = field_rows.get(); + size_t rows_to_read = field_rows.safeGet(); if (rows_to_read == 0) return; @@ -591,7 +591,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition( MergeTreeIndexPtr setIndexCreator(const IndexDescription & index) { - size_t max_rows = index.arguments[0].get(); + size_t max_rows = index.arguments[0].safeGet(); return std::make_shared(index, max_rows); } diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index 59a4b0fbf9c..efd9bb754e1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -377,12 +377,12 @@ MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) static constexpr auto default_distance_function = DISTANCE_FUNCTION_L2; String distance_function = default_distance_function; if (!index.arguments.empty()) - distance_function = index.arguments[0].get(); + distance_function = index.arguments[0].safeGet(); static constexpr auto default_scalar_kind = unum::usearch::scalar_kind_t::f16_k; auto scalar_kind = default_scalar_kind; if (index.arguments.size() > 1) - scalar_kind = nameToScalarKind.at(index.arguments[1].get()); + scalar_kind = nameToScalarKind.at(index.arguments[1].safeGet()); return std::make_shared(index, distance_function, scalar_kind); } @@ -408,14 +408,14 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { - String distance_name = index.arguments[0].get(); + String distance_name = index.arguments[0].safeGet(); if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) throw Exception(ErrorCodes::INCORRECT_DATA, "USearch index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } /// Check that a supported kind was passed as a second argument - if (index.arguments.size() > 1 && !nameToScalarKind.contains(index.arguments[1].get())) + if (index.arguments.size() > 1 && !nameToScalarKind.contains(index.arguments[1].safeGet())) { String supported_kinds; for (const auto & [name, kind] : nameToScalarKind) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index b240f80ee13..5b5bc244f92 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -241,7 +241,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) result += toString(DateLUT::serverTimezoneInstance().toNumYYYYMMDD(DayNum(value[i].safeGet()))); else if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) - result += toString(value[i].get().toUnderType()); + result += toString(value[i].safeGet().toUnderType()); else result += applyVisitor(to_string_visitor, value[i]); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 43c40dee77d..f0c26c302e1 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -222,17 +222,17 @@ static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet /// check the value with respect to threshold if (type == Field::Types::UInt64) { - const auto value = output_value.get(); + const auto value = output_value.safeGet(); return value > threshold; } else if (type == Field::Types::Int64) { - const auto value = output_value.get(); + const auto value = output_value.safeGet(); return value < -threshold || threshold < value; } else if (type == Field::Types::Float64) { - const auto value = output_value.get(); + const auto value = output_value.safeGet(); return value < -threshold || threshold < value; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 3f0603f6900..52857845e99 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -591,7 +591,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (ast->value.getType() != Field::Types::String) throw Exception(ErrorCodes::BAD_ARGUMENTS, format_str, error_msg); - graphite_config_name = ast->value.get(); + graphite_config_name = ast->value.safeGet(); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, format_str, error_msg); diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index ade3326288a..fd9f29b6375 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -32,19 +32,19 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) switch (field.getType()) { case Field::Types::Int64: - return field.get(); + return field.safeGet(); case Field::Types::UInt64: - return field.get(); + return field.safeGet(); case Field::Types::Float64: - return field.get(); + return field.safeGet(); case Field::Types::Int128: - return field.get(); + return field.safeGet(); case Field::Types::UInt128: - return field.get(); + return field.safeGet(); case Field::Types::Int256: - return field.get(); + return field.safeGet(); case Field::Types::UInt256: - return field.get(); + return field.safeGet(); default: return {}; } @@ -53,7 +53,7 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) { if (field.getType() == Field::Types::String) - return field.get(); + return field.safeGet(); return {}; } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index e69bbc1515b..a6f2f883e65 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -48,7 +48,7 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); if (isStringOrFixedString(data_type)) - return sketch.get_estimate(val.get()); + return sketch.get_estimate(val.safeGet()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); } diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 060b271d8f4..b95ccedb093 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -203,7 +203,7 @@ StoragePtr StorageFactory::get( } if (query.comment) - comment = query.comment->as().value.get(); + comment = query.comment->as().value.safeGet(); ASTs empty_engine_args; Arguments arguments{ diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 98cd5c4dfa9..20b02b59a10 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2203,11 +2203,11 @@ void registerStorageFile(StorageFactory & factory) { auto type = literal->value.getType(); if (type == Field::Types::Int64) - source_fd = static_cast(literal->value.get()); + source_fd = static_cast(literal->value.safeGet()); else if (type == Field::Types::UInt64) - source_fd = static_cast(literal->value.get()); + source_fd = static_cast(literal->value.safeGet()); else if (type == Field::Types::String) - StorageFile::parseFileSource(literal->value.get(), source_path, storage_args.path_to_archive); + StorageFile::parseFileSource(literal->value.safeGet(), source_path, storage_args.path_to_archive); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument must be path or file descriptor"); } diff --git a/src/Storages/StorageFuzzJSON.cpp b/src/Storages/StorageFuzzJSON.cpp index 9950d41f1c2..fc73f246d35 100644 --- a/src/Storages/StorageFuzzJSON.cpp +++ b/src/Storages/StorageFuzzJSON.cpp @@ -419,7 +419,7 @@ void fuzzJSONObject( if (val.fixed->getType() == Field::Types::Which::String) { out << fuzzJSONStructure(config, rnd, "\""); - writeText(val.fixed->get(), out); + writeText(val.fixed->safeGet(), out); out << fuzzJSONStructure(config, rnd, "\""); } else diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a0d6cf11b64..136df425813 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -341,10 +341,10 @@ void registerStorageJoin(StorageFactory & factory) else if (setting.name == "any_join_distinct_right_table_keys") old_any_join = setting.value; else if (setting.name == "disk") - disk_name = setting.value.get(); + disk_name = setting.value.safeGet(); else if (setting.name == "persistent") { - persistent = setting.value.get(); + persistent = setting.value.safeGet(); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown setting {} for storage {}", setting.name, args.engine_name); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index d3214e7ed13..e0a4af68824 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -294,7 +294,7 @@ public: { const auto * array_type = typeid_cast(data_type.get()); const auto & nested = array_type->getNestedType(); - const auto & array = array_field.get(); + const auto & array = array_field.safeGet(); if (!isArray(nested)) { @@ -312,7 +312,7 @@ public: if (!isArray(nested_array_type->getNestedType())) { - parseArrayContent(iter->get(), nested, ostr); + parseArrayContent(iter->safeGet(), nested, ostr); } else { diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6a7810b97f9..522884b4fef 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -105,8 +105,8 @@ protected: while (rows_count < max_block_size && db_table_num < total_tables) { - const std::string database_name = (*databases)[db_table_num].get(); - const std::string table_name = (*tables)[db_table_num].get(); + const std::string database_name = (*databases)[db_table_num].safeGet(); + const std::string table_name = (*tables)[db_table_num].safeGet(); ++db_table_num; ColumnsDescription columns; @@ -426,7 +426,7 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline, for (size_t i = 0; i < num_databases; ++i) { - const std::string database_name = (*database_column)[i].get(); + const std::string database_name = (*database_column)[i].safeGet(); if (database_name.empty()) { for (auto & [table_name, table] : external_tables) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 7ace8ee24aa..c87bdb6d26a 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -138,7 +138,7 @@ StoragesInfoStream::StoragesInfoStream(std::optional filter_by_datab for (size_t i = 0; i < rows; ++i) { - String database_name = (*database_column_for_filter)[i].get(); + String database_name = (*database_column_for_filter)[i].safeGet(); const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 806af4a7bf8..3be73aeda17 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -52,13 +52,13 @@ public: { StoragesInfo info; - info.database = (*database_column)[next_row].get(); - info.table = (*table_column)[next_row].get(); - UUID storage_uuid = (*storage_uuid_column)[next_row].get(); + info.database = (*database_column)[next_row].safeGet(); + info.table = (*table_column)[next_row].safeGet(); + UUID storage_uuid = (*storage_uuid_column)[next_row].safeGet(); auto is_same_table = [&storage_uuid, this] (size_t row) -> bool { - return (*storage_uuid_column)[row].get() == storage_uuid; + return (*storage_uuid_column)[row].safeGet() == storage_uuid; }; /// We may have two rows per table which differ in 'active' value. @@ -66,7 +66,7 @@ public: /// must collect the inactive parts. Remember this fact in StoragesInfo. for (; next_row < rows && is_same_table(next_row); ++next_row) { - const auto active = (*active_column)[next_row].get(); + const auto active = (*active_column)[next_row].safeGet(); if (active == 0) info.need_inactive_parts = true; } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 73b7908b75c..ea837da1e73 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -51,7 +51,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) const ColumnWithTypeAndName & data = block.getByName(name); size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) - res.insert((*data.column)[i].get()); + res.insert((*data.column)[i].safeGet()); return res; } diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 9d23f132759..1408e120bc5 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -102,16 +102,16 @@ ColumnsDescription getStructureOfRemoteTableInShard( { ColumnDescription column; - column.name = (*name)[i].get(); + column.name = (*name)[i].safeGet(); - String data_type_name = (*type)[i].get(); + String data_type_name = (*type)[i].safeGet(); column.type = data_type_factory.get(data_type_name); - String kind_name = (*default_kind)[i].get(); + String kind_name = (*default_kind)[i].safeGet(); if (!kind_name.empty()) { column.default_desc.kind = columnDefaultKindFromString(kind_name); - String expr_str = (*default_expr)[i].get(); + String expr_str = (*default_expr)[i].safeGet(); column.default_desc.expression = parseQuery( expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0, settings.max_parser_depth, settings.max_parser_backtracks); @@ -207,8 +207,8 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( size_t size = name_col.size(); for (size_t i = 0; i < size; ++i) { - auto name = name_col[i].get(); - auto type_name = type_col[i].get(); + auto name = name_col[i].safeGet(); + auto type_name = type_col[i].safeGet(); auto storage_column = storage_columns.tryGetPhysical(name); if (storage_column && storage_column->type->hasDynamicSubcolumnsDeprecated()) diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index 552b9fde986..69d24c879bd 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -83,7 +83,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt "Table function '{}' requires a String argument for EXPLAIN kind, got '{}'", getName(), queryToString(kind_arg)); - ASTExplainQuery::ExplainKind kind = ASTExplainQuery::fromString(kind_literal->value.get()); + ASTExplainQuery::ExplainKind kind = ASTExplainQuery::fromString(kind_literal->value.safeGet()); auto explain_query = std::make_shared(kind); const auto * settings_arg = function->arguments->children[1]->as(); @@ -92,7 +92,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt "Table function '{}' requires a serialized string settings argument, got '{}'", getName(), queryToString(function->arguments->children[1])); - const auto & settings_str = settings_arg->value.get(); + const auto & settings_str = settings_arg->value.safeGet(); if (!settings_str.empty()) { const Settings & settings = context->getSettingsRef(); diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 1b6d86f8fa5..e59ee52fd82 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -47,7 +47,7 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { fd = static_cast( - (type == Field::Types::Int64) ? literal->value.get() : literal->value.get()); + (type == Field::Types::Int64) ? literal->value.safeGet() : literal->value.safeGet()); if (fd < 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "File descriptor must be non-negative"); } diff --git a/src/TableFunctions/TableFunctionMergeTreeIndex.cpp b/src/TableFunctions/TableFunctionMergeTreeIndex.cpp index 06a48f0e25f..27ed50fb711 100644 --- a/src/TableFunctions/TableFunctionMergeTreeIndex.cpp +++ b/src/TableFunctions/TableFunctionMergeTreeIndex.cpp @@ -76,9 +76,9 @@ void TableFunctionMergeTreeIndex::parseArguments(const ASTPtr & ast_function, Co "Table function '{}' expected bool flag for 'with_marks' argument", getName()); if (value.getType() == Field::Types::Bool) - with_marks = value.get(); + with_marks = value.safeGet(); else - with_marks = value.get(); + with_marks = value.safeGet(); } if (!params.empty()) From 18d9bb2ade4e98051df007663a387eb74146c26f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 16:25:32 +0200 Subject: [PATCH 0727/1722] tests: attempt to fix 01600_parts_states_metrics_long (by forbid parallel run) CI: https://s3.amazonaws.com/clickhouse-test-reports/68134/8d4f822fee64d44440459b733c67dee5e9fb1e02/stateless_tests__tsan__s3_storage__[2_4].html Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/01600_parts_states_metrics_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 47b5a4dea13..8062bb0ba5d 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From eeda67042c08bedbd18c3b7a76cb8928e9975348 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 16:28:49 +0200 Subject: [PATCH 0728/1722] tests: make 01600_parts_states_metrics_long faster Signed-off-by: Azat Khuzhin --- .../01600_parts_states_metrics_long.sh | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 8062bb0ba5d..a07dd306b3e 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -5,6 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +function query() +{ + # NOTE: database_atomic_wait_for_drop_and_detach_synchronously needed only for local env, CI has it ON + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database_atomic_wait_for_drop_and_detach_synchronously=1" -d "$*" +} + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT (SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) @@ -18,13 +24,13 @@ verify() { for i in {1..5000} do - result=$( $CLICKHOUSE_CLIENT --query="$verify_sql" ) + result=$( query "$verify_sql" ) [ "$result" = "1" ] && echo "$result" && break sleep 0.1 if [[ $i -eq 5000 ]] then - $CLICKHOUSE_CLIENT " + query " SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; SELECT sum(active), sum(NOT active) FROM system.parts; SELECT sum(active), sum(NOT active) FROM system.projection_parts; @@ -34,17 +40,17 @@ verify() done } -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS test_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table (data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" +query "DROP TABLE IF EXISTS test_table" +query "CREATE TABLE test_table (data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" +query "INSERT INTO test_table VALUES ('1992-01-01')" verify -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" +query "INSERT INTO test_table VALUES ('1992-01-02')" verify -$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" +query "OPTIMIZE TABLE test_table FINAL" verify -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE test_table" +query "DROP TABLE test_table" verify From 0a8fb05ece2771439844c03456d43b02eb8f51cd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 16:23:23 +0000 Subject: [PATCH 0729/1722] fix after merge --- src/Core/Field.h | 7 +++++++ src/Disks/DiskFomAST.cpp | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 689ac38a235..13741183f21 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -869,6 +869,13 @@ constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t) || t == Field::Types::UInt64; } +constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t) +{ + return t == Field::Types::Int64 + || t == Field::Types::UInt64 + || t == Field::Types::Bool; +} + template auto & Field::safeGet() { diff --git a/src/Disks/DiskFomAST.cpp b/src/Disks/DiskFomAST.cpp index b2f1280c507..5329ff8748a 100644 --- a/src/Disks/DiskFomAST.cpp +++ b/src/Disks/DiskFomAST.cpp @@ -132,7 +132,7 @@ std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, Conte FlattenDiskConfigurationVisitor::Data data{context, attach}; FlattenDiskConfigurationVisitor{data}.visit(ast); - return assert_cast(*ast).value.get(); + return assert_cast(*ast).value.safeGet(); } void DiskFomAST::ensureDiskIsNotCustom(const std::string & disk_name, ContextPtr context) From 0a536cbf150916bff0ec63c6ef3b77df517868f8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 10 Aug 2024 18:58:28 +0200 Subject: [PATCH 0730/1722] Add batch size --- docker/test/stateless/run.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index a030be92506..17e39487e3e 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -193,8 +193,8 @@ ORDER BY tuple()" # create minio log webhooks for both audit and server logs # use async inserts to avoid creating too many parts -./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 -./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 +./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 +./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 max_retries=100 retry=1 @@ -376,6 +376,8 @@ done # collect minio audit and server logs +# wait for minio to flush its batch if it has any +sleep 1 clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE" clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" From 3f718626da61f0502115586425d728492d3a3ae3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Aug 2024 19:55:11 +0200 Subject: [PATCH 0731/1722] Better test --- tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh b/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh index fb55539d04a..021278955cd 100755 --- a/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh +++ b/tests/queries/0_stateless/03221_s3_imds_decent_timeout.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-asan, no-msan, no-tsan # ^ requires S3 CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -10,7 +10,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) while true do # This host (likely) drops packets sent to it (does not reply), so it is good for testing timeouts. - # At the same time, we expect that google.com does not drop packets and quickly replies with 404, which is a non-retriable error for S3. - AWS_EC2_METADATA_SERVICE_ENDPOINT='https://10.255.255.255/' ${CLICKHOUSE_LOCAL} --time --query "SELECT * FROM s3('https://google.com/test')" |& grep -v -F 404 | + # At the same time, we expect that the clickhouse host does not drop packets and quickly replies with 4xx, which is a non-retriable error for S3. + AWS_EC2_METADATA_SERVICE_ENDPOINT='https://10.255.255.255/' ${CLICKHOUSE_LOCAL} --time --query "SELECT * FROM s3('${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/nonexistent')" |& grep -v -F 404 | ${CLICKHOUSE_LOCAL} --input-format TSV "SELECT c1::Float64 < 1 FROM table" | grep 1 && break done From 7524b8f76712ce421fdebd1fe86c79128fea3ceb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Aug 2024 19:55:22 +0200 Subject: [PATCH 0732/1722] A slight improvement --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 78dbb72c199..f7701a2aab8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -895,7 +895,7 @@ void StorageMergeTree::loadDeduplicationLog() std::string path = fs::path(relative_data_path) / "deduplication_logs"; /// If either there is already a deduplication log, or we will be able to use it. - if (disk->exists(path) || !disk->isReadOnly()) + if (!disk->isReadOnly() || disk->exists(path)) { deduplication_log = std::make_unique(path, settings->non_replicated_deduplication_window, format_version, disk); deduplication_log->load(); From d53513a81a10b8230a30fdbb386aca1d067cbcfa Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sat, 10 Aug 2024 18:12:34 +0000 Subject: [PATCH 0733/1722] fix --- tests/queries/0_stateless/01710_projection_vertical_merges.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.sql b/tests/queries/0_stateless/01710_projection_vertical_merges.sql index 0f80d659e92..0d745e44b10 100644 --- a/tests/queries/0_stateless/01710_projection_vertical_merges.sql +++ b/tests/queries/0_stateless/01710_projection_vertical_merges.sql @@ -1,4 +1,5 @@ --- Tags: long +-- Tags: long, no-parallel +-- set no-parallel tag is to prevent timeout of this test drop table if exists t; From 80e926996319b926f3cbebf2050ed4a60666ee71 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 19:39:59 +0000 Subject: [PATCH 0734/1722] allow UInt64 <-> Int64 conversion --- src/Core/Field.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 13741183f21..ba8c66580ad 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -882,8 +882,11 @@ auto & Field::safeGet() const Types::Which target = TypeToEnum>>::value; /// bool is stored as uint64, will be returned as UInt64 when requested as bool or UInt64, as Int64 when requested as Int64 - if (target != which && !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64))) - throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); + /// also allow UInt64 <-> Int64 conversion + if (target != which && + !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64)) && + !(isInt64OrUInt64FieldType(which) && isInt64OrUInt64FieldType(target))) + throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); return get(); } From 556f66987897bd5065426e187bef4e0cba2a975c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 10 Aug 2024 21:52:55 +0000 Subject: [PATCH 0735/1722] Separate test into separate file to pass bugfix check --- tests/integration/parallel_skip.json | 4 + tests/integration/test_storage_kafka/test.py | 135 ---------- .../test_produce_http_interface.py | 243 ++++++++++++++++++ 3 files changed, 247 insertions(+), 135 deletions(-) create mode 100644 tests/integration/test_storage_kafka/test_produce_http_interface.py diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 99fa626bd1e..fca2126d824 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -162,9 +162,13 @@ "test_storage_kafka/test.py::test_system_kafka_consumers_rebalance_mv", "test_storage_kafka/test.py::test_formats_errors", "test_storage_kafka/test.py::test_multiple_read_in_materialized_views", + "test_storage_kafka/test.py::test_kafka_null_message", + + "test_storage_kafka/test_produce_http_interface.py::test_kafka_produce_http_interface_row_based_format", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_request_new_ticket_after_expiration", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", "test_storage_kerberized_kafka/test.py::test_kafka_config_from_sql_named_collection" + ] diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9b2f465c1b6..4b6c9922d74 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5524,141 +5524,6 @@ def test_kafka_null_message(kafka_cluster, create_query_generator): ) -def test_kafka_produce_http_interface_row_based_format(kafka_cluster): - # reproduction of #61060 with validating the written messages - admin_client = KafkaAdminClient( - bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) - ) - - topic_prefix = "http_row_" - - # It is important to have: - # - long enough messages - # - enough messages - # I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages - # For the number of messages it seems like at least 3 messages is necessary - expected_key = "01234567890123456789" - expected_value = "aaaaabbbbbccccc" - - insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')" - insert_query_template = "INSERT INTO {table_name} " + insert_query_end - - extra_settings = { - "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", - "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", - "Template": ", format_template_row='string_key_value.format'", - } - - # Only the formats that can be used both and input and output format are tested - # Reasons to exclude following formats: - # - JSONStrings: not actually an input format - # - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken, - # because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed. - # - ProtobufList: I didn't want to deal with the envelope and stuff - # - Npy: supports only single column - # - LineAsString: supports only single column - # - RawBLOB: supports only single column - formats_to_test = [ - "TabSeparated", - "TabSeparatedRaw", - "TabSeparatedWithNames", - "TabSeparatedWithNamesAndTypes", - "TabSeparatedRawWithNames", - "TabSeparatedRawWithNamesAndTypes", - "Template", - "CSV", - "CSVWithNames", - "CSVWithNamesAndTypes", - "CustomSeparated", - "CustomSeparatedWithNames", - "CustomSeparatedWithNamesAndTypes", - "Values", - "JSON", - "JSONColumns", - "JSONColumnsWithMetadata", - "JSONCompact", - "JSONCompactColumns", - "JSONEachRow", - "JSONStringsEachRow", - "JSONCompactEachRow", - "JSONCompactEachRowWithNames", - "JSONCompactEachRowWithNamesAndTypes", - "JSONCompactStringsEachRow", - "JSONCompactStringsEachRowWithNames", - "JSONCompactStringsEachRowWithNamesAndTypes", - "JSONObjectEachRow", - "BSONEachRow", - "TSKV", - "Protobuf", - "Avro", - "Parquet", - "Arrow", - "ArrowStream", - "ORC", - "RowBinary", - "RowBinaryWithNames", - "RowBinaryWithNamesAndTypes", - "Native", - "CapnProto", - "MsgPack", - ] - for format in formats_to_test: - logging.debug(f"Creating tables and writing messages to {format}") - topic = topic_prefix + format - kafka_create_topic(admin_client, topic) - - extra_setting = extra_settings.get(format, "") - - # kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug - instance.query( - f""" - DROP TABLE IF EXISTS test.view_{topic}; - DROP TABLE IF EXISTS test.consumer_{topic}; - CREATE TABLE test.kafka_writer_{topic} (key String, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = '{format}', - kafka_max_rows_per_message = 2 {extra_setting}; - - CREATE TABLE test.kafka_{topic} (key String, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = '{topic}', - kafka_group_name = '{topic}', - kafka_format = '{format}' {extra_setting}; - - CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS - SELECT key, value FROM test.kafka_{topic}; - """ - ) - - instance.http_query( - insert_query_template.format(table_name="test.kafka_writer_" + topic), - method="POST", - ) - - expected = f"""\ -{expected_key}\t{expected_value} -{expected_key}\t{expected_value} -{expected_key}\t{expected_value} -""" - # give some times for the readers to read the messages - for format in formats_to_test: - logging.debug(f"Checking result for {format}") - topic = topic_prefix + format - - result = instance.query_with_retry( - f"SELECT * FROM test.view_{topic}", - check_callback=lambda res: res.count("\n") == 3, - ) - - assert TSV(result) == TSV(expected) - - kafka_delete_topic(admin_client, topic) - - if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_kafka/test_produce_http_interface.py b/tests/integration/test_storage_kafka/test_produce_http_interface.py new file mode 100644 index 00000000000..fc10a07f239 --- /dev/null +++ b/tests/integration/test_storage_kafka/test_produce_http_interface.py @@ -0,0 +1,243 @@ +import time +import logging + +import pytest +from helpers.cluster import ClickHouseCluster, is_arm +from helpers.test_tools import TSV +from kafka import KafkaAdminClient +from kafka.admin import NewTopic + +if is_arm(): + pytestmark = pytest.mark.skip + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=["configs/kafka.xml", "configs/named_collection.xml"], + user_configs=["configs/users.xml"], + with_kafka=True, + with_zookeeper=True, # For Replicated Table + macros={ + "kafka_broker": "kafka1", + "kafka_topic_old": "old", + "kafka_group_name_old": "old", + "kafka_topic_new": "new", + "kafka_group_name_new": "new", + "kafka_client_id": "instance", + "kafka_format_json_each_row": "JSONEachRow", + }, + clickhouse_path_dir="clickhouse_path", +) + + +@pytest.fixture(scope="module") +def kafka_cluster(): + try: + cluster.start() + kafka_id = instance.cluster.kafka_docker_id + print(("kafka_id is {}".format(kafka_id))) + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def kafka_setup_teardown(): + instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + # logging.debug("kafka is available - running test") + yield # run test + + +def kafka_create_topic( + admin_client, + topic_name, + num_partitions=1, + replication_factor=1, + max_retries=50, + config=None, +): + logging.debug( + f"Kafka create topic={topic_name}, num_partitions={num_partitions}, replication_factor={replication_factor}" + ) + topics_list = [ + NewTopic( + name=topic_name, + num_partitions=num_partitions, + replication_factor=replication_factor, + topic_configs=config, + ) + ] + retries = 0 + while True: + try: + admin_client.create_topics(new_topics=topics_list, validate_only=False) + logging.debug("Admin client succeed") + return + except Exception as e: + retries += 1 + time.sleep(0.5) + if retries < max_retries: + logging.warning(f"Failed to create topic {e}") + else: + raise + + +def kafka_delete_topic(admin_client, topic, max_retries=50): + result = admin_client.delete_topics([topic]) + for topic, e in result.topic_error_codes: + if e == 0: + logging.debug(f"Topic {topic} deleted") + else: + logging.error(f"Failed to delete topic {topic}: {e}") + + retries = 0 + while True: + topics_listed = admin_client.list_topics() + logging.debug(f"TOPICS LISTED: {topics_listed}") + if topic not in topics_listed: + return + else: + retries += 1 + time.sleep(0.5) + if retries > max_retries: + raise Exception(f"Failed to delete topics {topic}, {result}") + + +def test_kafka_produce_http_interface_row_based_format(kafka_cluster): + # reproduction of #61060 with validating the written messages + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic_prefix = "http_row_" + + # It is important to have: + # - long enough messages + # - enough messages + # I don't know the exact requirement for message sizes, but it doesn't reproduce with short messages + # For the number of messages it seems like at least 3 messages is necessary + expected_key = "01234567890123456789" + expected_value = "aaaaabbbbbccccc" + + insert_query_end = f"(key, value) VALUES ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}'), ('{expected_key}', '{expected_value}')" + insert_query_template = "INSERT INTO {table_name} " + insert_query_end + + extra_settings = { + "Protobuf": ", kafka_schema = 'string_key_value.proto:StringKeyValuePair'", + "CapnProto": ", kafka_schema='string_key_value:StringKeyValuePair'", + "Template": ", format_template_row='string_key_value.format'", + } + + # Only the formats that can be used both and input and output format are tested + # Reasons to exclude following formats: + # - JSONStrings: not actually an input format + # - ProtobufSingle: I cannot make it work to parse the messages. Probably something is broken, + # because the producer can write multiple rows into a same message, which makes them impossible to parse properly. Should added after #67549 is fixed. + # - ProtobufList: I didn't want to deal with the envelope and stuff + # - Npy: supports only single column + # - LineAsString: supports only single column + # - RawBLOB: supports only single column + formats_to_test = [ + "TabSeparated", + "TabSeparatedRaw", + "TabSeparatedWithNames", + "TabSeparatedWithNamesAndTypes", + "TabSeparatedRawWithNames", + "TabSeparatedRawWithNamesAndTypes", + "Template", + "CSV", + "CSVWithNames", + "CSVWithNamesAndTypes", + "CustomSeparated", + "CustomSeparatedWithNames", + "CustomSeparatedWithNamesAndTypes", + "Values", + "JSON", + "JSONColumns", + "JSONColumnsWithMetadata", + "JSONCompact", + "JSONCompactColumns", + "JSONEachRow", + "JSONStringsEachRow", + "JSONCompactEachRow", + "JSONCompactEachRowWithNames", + "JSONCompactEachRowWithNamesAndTypes", + "JSONCompactStringsEachRow", + "JSONCompactStringsEachRowWithNames", + "JSONCompactStringsEachRowWithNamesAndTypes", + "JSONObjectEachRow", + "BSONEachRow", + "TSKV", + "Protobuf", + "Avro", + "Parquet", + "Arrow", + "ArrowStream", + "ORC", + "RowBinary", + "RowBinaryWithNames", + "RowBinaryWithNamesAndTypes", + "Native", + "CapnProto", + "MsgPack", + ] + for format in formats_to_test: + logging.debug(f"Creating tables and writing messages to {format}") + topic = topic_prefix + format + kafka_create_topic(admin_client, topic) + + extra_setting = extra_settings.get(format, "") + + # kafka_max_rows_per_message is set to 2 to make sure every format produces at least 2 messages, thus increasing the chance of catching a bug + instance.query( + f""" + DROP TABLE IF EXISTS test.view_{topic}; + DROP TABLE IF EXISTS test.consumer_{topic}; + CREATE TABLE test.kafka_writer_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}', + kafka_max_rows_per_message = 2 {extra_setting}; + + CREATE TABLE test.kafka_{topic} (key String, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_format = '{format}' {extra_setting}; + + CREATE MATERIALIZED VIEW test.view_{topic} Engine=Log AS + SELECT key, value FROM test.kafka_{topic}; + """ + ) + instance.http_query( + insert_query_template.format(table_name="test.kafka_writer_" + topic), + method="POST", + ) + + expected = f"""\ +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +{expected_key}\t{expected_value} +""" + # give some times for the readers to read the messages + for format in formats_to_test: + logging.debug(f"Checking result for {format}") + topic = topic_prefix + format + + result = instance.query_with_retry( + f"SELECT * FROM test.view_{topic}", + check_callback=lambda res: res.count("\n") == 3, + ) + + assert TSV(result) == TSV(expected) + + kafka_delete_topic(admin_client, topic) + + +if __name__ == "__main__": + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() From 613ebe367c1f811eea38d7c5e778cedddbfb0ce7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 10 Aug 2024 22:05:11 +0000 Subject: [PATCH 0736/1722] Only add extra cell when necessary --- tests/ci/report.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 15b1512896a..6779a6dae96 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -738,7 +738,7 @@ def create_test_html_report( if test_results: rows_part = [] num_fails = 0 - has_test_time = False + has_test_time = any(tr.time is not None for tr in test_results) has_log_urls = False # Display entires with logs at the top (they correspond to failed tests) @@ -770,12 +770,12 @@ def create_test_html_report( row.append(f'{test_result.status}') colspan += 1 - row.append("") - if test_result.time is not None: - has_test_time = True - row.append(str(test_result.time)) - row.append("") - colspan += 1 + if has_test_time: + if test_result.time is not None: + row.append(f"{test_result.time}") + else: + row.append("") + colspan += 1 if test_result.log_urls is not None: has_log_urls = True From e93584e741fab888977413c800df4595220e7552 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 10 Aug 2024 23:02:30 +0000 Subject: [PATCH 0737/1722] fix Field conversion to IPv4 --- src/Interpreters/convertFieldToType.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 3489852bbd5..738c51baa64 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -304,8 +304,8 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } if (which_type.isIPv4() && src.getType() == Field::Types::UInt64) { - /// convert to UInt32 which is the underlying type for native IPv4 - return convertNumericType(src, type); + /// convert through UInt32 which is the underlying type for native IPv4 + return static_cast(convertNumericType(src, type).safeGet()); } } else if (which_type.isUUID() && src.getType() == Field::Types::UUID) From ee3eec0a2a15592b2020d34b74d9f595e707c092 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Aug 2024 04:47:26 +0200 Subject: [PATCH 0738/1722] Update Credentials.cpp --- src/IO/S3/Credentials.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 9c5f6547933..d6f7542da6b 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -785,6 +785,8 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( /// EC2MetadataService throttles by delaying the response so the service client should set a large read timeout. /// EC2MetadataService delay is in order of seconds so it only make sense to retry after a couple of seconds. + /// But the connection timeout should be small because there is the case when there is no IMDS at all, + /// like outside of the cloud, on your own machines. aws_client_configuration.connectTimeoutMs = 10; aws_client_configuration.requestTimeoutMs = 1000; From 1142305b113e261d0c8910c0b622ba94727fe78d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 09:53:43 +0200 Subject: [PATCH 0739/1722] tests: fix 01246_buffer_flush flakiness due to slow trace_log flush Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/01246_buffer_flush.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01246_buffer_flush.sh b/tests/queries/0_stateless/01246_buffer_flush.sh index aea91a0bf6b..3c7b9038e1f 100755 --- a/tests/queries/0_stateless/01246_buffer_flush.sh +++ b/tests/queries/0_stateless/01246_buffer_flush.sh @@ -27,7 +27,7 @@ function wait_until() function get_buffer_delay() { local buffer_insert_id=$1 && shift - query "SYSTEM FLUSH LOGS" + $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" query " WITH (SELECT event_time_microseconds FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND type = 'QueryStart' AND query_id = '$buffer_insert_id') AS begin_, From 375de7ff6523a23fb7b898725a53004f24d047cd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 07:15:10 +0200 Subject: [PATCH 0740/1722] ci: add more logs in the functional tests reports Due to settings randomization 4096 is not enough even to show all settings, like here [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/68139/c852bd9dbaa317423234d4f15f21d64e59be42b5/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin --- docker/test/util/process_functional_tests_result.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index aa2ea686c46..ec9e14b1430 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -116,7 +116,7 @@ def process_test_log(log_path, broken_tests): test[0], test[1], test[2], - "".join(test[3])[:4096].replace("\t", "\\t").replace("\n", "\\n"), + "".join(test[3])[:8192].replace("\t", "\\t").replace("\n", "\\n"), ] for test in test_results ] From 45028620332d55391e900b6c7e75acb34df1d98c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Aug 2024 08:35:47 +0000 Subject: [PATCH 0741/1722] Fix no-SSE3 build --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index eb3afe0ccdf..b33e7083e32 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -179,7 +179,7 @@ else() message(STATUS "Not using QPL") endif () -if (OS_LINUX AND ARCH_AMD64) +if (OS_LINUX AND ARCH_AMD64 AND NOT NO_SSE3_OR_HIGHER) option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology Library (QATlib)" ${ENABLE_LIBRARIES}) elseif(ENABLE_QATLIB) message (${RECONFIGURE_MESSAGE_LEVEL} "QATLib is only supported on x86_64") From ece707c4436ab65fcb142f0eaae72f7eb2c3d8db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Aug 2024 19:43:29 +0200 Subject: [PATCH 0742/1722] Better test for Not-ready Set is passed in system.* tables - system.distribution_queue - system.replication_queue - system.rocksdb - system.databases - system.mutations - test for system.part_moves_between_shards will not be provided since it is a likely deprecated feature and the test requires some code (I've fixed it differently from #66018, but it does not make sense anymore, so I'm submitting only the test) --- ...3223_system_tables_set_not_ready.reference | 5 ++++ .../03223_system_tables_set_not_ready.sql | 30 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/03223_system_tables_set_not_ready.reference create mode 100644 tests/queries/0_stateless/03223_system_tables_set_not_ready.sql diff --git a/tests/queries/0_stateless/03223_system_tables_set_not_ready.reference b/tests/queries/0_stateless/03223_system_tables_set_not_ready.reference new file mode 100644 index 00000000000..e39523ed4f5 --- /dev/null +++ b/tests/queries/0_stateless/03223_system_tables_set_not_ready.reference @@ -0,0 +1,5 @@ +system.distribution_queue 1 +system.rocksdb 1 +system.databases 1 +system.mutations 1 +system.replication_queue 1 diff --git a/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql b/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql new file mode 100644 index 00000000000..907fa47143c --- /dev/null +++ b/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql @@ -0,0 +1,30 @@ +-- Tags: no-fasttest +-- Tag no-fasttest -- due to EmbeddedRocksDB + +drop table if exists null; +drop table if exists dist; +create table null as system.one engine=Null; +create table dist as null engine=Distributed(test_cluster_two_shards, currentDatabase(), 'null', rand()); +insert into dist settings prefer_localhost_replica=0 values (1); +select 'system.distribution_queue', count() from system.distribution_queue where exists(select 1) and database = currentDatabase(); + +drop table if exists rocksdb; +create table rocksdb (key Int) engine=EmbeddedRocksDB() primary key key; +insert into rocksdb values (1); +select 'system.rocksdb', count()>0 from system.rocksdb where exists(select 1) and database = currentDatabase(); + +select 'system.databases', count() from system.databases where exists(select 1) and database = currentDatabase(); + +drop table if exists mt; +create table mt (key Int) engine=MergeTree() order by key; +alter table mt delete where 1; +select 'system.mutations', count() from system.mutations where exists(select 1) and database = currentDatabase(); + +drop table if exists rep1; +drop table if exists rep2; +create table rep1 (key Int) engine=ReplicatedMergeTree('/{database}/rep', '{table}') order by key; +create table rep2 (key Int) engine=ReplicatedMergeTree('/{database}/rep', '{table}') order by key; +system stop fetches rep2; +insert into rep1 values (1); +system sync replica rep2 pull; +select 'system.replication_queue', count()>0 from system.replication_queue where exists(select 1) and database = currentDatabase(); From 53bc1b7e3539cde14cb34f26af296bde5c29449e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 11 Aug 2024 13:19:36 +0200 Subject: [PATCH 0743/1722] Revert "Refactor tests for (experimental) statistics" --- docs/en/development/tests.md | 4 +- .../statements/alter/statistics.md | 16 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- ...2864_statistics_count_min_sketch.reference | 14 ++ .../02864_statistics_count_min_sketch.sql | 70 ++++++ .../02864_statistics_ddl.reference | 37 ++- .../0_stateless/02864_statistics_ddl.sql | 234 ++++-------------- ...delayed_materialization_in_merge.reference | 12 - ...stics_delayed_materialization_in_merge.sql | 36 --- .../02864_statistics_exception.reference | 0 .../02864_statistics_exception.sql | 55 ++++ ..._statistics_materialize_in_merge.reference | 10 + .../02864_statistics_materialize_in_merge.sql | 52 ++++ .../02864_statistics_predicates.reference | 98 -------- .../02864_statistics_predicates.sql | 214 ---------------- .../02864_statistics_uniq.reference | 35 +++ .../0_stateless/02864_statistics_uniq.sql | 73 ++++++ .../02864_statistics_usage.reference | 20 -- .../0_stateless/02864_statistics_usage.sql | 42 ---- 19 files changed, 399 insertions(+), 625 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference create mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql create mode 100644 tests/queries/0_stateless/02864_statistics_exception.reference create mode 100644 tests/queries/0_stateless/02864_statistics_exception.sql create mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference create mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_predicates.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_predicates.sql create mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference create mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_usage.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_usage.sql diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 6cb36e2049b..269995a1a96 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. :::note A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs @@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes ### Adding a New Test -To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 7a1774a01b5..6880cef0e5c 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -8,28 +8,26 @@ sidebar_label: STATISTICS The following operations are available: -- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata. +- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. -- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata. +- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. -- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. +- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. -- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. +- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. -- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. -## Example: - -Adding two statistics types to two columns: +There is an example adding two statistics types to two columns: ``` ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` :::note -Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c7101021f02..625b1281c61 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3517,7 +3517,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistics is not safe " + "ALTER types of column {} with statistics is not not safe " "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference new file mode 100644 index 00000000000..02c41656a36 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference @@ -0,0 +1,14 @@ +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics count_min: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics multi-types: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) +Test LowCardinality and Nullable data type: +tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql new file mode 100644 index 00000000000..c730aa7b4a7 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql @@ -0,0 +1,70 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS tab SYNC; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_suspicious_low_cardinality_types=1; +SET mutations_sync = 2; + +CREATE TABLE tab +( + a String, + b UInt64, + c Int64, + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE tab; + +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'Test statistics count_min:'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min; +ALTER TABLE tab ADD STATISTICS c TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS a, b, c; + + +SELECT 'Test statistics multi-types:'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS a, b, c; + +DROP TABLE IF EXISTS tab SYNC; + + +SELECT 'Test LowCardinality and Nullable data type:'; +DROP TABLE IF EXISTS tab2 SYNC; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE tab2 +( + a LowCardinality(Int64) STATISTICS(count_min), + b Nullable(Int64) STATISTICS(count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(count_min), + pk String, +) Engine = MergeTree() ORDER BY pk; + +select name from system.tables where name = 'tab2' and database = currentDatabase(); + +DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_ddl.reference b/tests/queries/0_stateless/02864_statistics_ddl.reference index 0e453b0ee8a..a7ff5caa0b0 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.reference +++ b/tests/queries/0_stateless/02864_statistics_ddl.reference @@ -1,6 +1,31 @@ -CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) +10 +0 +After drop statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10), less(a, 10)) (removed) +10 +CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After add statistic +CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After materialize statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) +20 +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) +20 +CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After rename + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(c, 10)) (removed) +20 diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index 32b56a842b7..fe612efe2ac 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -1,195 +1,59 @@ --- Tags: no-fasttest --- no-fasttest: 'count_min' sketches need a 3rd party library - --- Tests that DDL statements which create / drop / materialize statistics - -SET mutations_sync = 1; +-- Tests that various DDL statements create/drop/materialize statistics DROP TABLE IF EXISTS tab; --- Error case: Can't create statistics when allow_experimental_statistics = 0 -CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - SET allow_experimental_statistics = 1; - --- Error case: Unknown statistics types are rejected -CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- Error case: The same statistics type can't exist more than once on a column -CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - -SET allow_suspicious_low_cardinality_types = 1; - --- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*) - --- tdigest requires data_type.isValueRepresentedByInteger --- These types work: -CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; --- These types don't work: -CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - --- uniq requires data_type.isValueRepresentedByInteger --- These types work: -CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; --- These types don't work: -CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - --- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String --- These types work: -CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; --- These types don't work: -CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - --- CREATE TABLE was easy, ALTER is more fun +SET allow_statistics_optimize = 1; CREATE TABLE tab ( - f64 Float64, - f64_tdigest Float64 STATISTICS(tdigest), - f32 Float32, - s String, - a Array(Float64) -) -Engine = MergeTree() -ORDER BY tuple(); + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; --- Error case: Unknown statistics types are rejected --- (relevant for ADD and MODIFY) -ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } --- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported - --- Error case: The same statistics type can't exist more than once on a column --- (relevant for ADD and MODIFY) --- Create the same statistics object twice -ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } --- Create an statistics which exists already -ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op -ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op - --- Error case: Column does not exist --- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE) --- Note that the results are unfortunately quite inconsistent ... -ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op -ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op -ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS } - --- Error case: Column exists but has no statistics --- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE) --- Note that the results are unfortunately quite inconsistent ... -ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op -ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op -ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS } - --- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the --- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that --- works and one that doesn't work. --- tdigest --- Works: -ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; -ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; --- Doesn't work: -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } --- uniq --- Works: -ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64; -ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; --- Doesn't work: -ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } --- count_min --- Works: -ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; -ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; --- Doesn't work: -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } - --- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing --- statistics objects (e.g. tdigest can be created on Float64 and UInt64) -ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - --- Finally, do a full-circle test of a good case. Print table definition after each step. --- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested. -SHOW CREATE TABLE tab; -ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq; -SHOW CREATE TABLE tab; -ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq; -SHOW CREATE TABLE tab; -ALTER TABLE tab CLEAR STATISTICS f64, f32; -SHOW CREATE TABLE tab; -ALTER TABLE tab MATERIALIZE STATISTICS f64, f32; -SHOW CREATE TABLE tab; -ALTER TABLE tab DROP STATISTICS f64, f32; SHOW CREATE TABLE tab; -DROP TABLE tab; +INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; +SELECT count(*) FROM tab WHERE b < NULL and a < '10'; + +ALTER TABLE tab DROP STATISTICS a, b; + +SELECT 'After drop statistic'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; + +SHOW CREATE TABLE tab; + +ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; + +SELECT 'After add statistic'; + +SHOW CREATE TABLE tab; + +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'After materialize statistic'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; + +OPTIMIZE TABLE tab FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE b < 10 and a < 10; + +ALTER TABLE tab RENAME COLUMN b TO c; +SHOW CREATE TABLE tab; + +SELECT 'After rename'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT count(*) FROM tab WHERE c < 10 and a < 10; + +DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference deleted file mode 100644 index eb5e685597c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference +++ /dev/null @@ -1,12 +0,0 @@ -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After truncate, insert, and materialize - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql deleted file mode 100644 index d469a4c2036..00000000000 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql +++ /dev/null @@ -1,36 +0,0 @@ --- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). --- (The concrete statistics type, column data type and predicate type don't matter) - --- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) - -DROP TABLE IF EXISTS tab; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET enable_analyzer = 1; - -SET materialize_statistics_on_insert = 0; - -CREATE TABLE tab -( - a Int64 STATISTICS(tdigest), - b Int16 STATISTICS(tdigest), -) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) - -OPTIMIZE TABLE tab FINAL; -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -TRUNCATE TABLE tab; -SET mutations_sync = 2; -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -SELECT 'After truncate, insert, and materialize'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql new file mode 100644 index 00000000000..289ffee6600 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_exception.sql @@ -0,0 +1,55 @@ +-- Tests creating/dropping/materializing statistics produces the right exceptions. + +DROP TABLE IF EXISTS tab; + +-- Can't create statistics when allow_experimental_statistics = 0 +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest) +) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +SET allow_experimental_statistics = 1; + +-- The same type of statistics can't exist more than once on a column +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest, tdigest) +) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- Unknown statistics types are rejected +CREATE TABLE tab +( + a Float64 STATISTICS(no_statistics_type) +) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- tDigest statistics can only be created on numeric columns +CREATE TABLE tab +( + a String STATISTICS(tdigest), +) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +CREATE TABLE tab +( + a Float64, + b String +) Engine = MergeTree() ORDER BY tuple(); + +ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest; +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; +-- Statistics can be created only on integer columns +ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS a; +ALTER TABLE tab DROP STATISTICS IF EXISTS a; +ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS a; +ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } + +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; +ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; +ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference new file mode 100644 index 00000000000..5e969cf41cb --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference @@ -0,0 +1,10 @@ +10 +10 +10 +statistics not used Condition less(b, 10_UInt8) moved to PREWHERE +statistics not used Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE +statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE +statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE +2 0 diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql new file mode 100644 index 00000000000..6606cff263f --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql @@ -0,0 +1,52 @@ +-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). + +DROP TABLE IF EXISTS tab; + +SET enable_analyzer = 1; +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +SET materialize_statistics_on_insert = 0; + +CREATE TABLE tab +( + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), +) ENGINE = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; + +SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; + +OPTIMIZE TABLE tab FINAL; + +SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; + +TRUNCATE TABLE tab; +SET mutations_sync = 2; + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; + +SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; + +DROP TABLE tab; + +SYSTEM FLUSH LOGS; + +SELECT log_comment, message FROM system.text_log JOIN +( + SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log + WHERE current_database = currentDatabase() + AND query LIKE 'SELECT count(*) FROM tab%' + AND type = 'QueryFinish' +) AS query_log USING (query_id) +WHERE message LIKE '%moved to PREWHERE%' +ORDER BY event_time_microseconds; + +SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) +FROM system.query_log +WHERE current_database = currentDatabase() + AND query LIKE 'INSERT INTO tab SELECT%' + AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference deleted file mode 100644 index ffbd7269e05..00000000000 --- a/tests/queries/0_stateless/02864_statistics_predicates.reference +++ /dev/null @@ -1,98 +0,0 @@ -u64 and = -10 -10 -10 -10 -0 -0 -0 -0 -10 -10 -10 -10 -u64 and < -70 -70 -70 -70 -80 -80 -80 -80 -70 -70 -70 -70 -f64 and = -10 -10 -10 -10 -0 -0 -0 -0 -10 -10 -10 -10 -0 -0 -0 -0 -f64 and < -70 -70 -70 -70 -80 -80 -80 -80 -70 -70 -70 -70 -80 -80 -80 -80 -dt and = -0 -0 -0 -0 -10 -10 -10 -10 -dt and < -10000 -10000 -10000 -10000 -70 -70 -70 -70 -b and = -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -5000 -0 -0 -0 -0 -s and = -10 -10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql deleted file mode 100644 index 779116cf19a..00000000000 --- a/tests/queries/0_stateless/02864_statistics_predicates.sql +++ /dev/null @@ -1,214 +0,0 @@ --- Tags: no-fasttest --- no-fasttest: 'count_min' sketches need a 3rd party library - --- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types. - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab -( - u64 UInt64, - u64_tdigest UInt64 STATISTICS(tdigest), - u64_count_min UInt64 STATISTICS(count_min), - u64_uniq UInt64 STATISTICS(uniq), - f64 Float64, - f64_tdigest Float64 STATISTICS(tdigest), - f64_count_min Float64 STATISTICS(count_min), - f64_uniq Float64 STATISTICS(uniq), - dt DateTime, - dt_tdigest DateTime STATISTICS(tdigest), - dt_count_min DateTime STATISTICS(count_min), - dt_uniq DateTime STATISTICS(uniq), - b Bool, - b_tdigest Bool STATISTICS(tdigest), - b_count_min Bool STATISTICS(count_min), - b_uniq Bool STATISTICS(uniq), - s String, - -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest - s_count_min String STATISTICS(count_min) - -- s_uniq String STATISTICS(uniq), -- not supported by uniq -) Engine = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO tab --- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; -SELECT number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 1000, - number % 2, - number % 2, - number % 2, - number % 2, - toString(number % 1000), - toString(number % 1000) -FROM system.numbers LIMIT 10000; - --- u64 ---------------------------------------------------- - -SELECT 'u64 and ='; - -SELECT count(*) FROM tab WHERE u64 = 7; -SELECT count(*) FROM tab WHERE u64_tdigest = 7; -SELECT count(*) FROM tab WHERE u64_count_min = 7; -SELECT count(*) FROM tab WHERE u64_uniq = 7; - -SELECT count(*) FROM tab WHERE u64 = 7.7; -SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; -SELECT count(*) FROM tab WHERE u64_count_min = 7.7; -SELECT count(*) FROM tab WHERE u64_uniq = 7.7; - -SELECT count(*) FROM tab WHERE u64 = '7'; -SELECT count(*) FROM tab WHERE u64_tdigest = '7'; -SELECT count(*) FROM tab WHERE u64_count_min = '7'; -SELECT count(*) FROM tab WHERE u64_uniq = '7'; - -SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } - -SELECT 'u64 and <'; - -SELECT count(*) FROM tab WHERE u64 < 7; -SELECT count(*) FROM tab WHERE u64_tdigest < 7; -SELECT count(*) FROM tab WHERE u64_count_min < 7; -SELECT count(*) FROM tab WHERE u64_uniq < 7; - -SELECT count(*) FROM tab WHERE u64 < 7.7; -SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; -SELECT count(*) FROM tab WHERE u64_count_min < 7.7; -SELECT count(*) FROM tab WHERE u64_uniq < 7.7; - -SELECT count(*) FROM tab WHERE u64 < '7'; -SELECT count(*) FROM tab WHERE u64_tdigest < '7'; -SELECT count(*) FROM tab WHERE u64_count_min < '7'; -SELECT count(*) FROM tab WHERE u64_uniq < '7'; - -SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } - --- f64 ---------------------------------------------------- - -SELECT 'f64 and ='; - -SELECT count(*) FROM tab WHERE f64 = 7; -SELECT count(*) FROM tab WHERE f64_tdigest = 7; -SELECT count(*) FROM tab WHERE f64_count_min = 7; -SELECT count(*) FROM tab WHERE f64_uniq = 7; - -SELECT count(*) FROM tab WHERE f64 = 7.7; -SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; -SELECT count(*) FROM tab WHERE f64_count_min = 7.7; -SELECT count(*) FROM tab WHERE f64_uniq = 7.7; - -SELECT count(*) FROM tab WHERE f64 = '7'; -SELECT count(*) FROM tab WHERE f64_tdigest = '7'; -SELECT count(*) FROM tab WHERE f64_count_min = '7'; -SELECT count(*) FROM tab WHERE f64_uniq = '7'; - -SELECT count(*) FROM tab WHERE f64 = '7.7'; -SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; -SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; -SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; - -SELECT 'f64 and <'; - -SELECT count(*) FROM tab WHERE f64 < 7; -SELECT count(*) FROM tab WHERE f64_tdigest < 7; -SELECT count(*) FROM tab WHERE f64_count_min < 7; -SELECT count(*) FROM tab WHERE f64_uniq < 7; - -SELECT count(*) FROM tab WHERE f64 < 7.7; -SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; -SELECT count(*) FROM tab WHERE f64_count_min < 7.7; -SELECT count(*) FROM tab WHERE f64_uniq < 7.7; - -SELECT count(*) FROM tab WHERE f64 < '7'; -SELECT count(*) FROM tab WHERE f64_tdigest < '7'; -SELECT count(*) FROM tab WHERE f64_count_min < '7'; -SELECT count(*) FROM tab WHERE f64_uniq < '7'; - -SELECT count(*) FROM tab WHERE f64 < '7.7'; -SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; -SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; -SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; - --- dt ---------------------------------------------------- - -SELECT 'dt and ='; - -SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; - -SELECT count(*) FROM tab WHERE dt = 7; -SELECT count(*) FROM tab WHERE dt_tdigest = 7; -SELECT count(*) FROM tab WHERE dt_count_min = 7; -SELECT count(*) FROM tab WHERE dt_uniq = 7; - -SELECT 'dt and <'; - -SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; -SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; - -SELECT count(*) FROM tab WHERE dt < 7; -SELECT count(*) FROM tab WHERE dt_tdigest < 7; -SELECT count(*) FROM tab WHERE dt_count_min < 7; -SELECT count(*) FROM tab WHERE dt_uniq < 7; - --- b ---------------------------------------------------- - -SELECT 'b and ='; - -SELECT count(*) FROM tab WHERE b = true; -SELECT count(*) FROM tab WHERE b_tdigest = true; -SELECT count(*) FROM tab WHERE b_count_min = true; -SELECT count(*) FROM tab WHERE b_uniq = true; - -SELECT count(*) FROM tab WHERE b = 'true'; -SELECT count(*) FROM tab WHERE b_tdigest = 'true'; -SELECT count(*) FROM tab WHERE b_count_min = 'true'; -SELECT count(*) FROM tab WHERE b_uniq = 'true'; - -SELECT count(*) FROM tab WHERE b = 1; -SELECT count(*) FROM tab WHERE b_tdigest = 1; -SELECT count(*) FROM tab WHERE b_count_min = 1; -SELECT count(*) FROM tab WHERE b_uniq = 1; - -SELECT count(*) FROM tab WHERE b = 1.1; -SELECT count(*) FROM tab WHERE b_tdigest = 1.1; -SELECT count(*) FROM tab WHERE b_count_min = 1.1; -SELECT count(*) FROM tab WHERE b_uniq = 1.1; - --- s ---------------------------------------------------- - -SELECT 's and ='; - -SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } --- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported -SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } --- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported - -SELECT count(*) FROM tab WHERE s = '7'; --- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported -SELECT count(*) FROM tab WHERE s_count_min = '7'; --- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference new file mode 100644 index 00000000000..77786dbdd8c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.reference @@ -0,0 +1,35 @@ +CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) +After modify TDigest + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) +After drop + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql new file mode 100644 index 00000000000..0f5f353c045 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_uniq.sql @@ -0,0 +1,73 @@ +DROP TABLE IF EXISTS t1; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; + +CREATE TABLE t1 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Int64 STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; + +SHOW CREATE TABLE t1; + +INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); + +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +OPTIMIZE TABLE t1 FINAL; + +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'After modify TDigest'; +ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; +ALTER TABLE t1 MATERIALIZE STATISTICS c; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + + +ALTER TABLE t1 DROP STATISTICS c; + +SELECT 'After drop'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE t2 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c LowCardinality(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3 +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest), + c Nullable(Int64) STATISTICS(tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk +SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; + +DROP TABLE IF EXISTS t3; + diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference deleted file mode 100644 index a9f669b88c1..00000000000 --- a/tests/queries/0_stateless/02864_statistics_usage.reference +++ /dev/null @@ -1,20 +0,0 @@ -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After drop statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) -After add and materialize statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) -After rename - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql deleted file mode 100644 index 4956bd27e87..00000000000 --- a/tests/queries/0_stateless/02864_statistics_usage.sql +++ /dev/null @@ -1,42 +0,0 @@ --- Test that the optimizer picks up column statistics --- (The concrete statistics type, column data type and predicate type don't matter) - --- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; -SET enable_analyzer = 1; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest) -) Engine = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -ALTER TABLE tab DROP STATISTICS a, b; -SELECT 'After drop statistic'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) - -ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; -SELECT 'After add and materialize statistic'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -OPTIMIZE TABLE tab FINAL; -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) - -ALTER TABLE tab RENAME COLUMN b TO c; -SELECT 'After rename'; -SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used) - -DROP TABLE IF EXISTS tab; From 29afd2de785450f2e7f5faec1dc6b35e166cefb4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 13:26:45 +0200 Subject: [PATCH 0744/1722] Remove "Processing configuration file" message from clickhouse-local Make the behaviour identical to the clickhouse-client Signed-off-by: Azat Khuzhin --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 6b0b8fc5b50..200beea7b63 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -143,7 +143,7 @@ void LocalServer::initialize(Poco::Util::Application & self) if (fs::exists(config_path)) { - ConfigProcessor config_processor(config_path, false, true); + ConfigProcessor config_processor(config_path); ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); getClientConfiguration().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); From 4fec61da55c1032f274da87198af59c78cd0d87e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Aug 2024 12:35:27 +0000 Subject: [PATCH 0745/1722] fix wrong datatype in system.kafka_consumers --- src/Storages/System/StorageSystemKafkaConsumers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 86713632339..5e790587716 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -79,7 +79,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & num_rebalance_revocations = assert_cast(*res_columns[index++]); auto & num_rebalance_assigments = assert_cast(*res_columns[index++]); auto & is_currently_used = assert_cast(*res_columns[index++]); - auto & last_used = assert_cast(*res_columns[index++]); + auto & last_used = assert_cast(*res_columns[index++]); auto & rdkafka_stat = assert_cast(*res_columns[index++]); const auto access = context->getAccess(); From 4ef3fe416debecefcea4d7336aac7c679092cf0c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 11 Aug 2024 13:08:53 +0000 Subject: [PATCH 0746/1722] Fix and simplify test --- .../02496_remove_redundant_sorting.reference | 68 ++++++++---------- .../02496_remove_redundant_sorting.sh | 43 ++++++----- ...emove_redundant_sorting_analyzer.reference | 71 ++++++++----------- 3 files changed, 82 insertions(+), 100 deletions(-) diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index a0a1fd60812..4d004f2f78f 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -465,6 +465,37 @@ Expression ((Projection + Before ORDER BY)) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression ((Projection + Before ORDER BY)) + Aggregating + Expression (Before GROUP BY) + Offset + Expression (Projection) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + Before ORDER BY))) + ReadFromStorage (Values) +-- execute +0 -- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function -- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order -- query @@ -527,40 +558,3 @@ Expression (Projection) 2 4 1 3 0 2 --- presence of an inner OFFSET retains the ORDER BY --- query -WITH - t1 AS ( - SELECT SUM(a) AS a, b - FROM - VALUES ( - 'b UInt32, a Int32', - (1, 1), - (2, 0) - ) - GROUP BY 2 - ) -SELECT - SUM(a) -FROM ( - SELECT a, b - FROM t1 - ORDER BY 1 DESC, 2 - OFFSET 1 -) t2 --- explain -Expression (Projection) - Expression (Before ORDER BY) - Aggregating - Expression (Before GROUP BY) - Offset - Expression (Projection) - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - Expression (Projection) - Expression (Before ORDER BY) - Aggregating - Expression (Before GROUP BY) - ReadFromStorage (Values) --- execute -0 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index d59b4387101..c9bd242e429 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -302,6 +302,27 @@ FROM )" run_query "$query" +echo "-- presence of an inner OFFSET retains the ORDER BY" +query="WITH + t1 AS ( + SELECT a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2" +run_query "$query" + echo "-- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function" ENABLE_OPTIMIZATION="SET query_plan_enable_optimizations=0;$ENABLE_OPTIMIZATION" echo "-- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order" @@ -329,25 +350,3 @@ FROM ORDER BY number DESC )" run_query "$query" - -echo "-- presence of an inner OFFSET retains the ORDER BY" -query="WITH - t1 AS ( - SELECT SUM(a) AS a, b - FROM - VALUES ( - 'b UInt32, a Int32', - (1, 1), - (2, 0) - ) - GROUP BY 2 - ) -SELECT - SUM(a) -FROM ( - SELECT a, b - FROM t1 - ORDER BY 1 DESC, 2 - OFFSET 1 -) t2" -run_query "$query" diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index 58441de5f22..dd5ac7bf706 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -464,6 +464,36 @@ Expression ((Project names + Projection)) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 +-- presence of an inner OFFSET retains the ORDER BY +-- query +WITH + t1 AS ( + SELECT a, b + FROM + VALUES ( + 'b UInt32, a Int32', + (1, 1), + (2, 0) + ) + ) +SELECT + SUM(a) +FROM ( + SELECT a, b + FROM t1 + ORDER BY 1 DESC, 2 + OFFSET 1 +) t2 +-- explain +Expression ((Project names + Projection)) + Aggregating + Expression ((Before GROUP BY + (Change column names to column identifiers + Project names))) + Offset + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))) + ReadFromStorage (Values) +-- execute +0 -- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function -- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order -- query @@ -533,44 +563,3 @@ Expression (Project names) 2 4 1 3 0 2 --- presence of an inner OFFSET retains the ORDER BY --- query -WITH - t1 AS ( - SELECT SUM(a) AS a, b - FROM - VALUES ( - 'b UInt32, a Int32', - (1, 1), - (2, 0) - ) - GROUP BY 2 - ) -SELECT - SUM(a) -FROM ( - SELECT a, b - FROM t1 - ORDER BY 1 DESC, 2 - OFFSET 1 -) t2 --- explain -Expression (Project names) - Expression (Projection) - Aggregating - Expression (Before GROUP BY) - Expression (Change column names to column identifiers) - Expression (Project names) - Offset - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - Expression (Projection) - Expression (Change column names to column identifiers) - Expression (Project names) - Expression (Projection) - Aggregating - Expression (Before GROUP BY) - Expression (Change column names to column identifiers) - ReadFromStorage (Values) --- execute -0 From 8e706265e6df3653de76224bfc050b4f52e49282 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Aug 2024 16:29:35 +0000 Subject: [PATCH 0747/1722] fix --- src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 6a0522b0676..3ca7e8183f1 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -125,7 +125,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d { Tuple new_tuple; - for (const auto & child : tuple) + for (auto & child : tuple) if (shardContains(child, name, data)) new_tuple.emplace_back(std::move(child)); From d314e5aa45fb8ac91324721ab278185b09437a40 Mon Sep 17 00:00:00 2001 From: Vladimir Varankin Date: Sun, 11 Aug 2024 18:37:29 +0200 Subject: [PATCH 0748/1722] typos in prometheus.md --- docs/en/interfaces/prometheus.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md index bf541901b34..8e7023cc51f 100644 --- a/docs/en/interfaces/prometheus.md +++ b/docs/en/interfaces/prometheus.md @@ -75,7 +75,7 @@ Data are received by this protocol and written to a [TimeSeries](/en/engines/tab /write - remote_writeremote_write db_name time_series_table
@@ -105,7 +105,7 @@ Data are read from a [TimeSeries](/en/engines/table-engines/special/time_series) /read - remote_readremote_read
db_name time_series_table
@@ -144,14 +144,14 @@ Multiple protocols can be specified together in one place: /write - remote_writeremote_write
db_name.time_series_table
/read - remote_readremote_read
db_name.time_series_table
From e384e2c38e405b1b4758adaa44cd321e6d7f41b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 18:34:33 +0200 Subject: [PATCH 0749/1722] tests: fix 02122_join_group_by_timeout flakiness CI found [1] failure of the test: 2024-08-11 21:06:07 /usr/share/clickhouse-test/queries/0_stateless/02122_join_group_by_timeout.sh: line 51: 52614 Killed timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT -q "SELECT a.name as n And the problem is not the server, but the client, since query executed for ~1 second: 2024.08.11 21:06:02.284318 [ 49232 ] {ba989ee2-f615-49ca-bcd8-31b3916aeb2c} executeQuery: (from [::1]:54144) (comment: 02122_join_group_by_timeout.sh) SELECT a.name as n FROM ( SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 ) AS a, ( SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000 ) as b FORMAT Null SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' (stage: Complete) 2024.08.11 21:06:03.331249 [ 49232 ] {ba989ee2-f615-49ca-bcd8-31b3916aeb2c} executeQuery: Read 517104 rows, 3.95 MiB in 1.072023 sec., 482362.78512681165 rows/sec., 3.68 MiB/sec. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67134/18da3f0ab63da1eef9396627d0dfd56cf5356f65/stateless_tests__msan__[1_4].html So instead of using timeout, let's use time from the system.query_log instead. Signed-off-by: Azat Khuzhin --- .../02122_join_group_by_timeout.reference | 6 +- .../02122_join_group_by_timeout.sh | 70 ++++++++++--------- 2 files changed, 41 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.reference b/tests/queries/0_stateless/02122_join_group_by_timeout.reference index f314e22e519..6500560e8fc 100644 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.reference +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.reference @@ -1,4 +1,6 @@ -Code: 159 -0 +Code: 159 +query_duration 1 +0 +query_duration 1 Code: 159 0 diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.sh b/tests/queries/0_stateless/02122_join_group_by_timeout.sh index 8380c5dbd0c..b4644878544 100755 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.sh +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.sh @@ -1,27 +1,23 @@ #!/usr/bin/env bash -# Tags: no-debug - -# no-debug: Query is canceled by timeout after max_execution_time, -# but sending an exception to the client may hang -# for more than MAX_PROCESS_WAIT seconds in a slow debug build, -# and test will fail. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -MAX_PROCESS_WAIT=5 - -IS_SANITIZER=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.warnings WHERE message like '%built with sanitizer%'") -if [ "$IS_SANITIZER" -gt 0 ]; then - # Query may hang for more than 5 seconds, especially in tsan build - MAX_PROCESS_WAIT=15 +TIMEOUT=5 +IS_SANITIZER_OR_DEBUG=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.warnings WHERE message like '%built with sanitizer%' or message like '%built in debug mode%'") +if [ "$IS_SANITIZER_OR_DEBUG" -gt 0 ]; then + # Increase the timeout due to in debug/sanitizers build: + # - client is slow + # - stacktrace resolving is slow + TIMEOUT=15 fi # TCP CLIENT: As of today (02/12/21) uses PullingAsyncPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) -timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \ - "SELECT * FROM +query_id=$(random_str 12) +$CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q " + SELECT * FROM ( SELECT a.name as n FROM @@ -34,28 +30,35 @@ timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \ GROUP BY n ) LIMIT 20 - FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq + FORMAT Null +" 2>&1 | grep -m1 -o "Code: 159" +$CLICKHOUSE_CLIENT -q "system flush logs" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" + ### Should stop pulling data and return what has been generated already (return code 0) -timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT -q \ - "SELECT a.name as n - FROM - ( - SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 - ) AS a, - ( - SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000 - ) as b - FORMAT Null - SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' - " +query_id=$(random_str 12) +$CLICKHOUSE_CLIENT --query_id "$query_id" -q " + SELECT a.name as n + FROM + ( + SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 + ) AS a, + ( + SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000 + ) as b + FORMAT Null + SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' +" echo $? +$CLICKHOUSE_CLIENT -q "system flush logs" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" # HTTP CLIENT: As of today (02/12/21) uses PullingPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) -${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d \ - "SELECT * FROM +${CLICKHOUSE_CURL} -q --max-time $TIMEOUT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d " + SELECT * FROM ( SELECT a.name as n FROM @@ -68,12 +71,13 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_exec GROUP BY n ) LIMIT 20 - FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq + FORMAT Null +" 2>&1 | grep -o "Code: 159" | sort | uniq ### Should stop pulling data and return what has been generated already (return code 0) -${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \ - "SELECT a.name as n +${CLICKHOUSE_CURL} -q --max-time $TIMEOUT -sS "$CLICKHOUSE_URL" -d " + SELECT a.name as n FROM ( SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000 @@ -83,5 +87,5 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \ ) as b FORMAT Null SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break' - " +" echo $? From 8a48b3334433fe5e77c23ff6df10e454db2d3f82 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Aug 2024 21:27:08 +0200 Subject: [PATCH 0750/1722] Fix settings/current_database in system.processes for async BACKUP/RESTORE Signed-off-by: Azat Khuzhin --- src/Backups/BackupsWorker.cpp | 4 ++++ src/Interpreters/ProcessList.h | 3 +++ 2 files changed, 7 insertions(+) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 0b93ae6d547..8b45c816817 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -490,6 +490,8 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context /// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously. auto process_list_element = context_in_use->getProcessListElement(); + /// Update context to preserve query information in processlist (settings, current_database) + process_list_element->updateContext(context_in_use); thread_pool.scheduleOrThrowOnError( [this, @@ -853,6 +855,8 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt /// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously. auto process_list_element = context_in_use->getProcessListElement(); + /// Update context to preserve query information in processlist (settings, current_database) + process_list_element->updateContext(context_in_use); thread_pool.scheduleOrThrowOnError( [this, diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index accb73e12df..248ba947bc1 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -244,6 +244,9 @@ public: /// Same as checkTimeLimit but it never throws [[nodiscard]] bool checkTimeLimitSoft(); + /// Use it in case of the query left in background to execute asynchronously + void updateContext(ContextWeakPtr weak_context) { context = std::move(weak_context); } + /// Get the reference for the start of the query. Used to synchronize with other Stopwatches UInt64 getQueryCPUStartTime() { return watch.getStart(); } }; From 5c8665c66069256f4e34fb32068fab2fcb90cc65 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Aug 2024 20:40:55 +0000 Subject: [PATCH 0751/1722] fix system.kafka_consumers and doc, fix tidy --- docs/en/operations/system-tables/kafka_consumers.md | 1 + src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp | 2 +- src/Storages/System/StorageSystemKafkaConsumers.cpp | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index 7e28a251e26..d58c9f754fd 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -24,6 +24,7 @@ Columns: - `num_rebalance_revocations`, (UInt64) - number of times the consumer was revoked its partitions - `num_rebalance_assignments`, (UInt64) - number of times the consumer was assigned to Kafka cluster - `is_currently_used`, (UInt8) - consumer is in use +- `last_used`, (UInt64) - last time this consumer was in use, unix time in microseconds - `rdkafka_stat` (String) - library internal statistic. See https://github.com/ClickHouse/librdkafka/blob/master/STATISTICS.md . Set `statistics_interval_ms` to 0 disable, default is 3000 (once in three seconds). Example: diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 3ca7e8183f1..86cec8659f5 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -129,7 +129,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d if (shardContains(child, name, data)) new_tuple.emplace_back(std::move(child)); - if (new_tuple.size() == 0) + if (new_tuple.empty()) new_tuple.emplace_back(std::move(tuple.back())); tuple_literal->value = std::move(new_tuple); diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 5e790587716..db6804d3ad7 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -42,7 +42,7 @@ ColumnsDescription StorageSystemKafkaConsumers::getColumnsDescription() {"num_rebalance_revocations", std::make_shared(), "Number of times the consumer was revoked its partitions."}, {"num_rebalance_assignments", std::make_shared(), "Number of times the consumer was assigned to Kafka cluster."}, {"is_currently_used", std::make_shared(), "The flag which shows whether the consumer is in use."}, - {"last_used", std::make_shared(6), "The last time this consumer was in use."}, + {"last_used", std::make_shared(), "The last time this consumer was in use, unix time in microseconds."}, {"rdkafka_stat", std::make_shared(), "Library internal statistic. Set statistics_interval_ms to 0 disable, default is 3000 (once in three seconds)."}, }; } From 967bd0566336f8c239f0045f703fc8fe428cb28f Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 11 Aug 2024 12:24:13 -0600 Subject: [PATCH 0752/1722] Add create_if_not_exists setting to Settings.h --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6f24db57026..1b52df76c45 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -896,6 +896,7 @@ class IColumn; M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ + M(Bool, create_if_not_exists, false, "Enable IF NOT EXISTS for CREATE statements by default", 0) \ \ \ /* ###################################### */ \ From f90b8327bea16ee81c12f0210d2602889a5944bc Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 11 Aug 2024 11:43:57 -0600 Subject: [PATCH 0753/1722] Update SettingsChangesHistory.cpp with new create_if_not_exists settings --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 511723f1873..202b21a92f0 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -75,6 +75,7 @@ static std::initializer_list Date: Sun, 11 Aug 2024 11:47:30 -0600 Subject: [PATCH 0754/1722] Add support for new create_if_not_exists setting in InterpreterCreateQuery.cpp --- src/Interpreters/InterpreterCreateQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a101e5e8f09..d899b8e111e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1946,6 +1946,8 @@ BlockIO InterpreterCreateQuery::execute() FunctionNameNormalizer::visit(query_ptr.get()); auto & create = query_ptr->as(); + create.if_not_exists |= getContext()->getSettingsRef().create_if_not_exists; + bool is_create_database = create.database && !create.table; if (!create.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { From 2af1134c08ab164b2d77af854166fa30d96fddd9 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 11 Aug 2024 12:17:24 -0600 Subject: [PATCH 0755/1722] Update settings.md docs with new create_if_not_exists settings --- docs/en/operations/settings/settings.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e432f4e038f..22f73a03729 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5637,3 +5637,9 @@ Possible values: - 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. Default value: `0`. + +## create_if_not_exists + +Enable IF NOT EXISTS for CREATE statements by default. If either this setting or IF NOT EXISTS is specified, then no Exception will be thrown when trying to create a new table. + +Default value: `false`. From cc0412c55372108116b6b05fb4fc66ebd4eccae2 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 11 Aug 2024 16:01:48 -0600 Subject: [PATCH 0756/1722] Add test 03221_create_if_not_exists_setting --- ...221_create_if_not_exists_setting.reference | 4 ++ .../03221_create_if_not_exists_setting.sh | 43 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/03221_create_if_not_exists_setting.reference create mode 100755 tests/queries/0_stateless/03221_create_if_not_exists_setting.sh diff --git a/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference b/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference new file mode 100644 index 00000000000..8740b05c9ca --- /dev/null +++ b/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference @@ -0,0 +1,4 @@ +57 +82 +0 +0 diff --git a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh new file mode 100755 index 00000000000..cfbe2eb8fd9 --- /dev/null +++ b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# $CLICKHOUSE_CLIENT -mn -q "SET create_if_not_exists=0;" # Default +$CLICKHOUSE_CLIENT -mn -q " +DROP TABLE IF EXISTS example_table; +CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; +CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; +" 2> /dev/null +# ensure failed error code +echo $? +$CLICKHOUSE_CLIENT -mn -q " +DROP DATABASE IF EXISTS example_database; +CREATE DATABASE example_database; +CREATE DATABASE example_database; +" 2> /dev/null +echo $? + +$CLICKHOUSE_CLIENT -mn -q " +SET create_if_not_exists=1; +DROP TABLE IF EXISTS example_table; +CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; +CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; +" +# ensure successful error code +echo $? + + +$CLICKHOUSE_CLIENT -mn -q " +SET create_if_not_exists=1; +DROP DATABASE IF EXISTS example_database; +CREATE DATABASE example_database; +CREATE DATABASE example_database; +" +echo $? + +$CLICKHOUSE_CLIENT -mn -q " +DROP DATABASE example_database; +DROP TABLE example_table; +" \ No newline at end of file From 957a0b6ea4c3e262a5c1fa664d81ab31d7e0d757 Mon Sep 17 00:00:00 2001 From: sakulali Date: Sun, 11 Aug 2024 00:12:36 +0800 Subject: [PATCH 0757/1722] Add a setting query_cache_tag --- docs/en/operations/query-cache.md | 10 ++++++ docs/en/operations/settings/settings.md | 11 +++++++ .../operations/system-tables/query_cache.md | 2 ++ src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/Cache/QueryCache.cpp | 16 +++++++--- src/Interpreters/Cache/QueryCache.h | 13 ++++++-- src/Interpreters/executeQuery.cpp | 5 +-- .../System/StorageSystemQueryCache.cpp | 5 ++- .../02494_query_cache_tag.reference | 14 ++++++++ .../0_stateless/02494_query_cache_tag.sql | 32 +++++++++++++++++++ 11 files changed, 100 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_tag.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_tag.sql diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 7a920671fc2..a6c4d74f4ac 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -143,6 +143,16 @@ value can be specified at session, profile or query level using setting [query_c Entries in the query cache are compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads from the query cache. To disable compression, use setting [query_cache_compress_entries](settings/settings.md#query-cache-compress-entries). +Entries in the query cache can separate by tag, using setting [query_cache_tag](settings/settings.md#query-cache-tag). Queries with different tags are considered different entries. For example, the result of query + +``` sql +SELECT 1 SETTINGS use_query_cache = true; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; +``` + +have different entries in the query cache, find the specified tag in system table [system.query_cache](system-tables/query_cache.md) + ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation, etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e432f4e038f..7b855665efb 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1800,6 +1800,17 @@ Possible values: Default value: `0`. +## query_cache_tag {#query-cache-tag} + +An arbitrary string to separate entries in the [query cache](../query-cache.md). +Queries with different values of this setting are considered different. + +Possible values: + +- string: name of query cache tag + +Default value: `''`. + ## query_cache_max_size_in_bytes {#query-cache-max-size-in-bytes} The maximum amount of memory (in bytes) the current user may allocate in the [query cache](../query-cache.md). 0 means unlimited. diff --git a/docs/en/operations/system-tables/query_cache.md b/docs/en/operations/system-tables/query_cache.md index a9f86f5fc2b..393b37d3616 100644 --- a/docs/en/operations/system-tables/query_cache.md +++ b/docs/en/operations/system-tables/query_cache.md @@ -14,6 +14,7 @@ Columns: - `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed. - `expires_at` ([DateTime](../../sql-reference/data-types/datetime.md)) — When the query cache entry becomes stale. - `key_hash` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — A hash of the query string, used as a key to find query cache entries. +- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — An arbitrary string to separate entries in the query cache. **Example** @@ -31,6 +32,7 @@ shared: 0 compressed: 1 expires_at: 2023-10-13 13:35:45 key_hash: 12188185624808016954 +tag: 1 row in set. Elapsed: 0.004 sec. ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4559cc67b35..ed58f8041d0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -676,6 +676,7 @@ class IColumn; M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \ M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \ M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \ + M(String, query_cache_tag, "", "An arbitrary string to separate entries in the query cache. Queries with different values of this setting are considered different.", 0) \ M(Bool, enable_sharing_sets_for_mutations, true, "Allow sharing set objects build for IN subqueries between different tasks of the same mutation. This reduces memory usage and CPU consumption", 0) \ \ M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c6392044f72..49a325b07b1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -84,6 +84,7 @@ static std::initializer_list user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed_) - : ast_hash(calculateAstHash(ast_, current_database, settings)) + bool is_compressed_, + const String & tag_) + : ast_hash(calculateAstHash(ast_, current_database, settings, tag_)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -242,11 +247,12 @@ QueryCache::Key::Key( , expires_at(expires_at_) , is_compressed(is_compressed_) , query_string(queryStringFromAST(ast_)) + , tag(tag_) { } -QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_) - : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles +QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_, const String & tag_) + : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false, tag_) /// dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 461197cac32..54de5edb145 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -88,6 +88,10 @@ public: /// SYSTEM.QUERY_CACHE. const String query_string; + /// An arbitrary string to separate entries in the query cache. + /// Queries with different values of this setting are considered different. + const String tag; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, const String & current_database, @@ -96,10 +100,15 @@ public: std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed); + bool is_compressed, + const String & tag_); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). - Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_); + Key(ASTPtr ast_, + const String & current_database, + const Settings & settings, + std::optional user_id_, const std::vector & current_user_roles_, + const String & tag_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index fe87eed5570..6422d3128fa 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1129,7 +1129,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles()); + QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles(), settings.query_cache_tag); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1258,7 +1258,8 @@ static std::tuple executeQueryImpl( context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), - settings.query_cache_compress_entries); + settings.query_cache_compress_entries, + settings.query_cache_tag); const size_t num_query_runs = settings.query_cache_min_query_runs ? query_cache->recordQueryRun(key) : 1; /// try to avoid locking a mutex in recordQueryRun() if (num_query_runs <= settings.query_cache_min_query_runs) diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 4c54d4ae16f..f81d50e8806 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -1,6 +1,7 @@ #include "StorageSystemQueryCache.h" #include #include +#include #include #include #include @@ -19,7 +20,8 @@ ColumnsDescription StorageSystemQueryCache::getColumnsDescription() {"shared", std::make_shared(), "If the query cache entry is shared between multiple users."}, {"compressed", std::make_shared(), "If the query cache entry is compressed."}, {"expires_at", std::make_shared(), "When the query cache entry becomes stale."}, - {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."} + {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."}, + {"tag", std::make_shared(std::make_shared()), "An arbitrary string to separate entries in the query cache."} }; } @@ -56,6 +58,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[4]->insert(key.is_compressed); res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); res_columns[6]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258) + res_columns[7]->insert(key.tag); } } diff --git a/tests/queries/0_stateless/02494_query_cache_tag.reference b/tests/queries/0_stateless/02494_query_cache_tag.reference new file mode 100644 index 00000000000..055d3d4c5bb --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_tag.reference @@ -0,0 +1,14 @@ +1 +1 +--- +1 +1 +1 +2 +--- +1 +1 +1 +2 +1 +3 diff --git a/tests/queries/0_stateless/02494_query_cache_tag.sql b/tests/queries/0_stateless/02494_query_cache_tag.sql new file mode 100644 index 00000000000..054607058e8 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_tag.sql @@ -0,0 +1,32 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SYSTEM DROP QUERY CACHE; + +-- Cache the query after the query invocation +SELECT 1 SETTINGS use_query_cache = true; +SELECT COUNT(*) FROM system.query_cache; + +SELECT '---'; + +SYSTEM DROP QUERY CACHE; + +-- Queries with tag value of this setting or not are considered different cache entries. +SELECT 1 SETTINGS use_query_cache = true; +SELECT COUNT(*) FROM system.query_cache; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; +SELECT COUNT(*) FROM system.query_cache; + +SELECT '---'; + +SYSTEM DROP QUERY CACHE; + +-- Queries with different tags values of this setting are considered different cache entries. +SELECT 1 SETTINGS use_query_cache = true; +SELECT COUNT(*) FROM system.query_cache; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; +SELECT COUNT(*) FROM system.query_cache; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; +SELECT COUNT(*) FROM system.query_cache; + +SYSTEM DROP QUERY CACHE; From 4866581a6769d9dda8da236286d15c84828ccc4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 04:59:21 +0200 Subject: [PATCH 0758/1722] Fix DeltaLake partitioned by Bool when it is true --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c896a760597..89b48f08438 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -332,6 +332,8 @@ struct DeltaLakeMetadataImpl WhichDataType which(check_type->getTypeId()); if (which.isStringOrFixedString()) return value; + else if (isBool(data_type)) + return parse(value); else if (which.isInt8()) return parse(value); else if (which.isUInt8()) From 7e277d5656c4e66da2f830386d066ec491384d0e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 05:04:51 +0200 Subject: [PATCH 0759/1722] Fix Upgrade check --- docker/test/upgrade/run.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index a4c4c75e5b3..e9768873f6a 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -129,6 +129,7 @@ configure # Check that all new/changed setting were added in settings changes history. # Some settings can be different for builds with sanitizers, so we check +# Also the automatic value of 'max_threads' and similar was displayed as "'auto(...)'" in previous versions instead of "auto(...)". # settings changes only for non-sanitizer builds. IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM system.build_options WHERE name = 'CXX_FLAGS'") if [ "${IS_SANITIZED}" -eq "0" ] @@ -145,7 +146,9 @@ then old_settings.value AS old_value FROM new_settings LEFT JOIN old_settings ON new_settings.name = old_settings.name - WHERE (new_settings.value != old_settings.value) AND (name NOT IN ( + WHERE (new_value != old_value) + AND NOT (startsWith(new_value, 'auto(') AND old_value LIKE '%auto(%') + AND (name NOT IN ( SELECT arrayJoin(tupleElement(changes, 'name')) FROM ( @@ -177,7 +180,7 @@ then if [ -s changed_settings.txt ] then mv changed_settings.txt /test_output/ - echo -e "Changed settings are not reflected in settings changes history (see changed_settings.txt)$FAIL$(head_escaped /test_output/changed_settings.txt)" >> /test_output/test_results.tsv + echo -e "Changed settings are not reflected in the settings changes history (see changed_settings.txt)$FAIL$(head_escaped /test_output/changed_settings.txt)" >> /test_output/test_results.tsv else echo -e "There are no changed settings or they are reflected in settings changes history$OK" >> /test_output/test_results.tsv fi From 1e15574a25b2eaa4d08c1a14a005ba0b3ebcfc23 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 05:53:43 +0200 Subject: [PATCH 0760/1722] Fix ExternalDistributed --- src/Storages/StorageExternalDistributed.cpp | 35 ++++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 4277387bc5d..9fc8b588c89 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -1,4 +1,4 @@ -#include "StorageExternalDistributed.h" +#include #include #include @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -111,7 +113,30 @@ void registerStorageExternalDistributed(StorageFactory & factory) std::unordered_set shards; ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end()); - String addresses_expr = checkAndGetLiteralArgument(engine_args[1], "addresses"); + + ASTPtr * address_arg = nullptr; + + /// If there is a named collection argument, named `addresses_expr` + for (auto & node : inner_engine_args) + { + if (ASTFunction * func = node->as(); func && func->name == "equals" && func->arguments) + { + if (ASTExpressionList * func_args = func->arguments->as(); func_args && func_args->children.size() == 2) + { + if (ASTIdentifier * arg_name = func_args->children[0]->as(); arg_name && arg_name->name() == "addresses_expr") + { + address_arg = &func_args->children[1]; + break; + } + } + } + } + + /// Otherwise it is the first argument. + if (!address_arg) + address_arg = &inner_engine_args.at(0); + + String addresses_expr = checkAndGetLiteralArgument(*address_arg, "addresses"); Strings shards_addresses = get_addresses(addresses_expr); auto engine_name = checkAndGetLiteralArgument(engine_args[0], "engine_name"); @@ -120,7 +145,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) auto format_settings = StorageURL::getFormatSettingsFromArgs(args); for (const auto & shard_address : shards_addresses) { - inner_engine_args.at(0) = std::make_shared(shard_address); + *address_arg = std::make_shared(shard_address); auto configuration = StorageURL::getConfiguration(inner_engine_args, context); auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses); if (uri_options.size() > 1) @@ -144,7 +169,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) MySQLSettings mysql_settings; for (const auto & shard_address : shards_addresses) { - inner_engine_args.at(0) = std::make_shared(shard_address); + *address_arg = std::make_shared(shard_address); auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings); configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306); auto pool = createMySQLPoolWithFailover(configuration, mysql_settings); @@ -160,7 +185,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) { for (const auto & shard_address : shards_addresses) { - inner_engine_args.at(0) = std::make_shared(shard_address); + *address_arg = std::make_shared(shard_address); auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context); configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432); auto pool = std::make_shared( From a41c1305887d08f43c02e354bb307f69a16b3fb0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 06:05:44 +0200 Subject: [PATCH 0761/1722] Update 02675_profile_events_from_query_log_and_client.sh --- .../02675_profile_events_from_query_log_and_client.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index 894b2b61563..ff534a6a2e6 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-merge-tree-settings # Tag no-fasttest: needs s3 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 5acf9f6f8160ee6de2845b13bed07d63832ca3fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:01:52 +0200 Subject: [PATCH 0762/1722] Fix `test_cluster_all_replicas` --- tests/integration/test_cluster_all_replicas/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_cluster_all_replicas/test.py b/tests/integration/test_cluster_all_replicas/test.py index d8bad180e1b..9797db7c498 100644 --- a/tests/integration/test_cluster_all_replicas/test.py +++ b/tests/integration/test_cluster_all_replicas/test.py @@ -21,14 +21,14 @@ def start_cluster(): def test_cluster(start_cluster): assert ( node1.query( - "SELECT hostName() FROM clusterAllReplicas('one_shard_two_nodes', system.one)" + "SELECT hostName() FROM clusterAllReplicas('one_shard_two_nodes', system.one) ORDER BY ALL" ) == "node1\nnode2\n" ) assert set( node1.query( - """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) ORDER BY dummy""" + """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) ORDER BY ALL""" ).splitlines() ) == {"node1\t0", "node2\t0"} @@ -48,7 +48,7 @@ def test_global_in(start_cluster): assert set( node1.query( - """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u""" + """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u ORDER BY ALL""" ).splitlines() ) == {"node1\t0", "node2\t0"} @@ -63,7 +63,7 @@ def test_global_in(start_cluster): def test_skip_unavailable_replica(start_cluster, cluster): assert ( node1.query( - f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) settings skip_unavailable_shards=1" + f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) ORDER BY ALL settings skip_unavailable_shards=1" ) == "node1\nnode2\n" ) @@ -81,5 +81,5 @@ def test_error_on_unavailable_replica(start_cluster, cluster): # so when skip_unavailable_shards=0 - any unavailable replica should lead to an error with pytest.raises(QueryRuntimeException): node1.query( - f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) settings skip_unavailable_shards=0" + f"SELECT hostName() FROM clusterAllReplicas('{cluster}', system.one) ORDER BY ALL settings skip_unavailable_shards=0" ) From 20b97a45bf3c73960e71e1a158cec35ec522ccff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:09:42 +0200 Subject: [PATCH 0763/1722] Fix fundamentally broken test CC @azat --- tests/integration/test_throttling/test.py | 62 ++++++++++------------- 1 file changed, 28 insertions(+), 34 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index c53c2bb1ddf..4bd96e2756d 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -121,21 +121,15 @@ def node_update_config(mode, setting, value=None): node.restart_clickhouse() -def assert_took(took, should_took): +def assert_took(took, should_take): # we need to decrease the lower limit because the server limits could # be enforced by throttling some server background IO instead of query IO # and we have no control over it - # - # and the same for upper limit, it can be slightly larger, due to for - # instance network latencies or CPU starvation - if should_took > 0: - assert took >= should_took * 0.85 and took <= should_took * 1.8 - else: - assert took >= should_took * 0.85 + assert took >= should_take * 0.85 @pytest.mark.parametrize( - "policy,backup_name,mode,setting,value,should_took", + "policy,backup_name,mode,setting,value,should_take", [ # # Local -> Local @@ -149,7 +143,7 @@ def assert_took(took, should_took): 0, id="no_local_throttling", ), - # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "default", next_backup_name("local"), @@ -159,7 +153,7 @@ def assert_took(took, should_took): 7, id="user_local_throttling", ), - # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # reading 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "default", next_backup_name("local"), @@ -181,7 +175,7 @@ def assert_took(took, should_took): 0, id="no_remote_to_local_throttling", ), - # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "s3", next_backup_name("local"), @@ -191,7 +185,7 @@ def assert_took(took, should_took): 7, id="user_remote_to_local_throttling", ), - # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # reading 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "s3", next_backup_name("local"), @@ -252,7 +246,7 @@ def assert_took(took, should_took): 0, id="no_local_to_remote_throttling", ), - # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "default", next_backup_name("remote"), @@ -262,7 +256,7 @@ def assert_took(took, should_took): 7, id="user_local_to_remote_throttling", ), - # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # reading 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "default", next_backup_name("remote"), @@ -274,7 +268,7 @@ def assert_took(took, should_took): ), ], ) -def test_backup_throttling(policy, backup_name, mode, setting, value, should_took): +def test_backup_throttling(policy, backup_name, mode, setting, value, should_take): node_update_config(mode, setting, value) node.query( f""" @@ -284,7 +278,7 @@ def test_backup_throttling(policy, backup_name, mode, setting, value, should_too """ ) _, took = elapsed(node.query, f"backup table data to {backup_name}") - assert_took(took, should_took) + assert_took(took, should_take) def test_backup_throttling_override(): @@ -305,18 +299,18 @@ def test_backup_throttling_override(): "max_backup_bandwidth": "500K", }, ) - # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds + # reading 1e6*8 bytes with 500Ki default bandwidth should take (8-0.5)/0.5=15 seconds assert_took(took, 15) @pytest.mark.parametrize( - "policy,mode,setting,value,should_took", + "policy,mode,setting,value,should_take", [ # # Local # pytest.param("default", None, None, None, 0, id="no_local_throttling"), - # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "default", "user", @@ -325,7 +319,7 @@ def test_backup_throttling_override(): 7, id="user_local_throttling", ), - # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # reading 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "default", "server", @@ -338,7 +332,7 @@ def test_backup_throttling_override(): # Remote # pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), - # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "s3", "user", @@ -347,7 +341,7 @@ def test_backup_throttling_override(): 7, id="user_remote_throttling", ), - # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # reading 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "s3", "server", @@ -358,7 +352,7 @@ def test_backup_throttling_override(): ), ], ) -def test_read_throttling(policy, mode, setting, value, should_took): +def test_read_throttling(policy, mode, setting, value, should_take): node_update_config(mode, setting, value) node.query( f""" @@ -368,17 +362,17 @@ def test_read_throttling(policy, mode, setting, value, should_took): """ ) _, took = elapsed(node.query, f"select * from data") - assert_took(took, should_took) + assert_took(took, should_take) @pytest.mark.parametrize( - "policy,mode,setting,value,should_took", + "policy,mode,setting,value,should_take", [ # # Local # pytest.param("default", None, None, None, 0, id="no_local_throttling"), - # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "default", "user", @@ -387,7 +381,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): 7, id="local_user_throttling", ), - # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # reading 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "default", "server", @@ -400,7 +394,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # Remote # pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), - # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # writing 1e6*8 bytes with 1M default bandwidth should take (8-1)/1=7 seconds pytest.param( "s3", "user", @@ -409,7 +403,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): 7, id="user_remote_throttling", ), - # writing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # writing 1e6*8 bytes with 2M default bandwidth should take (8-2)/2=3 seconds pytest.param( "s3", "server", @@ -420,7 +414,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): ), ], ) -def test_write_throttling(policy, mode, setting, value, should_took): +def test_write_throttling(policy, mode, setting, value, should_take): node_update_config(mode, setting, value) node.query( f""" @@ -429,7 +423,7 @@ def test_write_throttling(policy, mode, setting, value, should_took): """ ) _, took = elapsed(node.query, f"insert into data select * from numbers(1e6)") - assert_took(took, should_took) + assert_took(took, should_take) def test_max_mutations_bandwidth_for_server(): @@ -444,7 +438,7 @@ def test_max_mutations_bandwidth_for_server(): node.query, "alter table data update key = -key where 1 settings mutations_sync = 1", ) - # reading 1e6*8 bytes with 1M/s bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M/s bandwidth should take (8-1)/1=7 seconds assert_took(took, 7) @@ -457,5 +451,5 @@ def test_max_merges_bandwidth_for_server(): ) node.query("insert into data select * from numbers(1e6)") _, took = elapsed(node.query, "optimize table data final") - # reading 1e6*8 bytes with 1M/s bandwith should take (8-1)/1=7 seconds + # reading 1e6*8 bytes with 1M/s bandwidth should take (8-1)/1=7 seconds assert_took(took, 7) From b3504def35ba6baf7a7cf1f9b84a72e8f70f95f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:15:08 +0200 Subject: [PATCH 0764/1722] Fix leftovers --- tests/config/config.d/transactions.xml | 4 ++-- .../test_distributed_type_object/configs/remote_servers.xml | 2 +- .../configs/host_regexp.xml | 4 ++-- .../configs/listen_host.xml | 2 +- .../test_jbod_ha/configs/config.d/storage_configuration.xml | 2 +- .../integration/test_server_reload/configs/default_passwd.xml | 2 +- .../test_server_reload/configs/overrides_from_zk.xml | 2 +- tests/integration/test_version_update/configs/log_conf.xml | 2 +- 8 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/config/config.d/transactions.xml b/tests/config/config.d/transactions.xml index 9948b1f1865..64e166b81b5 100644 --- a/tests/config/config.d/transactions.xml +++ b/tests/config/config.d/transactions.xml @@ -1,4 +1,4 @@ - + 42 @@ -18,4 +18,4 @@ 0.01 - + diff --git a/tests/integration/test_distributed_type_object/configs/remote_servers.xml b/tests/integration/test_distributed_type_object/configs/remote_servers.xml index ebce4697529..0ea61f0d5fc 100644 --- a/tests/integration/test_distributed_type_object/configs/remote_servers.xml +++ b/tests/integration/test_distributed_type_object/configs/remote_servers.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml index 7a2141e6c7e..0bf7fad9a70 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml @@ -1,4 +1,4 @@ - + @@ -8,4 +8,4 @@ default - \ No newline at end of file + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml index 58ef55cd3f3..4f0841ab8b6 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml @@ -1,4 +1,4 @@ - + :: 0.0.0.0 1 diff --git a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml index b5c351d105b..b2c4645644a 100644 --- a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml @@ -1,4 +1,4 @@ - + 1000 diff --git a/tests/integration/test_server_reload/configs/default_passwd.xml b/tests/integration/test_server_reload/configs/default_passwd.xml index f79149e7e23..45ae005bd19 100644 --- a/tests/integration/test_server_reload/configs/default_passwd.xml +++ b/tests/integration/test_server_reload/configs/default_passwd.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_server_reload/configs/overrides_from_zk.xml b/tests/integration/test_server_reload/configs/overrides_from_zk.xml index d420faa88a2..b17c5c9fa99 100644 --- a/tests/integration/test_server_reload/configs/overrides_from_zk.xml +++ b/tests/integration/test_server_reload/configs/overrides_from_zk.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_version_update/configs/log_conf.xml b/tests/integration/test_version_update/configs/log_conf.xml index f9d15e572aa..17215c1759d 100644 --- a/tests/integration/test_version_update/configs/log_conf.xml +++ b/tests/integration/test_version_update/configs/log_conf.xml @@ -1,4 +1,4 @@ - + trace /var/log/clickhouse-server/log.log From b92a8f0fbcec640fbac35fca1b3fb23914d44990 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:16:25 +0200 Subject: [PATCH 0765/1722] Fix leftovers --- .../external-authenticators/kerberos.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/zh/operations/external-authenticators/kerberos.md b/docs/zh/operations/external-authenticators/kerberos.md index 649a0b9bd48..d1a39bbc952 100644 --- a/docs/zh/operations/external-authenticators/kerberos.md +++ b/docs/zh/operations/external-authenticators/kerberos.md @@ -23,30 +23,30 @@ slug: /zh/operations/external-authenticators/kerberos 示例 (进入 `config.xml`): ```xml - + - + ``` 主体规范: ```xml - + HTTP/clickhouse.example.com@EXAMPLE.COM - + ``` 按领域过滤: ```xml - + EXAMPLE.COM - + ``` !!! warning "注æ„" @@ -74,7 +74,7 @@ Kerberos主体å称格å¼é€šå¸¸éµå¾ªä»¥ä¸‹æ¨¡å¼: 示例 (进入 `users.xml`): ``` - + @@ -85,7 +85,7 @@ Kerberos主体å称格å¼é€šå¸¸éµå¾ªä»¥ä¸‹æ¨¡å¼: - + ``` !!! warning "警告" From 6f189e9eb7ded15df4ddbe7f90dafb28feaab2e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:17:17 +0200 Subject: [PATCH 0766/1722] Fix leftovers --- .../test_distributed_type_object/configs/remote_servers.xml | 2 +- .../configs/host_regexp.xml | 2 +- .../configs/listen_host.xml | 2 +- .../test_jbod_ha/configs/config.d/storage_configuration.xml | 2 +- tests/integration/test_server_reload/configs/default_passwd.xml | 2 +- .../test_server_reload/configs/overrides_from_zk.xml | 2 +- tests/integration/test_version_update/configs/log_conf.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_distributed_type_object/configs/remote_servers.xml b/tests/integration/test_distributed_type_object/configs/remote_servers.xml index 0ea61f0d5fc..68b420f36b4 100644 --- a/tests/integration/test_distributed_type_object/configs/remote_servers.xml +++ b/tests/integration/test_distributed_type_object/configs/remote_servers.xml @@ -15,4 +15,4 @@ - + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml index 0bf7fad9a70..9329c8dbde2 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/host_regexp.xml @@ -8,4 +8,4 @@ default - + diff --git a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml index 4f0841ab8b6..9c27c612f63 100644 --- a/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml +++ b/tests/integration/test_host_regexp_hosts_file_resolution/configs/listen_host.xml @@ -2,4 +2,4 @@ :: 0.0.0.0 1 - + diff --git a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml index b2c4645644a..fb9acc58ad6 100644 --- a/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_jbod_ha/configs/config.d/storage_configuration.xml @@ -27,4 +27,4 @@ - + diff --git a/tests/integration/test_server_reload/configs/default_passwd.xml b/tests/integration/test_server_reload/configs/default_passwd.xml index 45ae005bd19..9d664cbf9c4 100644 --- a/tests/integration/test_server_reload/configs/default_passwd.xml +++ b/tests/integration/test_server_reload/configs/default_passwd.xml @@ -9,4 +9,4 @@ 123 - + diff --git a/tests/integration/test_server_reload/configs/overrides_from_zk.xml b/tests/integration/test_server_reload/configs/overrides_from_zk.xml index b17c5c9fa99..aa6105f6ebe 100644 --- a/tests/integration/test_server_reload/configs/overrides_from_zk.xml +++ b/tests/integration/test_server_reload/configs/overrides_from_zk.xml @@ -7,4 +7,4 @@ - + diff --git a/tests/integration/test_version_update/configs/log_conf.xml b/tests/integration/test_version_update/configs/log_conf.xml index 17215c1759d..27c7107ce5e 100644 --- a/tests/integration/test_version_update/configs/log_conf.xml +++ b/tests/integration/test_version_update/configs/log_conf.xml @@ -8,4 +8,4 @@ /var/log/clickhouse-server/stderr.log /var/log/clickhouse-server/stdout.log - + From d31c56e0d9756642d90885016369e9ca7994e3f0 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 11 Aug 2024 23:17:24 -0600 Subject: [PATCH 0767/1722] Update 03221_create_if_not_exists_setting.sh --- .../queries/0_stateless/03221_create_if_not_exists_setting.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh index cfbe2eb8fd9..8dcde8977bc 100755 --- a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh +++ b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh @@ -38,6 +38,6 @@ CREATE DATABASE example_database; echo $? $CLICKHOUSE_CLIENT -mn -q " -DROP DATABASE example_database; -DROP TABLE example_table; +DROP DATABASE IF EXISTS example_database; +DROP TABLE IF EXISTS example_table; " \ No newline at end of file From 6016dc96aae57f38ef3ace1ed687b82bcc437425 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:19:54 +0200 Subject: [PATCH 0768/1722] Fix test `01172_transaction_counters` --- programs/server/config.d/transactions.xml | 1 + .../01172_transaction_counters.reference | 44 +++++++++---------- .../01172_transaction_counters.sql | 1 - 3 files changed, 23 insertions(+), 23 deletions(-) create mode 120000 programs/server/config.d/transactions.xml diff --git a/programs/server/config.d/transactions.xml b/programs/server/config.d/transactions.xml new file mode 120000 index 00000000000..be9de46b607 --- /dev/null +++ b/programs/server/config.d/transactions.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/transactions.xml \ No newline at end of file diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 24083d7d40b..0fd73c7bcec 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -16,25 +16,25 @@ 7 all_3_3_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 7 all_4_4_0 (0,0,'00000000-0000-0000-0000-000000000000') 0 8 1 -1 1 AddPart 1 1 1 1 all_1_1_0 -2 1 Begin 1 1 1 1 -2 1 AddPart 1 1 1 1 all_2_2_0 -2 1 Rollback 1 1 1 1 -3 1 Begin 1 1 1 1 -3 1 AddPart 1 1 1 1 all_3_3_0 -3 1 Commit 1 1 1 0 -1 1 LockPart 1 1 1 1 all_2_2_0 -4 1 Begin 1 1 1 1 -4 1 AddPart 1 1 1 1 all_4_4_0 -4 1 Commit 1 1 1 0 -5 1 Begin 1 1 1 1 -5 1 AddPart 1 1 1 1 all_5_5_0 -5 1 LockPart 1 1 1 1 all_1_1_0 -5 1 LockPart 1 1 1 1 all_3_3_0 -5 1 LockPart 1 1 1 1 all_4_4_0 -5 1 LockPart 1 1 1 1 all_5_5_0 -5 1 UnlockPart 1 1 1 1 all_1_1_0 -5 1 UnlockPart 1 1 1 1 all_3_3_0 -5 1 UnlockPart 1 1 1 1 all_4_4_0 -5 1 UnlockPart 1 1 1 1 all_5_5_0 -5 1 Rollback 1 1 1 1 +1 AddPart 1 1 1 1 all_1_1_0 +2 Begin 1 1 1 1 +2 AddPart 1 1 1 1 all_2_2_0 +2 Rollback 1 1 1 1 +3 Begin 1 1 1 1 +3 AddPart 1 1 1 1 all_3_3_0 +3 Commit 1 1 1 0 +1 LockPart 1 1 1 1 all_2_2_0 +4 Begin 1 1 1 1 +4 AddPart 1 1 1 1 all_4_4_0 +4 Commit 1 1 1 0 +5 Begin 1 1 1 1 +5 AddPart 1 1 1 1 all_5_5_0 +5 LockPart 1 1 1 1 all_1_1_0 +5 LockPart 1 1 1 1 all_3_3_0 +5 LockPart 1 1 1 1 all_4_4_0 +5 LockPart 1 1 1 1 all_5_5_0 +5 UnlockPart 1 1 1 1 all_1_1_0 +5 UnlockPart 1 1 1 1 all_3_3_0 +5 UnlockPart 1 1 1 1 all_4_4_0 +5 UnlockPart 1 1 1 1 all_5_5_0 +5 Rollback 1 1 1 1 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index a809e4196e9..581b45cd15c 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -42,7 +42,6 @@ rollback; system flush logs; select indexOf((select arraySort(groupUniqArray(tid)) from system.transactions_info_log where database=currentDatabase() and table='txn_counters'), tid), - (toDecimal64(now64(6), 6) - toDecimal64(event_time, 6)) < 100, type, thread_id!=0, length(query_id)=length(queryID()) or type='Commit' and query_id='', -- ignore fault injection after commit From c6ae8abd87b1741472cdbb1ebcdf64c36570dda1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 07:39:25 +0200 Subject: [PATCH 0769/1722] Fix `00600_replace_running_query` --- .../00600_replace_running_query.sh | 21 ++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index 7a71d17f19b..ad7c49a9ad3 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -13,7 +13,18 @@ ${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600${TEST_PREFIX} function wait_for_query_to_start() { - while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done + while [[ 0 -eq $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") ]] + do + sleep 0.1 + done +} + +function wait_for_queries_to_finish() +{ + while [[ 0 -ne $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query NOT LIKE '%this query%'") ]] + do + sleep 0.1 + done } @@ -25,8 +36,9 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d # Wait for it to be replaced wait +wait_for_queries_to_finish -${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & +${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' & wait_for_query_to_start '42' # Trying to run another query with the same query_id @@ -37,10 +49,13 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=42&replace_running_query=1" -d 'S $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" > /dev/null wait +wait_for_queries_to_finish -${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & +${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' & wait_for_query_to_start '42' ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null wait +wait_for_queries_to_finish + ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --query='SELECT 44' ${CLICKHOUSE_CLIENT} -q "drop user u_00600${TEST_PREFIX}" From 574c445be9368ee481e8d213251106233d417f69 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 16:29:07 +0000 Subject: [PATCH 0770/1722] Refactor tests for (experimental) statistics --- docs/en/development/tests.md | 4 +- .../statements/alter/statistics.md | 16 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- ...2864_statistics_count_min_sketch.reference | 14 -- .../02864_statistics_count_min_sketch.sql | 70 ------ .../02864_statistics_ddl.reference | 37 +-- .../0_stateless/02864_statistics_ddl.sql | 234 ++++++++++++++---- ...delayed_materialization_in_merge.reference | 12 + ...stics_delayed_materialization_in_merge.sql | 36 +++ .../02864_statistics_exception.reference | 0 .../02864_statistics_exception.sql | 55 ---- ..._statistics_materialize_in_merge.reference | 10 - .../02864_statistics_materialize_in_merge.sql | 52 ---- .../02864_statistics_predicates.reference | 98 ++++++++ .../02864_statistics_predicates.sql | 214 ++++++++++++++++ .../02864_statistics_uniq.reference | 35 --- .../0_stateless/02864_statistics_uniq.sql | 73 ------ .../02864_statistics_usage.reference | 20 ++ .../0_stateless/02864_statistics_usage.sql | 42 ++++ 19 files changed, 625 insertions(+), 399 deletions(-) delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.sql create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference create mode 100644 tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_exception.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.reference create mode 100644 tests/queries/0_stateless/02864_statistics_predicates.sql delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_uniq.sql create mode 100644 tests/queries/0_stateless/02864_statistics_usage.reference create mode 100644 tests/queries/0_stateless/02864_statistics_usage.sql diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 269995a1a96..6cb36e2049b 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. :::note A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs @@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes ### Adding a New Test -To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/sql-reference/statements/alter/statistics.md b/docs/en/sql-reference/statements/alter/statistics.md index 6880cef0e5c..7a1774a01b5 100644 --- a/docs/en/sql-reference/statements/alter/statistics.md +++ b/docs/en/sql-reference/statements/alter/statistics.md @@ -8,26 +8,28 @@ sidebar_label: STATISTICS The following operations are available: -- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata. +- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata. -- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata. +- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata. -- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. +- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns. -- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. +- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`. -- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing statistics metadata via ZooKeeper. -There is an example adding two statistics types to two columns: +## Example: + +Adding two statistics types to two columns: ``` ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq; ``` :::note -Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 625b1281c61..c7101021f02 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3517,7 +3517,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistics is not not safe " + "ALTER types of column {} with statistics is not safe " "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference deleted file mode 100644 index 02c41656a36..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql deleted file mode 100644 index c730aa7b4a7..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql +++ /dev/null @@ -1,70 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS tab SYNC; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET allow_suspicious_low_cardinality_types=1; -SET mutations_sync = 2; - -CREATE TABLE tab -( - a String, - b UInt64, - c Int64, - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE tab; - -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'Test statistics count_min:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min; -ALTER TABLE tab ADD STATISTICS c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - - -SELECT 'Test statistics multi-types:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab DROP STATISTICS a, b, c; - -DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_ddl.reference b/tests/queries/0_stateless/02864_statistics_ddl.reference index a7ff5caa0b0..0e453b0ee8a 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.reference +++ b/tests/queries/0_stateless/02864_statistics_ddl.reference @@ -1,31 +1,6 @@ -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -10 -0 -After drop statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(a, 10)) (removed) -10 -CREATE TABLE default.tab\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After add statistic -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After materialize statistic - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(b, 10)) (removed) -20 -CREATE TABLE default.tab\n(\n `a` Float64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After rename - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, 10)) (removed) -20 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64 STATISTICS(tdigest, uniq),\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32 STATISTICS(tdigest, uniq),\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `f64` Float64,\n `f64_tdigest` Float64 STATISTICS(tdigest),\n `f32` Float32,\n `s` String,\n `a` Array(Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index fe612efe2ac..32b56a842b7 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -1,59 +1,195 @@ --- Tests that various DDL statements create/drop/materialize statistics +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests that DDL statements which create / drop / materialize statistics + +SET mutations_sync = 1; DROP TABLE IF EXISTS tab; +-- Error case: Can't create statistics when allow_experimental_statistics = 0 +CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; + +-- Error case: Unknown statistics types are rejected +CREATE TABLE tab (col Float64 STATISTICS(no_statistics_type)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +-- Error case: The same statistics type can't exist more than once on a column +CREATE TABLE tab (col Float64 STATISTICS(tdigest, tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } + +SET allow_suspicious_low_cardinality_types = 1; + +-- Statistics can only be created on columns of specific data types (depending on the statistics kind), (*) + +-- tdigest requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- uniq requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- count_min requires data_type.isValueRepresentedByInteger or data_type = (Fixed)String +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col String STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col FixedString(1) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col Array(Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + +-- CREATE TABLE was easy, ALTER is more fun CREATE TABLE tab ( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f32 Float32, + s String, + a Array(Float64) +) +Engine = MergeTree() +ORDER BY tuple(); +-- Error case: Unknown statistics types are rejected +-- (relevant for ADD and MODIFY) +ALTER TABLE tab ADD STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } +-- for some reason, ALTER TABLE tab MODIFY STATISTICS IF EXISTS is not supported + +-- Error case: The same statistics type can't exist more than once on a column +-- (relevant for ADD and MODIFY) +-- Create the same statistics object twice +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest, tdigest; -- { serverError INCORRECT_QUERY } +-- Create an statistics which exists already +ALTER TABLE tab ADD STATISTICS f64_tdigest TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS f64_tdigest TYPE tdigest; -- no-op +ALTER TABLE tab MODIFY STATISTICS f64_tdigest TYPE tdigest; -- no-op + +-- Error case: Column does not exist +-- (relevant for ADD, MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab ADD STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab ADD STATISTICS IF NOT EXISTS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS no_such_column TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab CLEAR STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS no_such_column; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS no_such_column; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS no_such_column; -- { serverError ILLEGAL_STATISTICS } + +-- Error case: Column exists but has no statistics +-- (relevant for MODIFY, DROP, CLEAR, and MATERIALIZE) +-- Note that the results are unfortunately quite inconsistent ... +ALTER TABLE tab MODIFY STATISTICS s TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab DROP STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab CLEAR STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab CLEAR STATISTICS IF EXISTS s; -- no-op +ALTER TABLE tab MATERIALIZE STATISTICS s; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MATERIALIZE STATISTICS IF EXISTS s; -- { serverError ILLEGAL_STATISTICS } + +-- We don't check systematically that that statistics can only be created via ALTER ADD STATISTICS on columns of specific data types (the +-- internal type validation code is tested already above, (*)). Only do a rudimentary check for each statistics type with a data type that +-- works and one that doesn't work. +-- tdigest +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE tdigest; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } +-- uniq +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE uniq; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE uniq; -- { serverError ILLEGAL_STATISTICS } +-- count_min +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } + +-- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing +-- statistics objects (e.g. tdigest can be created on Float64 and UInt64) +ALTER TABLE tab MODIFY COLUMN f64_tdigest UInt64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } + +-- Finally, do a full-circle test of a good case. Print table definition after each step. +-- Intentionally specifying _two_ columns and _two_ statistics types to have that also tested. +SHOW CREATE TABLE tab; +ALTER TABLE tab ADD STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab MODIFY STATISTICS f64, f32 TYPE tdigest, uniq; +SHOW CREATE TABLE tab; +ALTER TABLE tab CLEAR STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab MATERIALIZE STATISTICS f64, f32; +SHOW CREATE TABLE tab; +ALTER TABLE tab DROP STATISTICS f64, f32; SHOW CREATE TABLE tab; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; -SELECT count(*) FROM tab WHERE b < NULL and a < '10'; - -ALTER TABLE tab DROP STATISTICS a, b; - -SELECT 'After drop statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; - -SELECT 'After add statistic'; - -SHOW CREATE TABLE tab; - -ALTER TABLE tab MATERIALIZE STATISTICS a, b; -INSERT INTO tab select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000; - -SELECT 'After materialize statistic'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -OPTIMIZE TABLE tab FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE b < 10 and a < 10; - -ALTER TABLE tab RENAME COLUMN b TO c; -SHOW CREATE TABLE tab; - -SELECT 'After rename'; -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT count(*) FROM tab WHERE c < 10 and a < 10; - -DROP TABLE IF EXISTS tab; +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference new file mode 100644 index 00000000000..eb5e685597c --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference @@ -0,0 +1,12 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After truncate, insert, and materialize + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql new file mode 100644 index 00000000000..d469a4c2036 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.sql @@ -0,0 +1,36 @@ +-- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +DROP TABLE IF EXISTS tab; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET enable_analyzer = 1; + +SET materialize_statistics_on_insert = 0; + +CREATE TABLE tab +( + a Int64 STATISTICS(tdigest), + b Int16 STATISTICS(tdigest), +) ENGINE = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. + +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +TRUNCATE TABLE tab; +SET mutations_sync = 2; +INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +SELECT 'After truncate, insert, and materialize'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_exception.reference b/tests/queries/0_stateless/02864_statistics_exception.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02864_statistics_exception.sql b/tests/queries/0_stateless/02864_statistics_exception.sql deleted file mode 100644 index 289ffee6600..00000000000 --- a/tests/queries/0_stateless/02864_statistics_exception.sql +++ /dev/null @@ -1,55 +0,0 @@ --- Tests creating/dropping/materializing statistics produces the right exceptions. - -DROP TABLE IF EXISTS tab; - --- Can't create statistics when allow_experimental_statistics = 0 -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - -SET allow_experimental_statistics = 1; - --- The same type of statistics can't exist more than once on a column -CREATE TABLE tab -( - a Float64 STATISTICS(tdigest, tdigest) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- Unknown statistics types are rejected -CREATE TABLE tab -( - a Float64 STATISTICS(no_statistics_type) -) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } - --- tDigest statistics can only be created on numeric columns -CREATE TABLE tab -( - a String STATISTICS(tdigest), -) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } - -CREATE TABLE tab -( - a Float64, - b String -) Engine = MergeTree() ORDER BY tuple(); - -ALTER TABLE tab ADD STATISTICS a TYPE no_statistics_type; -- { serverError INCORRECT_QUERY } -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS IF NOT EXISTS a TYPE tdigest; -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab MODIFY STATISTICS a TYPE tdigest; --- Statistics can be created only on integer columns -ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS b; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab DROP STATISTICS a; -ALTER TABLE tab DROP STATISTICS IF EXISTS a; -ALTER TABLE tab CLEAR STATISTICS a; -- { serverError ILLEGAL_STATISTICS } -ALTER TABLE tab CLEAR STATISTICS IF EXISTS a; -ALTER TABLE tab MATERIALIZE STATISTICS b; -- { serverError ILLEGAL_STATISTICS } - -ALTER TABLE tab ADD STATISTICS a TYPE tdigest; -ALTER TABLE tab MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; -ALTER TABLE tab MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference deleted file mode 100644 index 5e969cf41cb..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.reference +++ /dev/null @@ -1,10 +0,0 @@ -10 -10 -10 -statistics not used Condition less(b, 10_UInt8) moved to PREWHERE -statistics not used Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after merge Condition less(b, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(a, 10_UInt8) moved to PREWHERE -statistics used after materialize Condition less(b, 10_UInt8) moved to PREWHERE -2 0 diff --git a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql b/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql deleted file mode 100644 index 6606cff263f..00000000000 --- a/tests/queries/0_stateless/02864_statistics_materialize_in_merge.sql +++ /dev/null @@ -1,52 +0,0 @@ --- Tests delayed materialization of statistics in merge instead of during insert (setting 'materialize_statistics_on_insert = 0'). - -DROP TABLE IF EXISTS tab; - -SET enable_analyzer = 1; -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; - -SET materialize_statistics_on_insert = 0; - -CREATE TABLE tab -( - a Int64 STATISTICS(tdigest), - b Int16 STATISTICS(tdigest), -) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0; -- TODO: there is a bug in vertical merge with statistics. - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics not used'; - -OPTIMIZE TABLE tab FINAL; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after merge'; - -TRUNCATE TABLE tab; -SET mutations_sync = 2; - -INSERT INTO tab SELECT number, -number FROM system.numbers LIMIT 10000; -ALTER TABLE tab MATERIALIZE STATISTICS a, b; - -SELECT count(*) FROM tab WHERE b < 10 and a < 10 SETTINGS log_comment = 'statistics used after materialize'; - -DROP TABLE tab; - -SYSTEM FLUSH LOGS; - -SELECT log_comment, message FROM system.text_log JOIN -( - SELECT Settings['log_comment'] AS log_comment, query_id FROM system.query_log - WHERE current_database = currentDatabase() - AND query LIKE 'SELECT count(*) FROM tab%' - AND type = 'QueryFinish' -) AS query_log USING (query_id) -WHERE message LIKE '%moved to PREWHERE%' -ORDER BY event_time_microseconds; - -SELECT count(), sum(ProfileEvents['MergeTreeDataWriterStatisticsCalculationMicroseconds']) -FROM system.query_log -WHERE current_database = currentDatabase() - AND query LIKE 'INSERT INTO tab SELECT%' - AND type = 'QueryFinish'; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference new file mode 100644 index 00000000000..ffbd7269e05 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -0,0 +1,98 @@ +u64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +10 +u64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +70 +f64 and = +10 +10 +10 +10 +0 +0 +0 +0 +10 +10 +10 +10 +0 +0 +0 +0 +f64 and < +70 +70 +70 +70 +80 +80 +80 +80 +70 +70 +70 +70 +80 +80 +80 +80 +dt and = +0 +0 +0 +0 +10 +10 +10 +10 +dt and < +10000 +10000 +10000 +10000 +70 +70 +70 +70 +b and = +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +5000 +0 +0 +0 +0 +s and = +10 +10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql new file mode 100644 index 00000000000..779116cf19a --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -0,0 +1,214 @@ +-- Tags: no-fasttest +-- no-fasttest: 'count_min' sketches need a 3rd party library + +-- Tests the cross product of all predicates with all right-hand sides on all data types and all statistics types. + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + u64 UInt64, + u64_tdigest UInt64 STATISTICS(tdigest), + u64_count_min UInt64 STATISTICS(count_min), + u64_uniq UInt64 STATISTICS(uniq), + f64 Float64, + f64_tdigest Float64 STATISTICS(tdigest), + f64_count_min Float64 STATISTICS(count_min), + f64_uniq Float64 STATISTICS(uniq), + dt DateTime, + dt_tdigest DateTime STATISTICS(tdigest), + dt_count_min DateTime STATISTICS(count_min), + dt_uniq DateTime STATISTICS(uniq), + b Bool, + b_tdigest Bool STATISTICS(tdigest), + b_count_min Bool STATISTICS(count_min), + b_uniq Bool STATISTICS(uniq), + s String, + -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest + s_count_min String STATISTICS(count_min) + -- s_uniq String STATISTICS(uniq), -- not supported by uniq +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab +-- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; +SELECT number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 1000, + number % 2, + number % 2, + number % 2, + number % 2, + toString(number % 1000), + toString(number % 1000) +FROM system.numbers LIMIT 10000; + +-- u64 ---------------------------------------------------- + +SELECT 'u64 and ='; + +SELECT count(*) FROM tab WHERE u64 = 7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7; +SELECT count(*) FROM tab WHERE u64_count_min = 7; +SELECT count(*) FROM tab WHERE u64_uniq = 7; + +SELECT count(*) FROM tab WHERE u64 = 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE u64_count_min = 7.7; +SELECT count(*) FROM tab WHERE u64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE u64 = '7'; +SELECT count(*) FROM tab WHERE u64_tdigest = '7'; +SELECT count(*) FROM tab WHERE u64_count_min = '7'; +SELECT count(*) FROM tab WHERE u64_uniq = '7'; + +SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } + +SELECT 'u64 and <'; + +SELECT count(*) FROM tab WHERE u64 < 7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7; +SELECT count(*) FROM tab WHERE u64_count_min < 7; +SELECT count(*) FROM tab WHERE u64_uniq < 7; + +SELECT count(*) FROM tab WHERE u64 < 7.7; +SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE u64_count_min < 7.7; +SELECT count(*) FROM tab WHERE u64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE u64 < '7'; +SELECT count(*) FROM tab WHERE u64_tdigest < '7'; +SELECT count(*) FROM tab WHERE u64_count_min < '7'; +SELECT count(*) FROM tab WHERE u64_uniq < '7'; + +SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } + +-- f64 ---------------------------------------------------- + +SELECT 'f64 and ='; + +SELECT count(*) FROM tab WHERE f64 = 7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7; +SELECT count(*) FROM tab WHERE f64_count_min = 7; +SELECT count(*) FROM tab WHERE f64_uniq = 7; + +SELECT count(*) FROM tab WHERE f64 = 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE f64_count_min = 7.7; +SELECT count(*) FROM tab WHERE f64_uniq = 7.7; + +SELECT count(*) FROM tab WHERE f64 = '7'; +SELECT count(*) FROM tab WHERE f64_tdigest = '7'; +SELECT count(*) FROM tab WHERE f64_count_min = '7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7'; + +SELECT count(*) FROM tab WHERE f64 = '7.7'; +SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; +SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; + +SELECT 'f64 and <'; + +SELECT count(*) FROM tab WHERE f64 < 7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7; +SELECT count(*) FROM tab WHERE f64_count_min < 7; +SELECT count(*) FROM tab WHERE f64_uniq < 7; + +SELECT count(*) FROM tab WHERE f64 < 7.7; +SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE f64_count_min < 7.7; +SELECT count(*) FROM tab WHERE f64_uniq < 7.7; + +SELECT count(*) FROM tab WHERE f64 < '7'; +SELECT count(*) FROM tab WHERE f64_tdigest < '7'; +SELECT count(*) FROM tab WHERE f64_count_min < '7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7'; + +SELECT count(*) FROM tab WHERE f64 < '7.7'; +SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; +SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; +SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; + +-- dt ---------------------------------------------------- + +SELECT 'dt and ='; + +SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt = 7; +SELECT count(*) FROM tab WHERE dt_tdigest = 7; +SELECT count(*) FROM tab WHERE dt_count_min = 7; +SELECT count(*) FROM tab WHERE dt_uniq = 7; + +SELECT 'dt and <'; + +SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; + +SELECT count(*) FROM tab WHERE dt < 7; +SELECT count(*) FROM tab WHERE dt_tdigest < 7; +SELECT count(*) FROM tab WHERE dt_count_min < 7; +SELECT count(*) FROM tab WHERE dt_uniq < 7; + +-- b ---------------------------------------------------- + +SELECT 'b and ='; + +SELECT count(*) FROM tab WHERE b = true; +SELECT count(*) FROM tab WHERE b_tdigest = true; +SELECT count(*) FROM tab WHERE b_count_min = true; +SELECT count(*) FROM tab WHERE b_uniq = true; + +SELECT count(*) FROM tab WHERE b = 'true'; +SELECT count(*) FROM tab WHERE b_tdigest = 'true'; +SELECT count(*) FROM tab WHERE b_count_min = 'true'; +SELECT count(*) FROM tab WHERE b_uniq = 'true'; + +SELECT count(*) FROM tab WHERE b = 1; +SELECT count(*) FROM tab WHERE b_tdigest = 1; +SELECT count(*) FROM tab WHERE b_count_min = 1; +SELECT count(*) FROM tab WHERE b_uniq = 1; + +SELECT count(*) FROM tab WHERE b = 1.1; +SELECT count(*) FROM tab WHERE b_tdigest = 1.1; +SELECT count(*) FROM tab WHERE b_count_min = 1.1; +SELECT count(*) FROM tab WHERE b_uniq = 1.1; + +-- s ---------------------------------------------------- + +SELECT 's and ='; + +SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } +-- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported + +SELECT count(*) FROM tab WHERE s = '7'; +-- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported +SELECT count(*) FROM tab WHERE s_count_min = '7'; +-- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02864_statistics_uniq.reference b/tests/queries/0_stateless/02864_statistics_uniq.reference deleted file mode 100644 index 77786dbdd8c..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.reference +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE default.t1\n(\n `a` Float64 STATISTICS(tdigest),\n `b` Int64 STATISTICS(tdigest),\n `c` Int64 STATISTICS(tdigest, uniq),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -After insert - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After merge - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(c, 11), less(a, 10), less(b, 10)) (removed) -After modify TDigest - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(c, -1), less(a, 10), less(b, 10)) (removed) -After drop - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 11), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), equals(c, 0), less(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(a, 10), less(c, -1), less(b, 10)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_uniq.sql b/tests/queries/0_stateless/02864_statistics_uniq.sql deleted file mode 100644 index 0f5f353c045..00000000000 --- a/tests/queries/0_stateless/02864_statistics_uniq.sql +++ /dev/null @@ -1,73 +0,0 @@ -DROP TABLE IF EXISTS t1; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET mutations_sync = 1; - -CREATE TABLE t1 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Int64 STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE t1; - -INSERT INTO t1 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; -INSERT INTO t1 select 0, 0, 11, generateUUIDv4(); - -SELECT 'After insert'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -OPTIMIZE TABLE t1 FINAL; - -SELECT 'After merge'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -SELECT 'After modify TDigest'; -ALTER TABLE t1 MODIFY STATISTICS c TYPE TDigest; -ALTER TABLE t1 MATERIALIZE STATISTICS c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - - -ALTER TABLE t1 DROP STATISTICS c; - -SELECT 'After drop'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 11 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c = 0 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and c < -1 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE t2 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c LowCardinality(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t2 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t2; -DROP TABLE IF EXISTS t3; - -CREATE TABLE t3 -( - a Float64 STATISTICS(tdigest), - b Int64 STATISTICS(tdigest), - c Nullable(Int64) STATISTICS(tdigest, uniq), - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO t3 select number, -number, number/1000, generateUUIDv4() FROM system.numbers LIMIT 10000; - -DROP TABLE IF EXISTS t3; - diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference new file mode 100644 index 00000000000..a9f669b88c1 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.reference @@ -0,0 +1,20 @@ +After insert + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After drop statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) +After add and materialize statistic + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After merge + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) +After rename + Prewhere info + Prewhere filter + Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.sql b/tests/queries/0_stateless/02864_statistics_usage.sql new file mode 100644 index 00000000000..4956bd27e87 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_usage.sql @@ -0,0 +1,42 @@ +-- Test that the optimizer picks up column statistics +-- (The concrete statistics type, column data type and predicate type don't matter) + +-- Checks by the predicate evaluation order in EXPLAIN. This is quite fragile, a better approach would be helpful (maybe 'send_logs_level'?) + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET mutations_sync = 1; +SET enable_analyzer = 1; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + a Float64 STATISTICS(tdigest), + b Int64 STATISTICS(tdigest) +) Engine = MergeTree() ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After insert'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab DROP STATISTICS a, b; +SELECT 'After drop statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks b first, then a (statistics not used) + +ALTER TABLE tab ADD STATISTICS a, b TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b; +INSERT INTO tab select number, -number FROM system.numbers LIMIT 10000; +SELECT 'After add and materialize statistic'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +OPTIMIZE TABLE tab FINAL; +SELECT 'After merge'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then b (statistics used) + +ALTER TABLE tab RENAME COLUMN b TO c; +SELECT 'After rename'; +SELECT replaceRegexpAll(explain, '__table1\.', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%'; -- checks a first, then c (statistics used) + +DROP TABLE IF EXISTS tab; From ee433684ddc4614a5bb93dbfd1e3b481a2f343d6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Aug 2024 10:55:16 +0000 Subject: [PATCH 0771/1722] Follow-up to ClickHouse#63898, pt. III --- .../02421_simple_queries_for_opentelemetry.sh | 10 +++---- .../02458_insert_select_progress_tcp.sh | 2 +- .../02476_analyzer_identifier_hints.sh | 8 +++--- .../0_stateless/02480_tets_show_full.sh | 2 +- .../0_stateless/02482_load_parts_refcounts.sh | 2 +- .../02496_remove_redundant_sorting.sh | 6 ++--- .../02497_storage_file_reader_selection.sh | 2 +- .../02500_remove_redundant_distinct.sh | 6 ++--- ...atabase_replicated_no_arguments_for_rmt.sh | 4 +-- .../0_stateless/02532_send_logs_level_test.sh | 2 +- .../02555_davengers_rename_chain.sh | 8 +++--- ...02572_query_views_log_background_thread.sh | 8 +++--- .../02703_max_local_read_bandwidth.sh | 4 +-- .../02703_max_local_write_bandwidth.sh | 4 +-- .../0_stateless/02704_max_backup_bandwidth.sh | 4 +-- .../0_stateless/02724_limit_num_mutations.sh | 8 +++--- .../02725_async_insert_table_setting.sh | 4 +-- .../queries/0_stateless/02841_local_assert.sh | 6 ++--- ...rge_across_partitions_final_with_lonely.sh | 6 ++--- .../02871_clickhouse_client_restart_pager.sh | 2 +- .../02875_clickhouse_local_multiquery.sh | 4 +-- .../02875_merge_engine_set_index.sh | 4 +-- ...tion_table_with_explicit_insert_columns.sh | 2 +- ...ture_from_insertion_table_with_defaults.sh | 2 +- .../02883_named_collections_override.sh | 2 +- .../02884_async_insert_native_protocol_1.sh | 4 +-- .../02884_async_insert_native_protocol_2.sh | 4 +-- .../02884_async_insert_native_protocol_3.sh | 4 +-- .../02884_async_insert_native_protocol_4.sh | 4 +-- .../02885_ephemeral_columns_from_file.sh | 2 +- .../0_stateless/02895_npy_output_format.sh | 2 +- ...ak_memory_usage_http_headers_regression.sh | 2 +- .../02903_empty_order_by_throws_error.sh | 4 +-- .../02903_rmt_retriable_merge_exception.sh | 6 ++--- ...904_empty_order_by_with_setting_enabled.sh | 8 +++--- .../02907_backup_mv_with_no_inner_table.sh | 8 +++--- .../02907_backup_mv_with_no_source_table.sh | 12 ++++----- .../02907_backup_restore_default_nullable.sh | 4 +-- .../02907_backup_restore_flatten_nested.sh | 8 +++--- .../02907_clickhouse_dictionary_bug.sh | 2 +- .../02907_system_backups_profile_events.sh | 6 ++--- .../0_stateless/02908_Npy_files_caching.sh | 4 +-- .../0_stateless/02908_table_ttl_dependency.sh | 6 ++--- .../02909_settings_in_json_schema_cache.sh | 2 +- .../02915_input_table_function_in_subquery.sh | 2 +- .../02915_lazy_loading_of_base_backups.sh | 26 +++++++++---------- .../0_stateless/02916_dictionary_access.sh | 8 +++--- .../0_stateless/02916_joinget_dependency.sh | 6 ++--- .../02930_client_file_log_comment.sh | 2 +- ...lumn_use_structure_from_insertion_table.sh | 2 +- .../02940_system_stacktrace_optimizations.sh | 6 ++--- ..._alter_metadata_merge_checksum_mismatch.sh | 8 +++--- .../02947_merge_tree_index_table_3.sh | 4 +-- ...2950_dictionary_ssd_cache_short_circuit.sh | 2 +- .../02950_distributed_initial_query_event.sh | 2 +- .../02974_backup_query_format_null.sh | 4 +-- ...ert_select_resize_to_max_insert_threads.sh | 2 +- .../03008_deduplication_random_setttings.sh | 6 ++--- .../03008_local_plain_rewritable.sh | 14 +++++----- .../03031_clickhouse_local_input.sh | 8 +++--- .../0_stateless/03032_async_backup_restore.sh | 4 +-- .../03096_http_interface_role_query_param.sh | 4 +-- ...03140_client_subsequent_external_tables.sh | 2 +- .../03143_prewhere_profile_events.sh | 12 ++++----- .../03145_non_loaded_projection_backup.sh | 12 ++++----- .../03155_test_move_to_prewhere.sh | 4 +-- .../03156_default_multiquery_split.sh | 2 +- .../0_stateless/03169_time_virtual_column.sh | 2 +- .../03173_parallel_replicas_join_bug.sh | 2 +- .../03198_settings_in_csv_tsv_schema_cache.sh | 8 +++--- .../03198_unload_primary_key_outdated.sh | 4 +-- .../03199_dictionary_table_access.sh | 8 +++--- ...s_to_read_for_schema_inference_in_cache.sh | 2 +- 73 files changed, 186 insertions(+), 186 deletions(-) diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh index 98b571c5968..91e85eabcb8 100755 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # $2 - query function execute_query() { - ${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=1 --query_id $1 -nq " + ${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=1 --query_id $1 -q " ${2} " } @@ -18,7 +18,7 @@ function execute_query() # so we only to check the db.statement only function check_query_span_query_only() { -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; SELECT attribute['db.statement'] as query FROM system.opentelemetry_span_log @@ -31,7 +31,7 @@ ${CLICKHOUSE_CLIENT} -nq " function check_query_span() { -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; SELECT attribute['db.statement'] as query, attribute['clickhouse.read_rows'] as read_rows, @@ -47,7 +47,7 @@ ${CLICKHOUSE_CLIENT} -nq " # # Set up # -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.opentelemetry_test; CREATE TABLE ${CLICKHOUSE_DATABASE}.opentelemetry_test (id UInt64) Engine=MergeTree Order By id; " @@ -79,4 +79,4 @@ check_query_span $query_id # ${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.opentelemetry_test; -" \ No newline at end of file +" diff --git a/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh b/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh index ae3ea017fbb..178da822d41 100755 --- a/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh +++ b/tests/queries/0_stateless/02458_insert_select_progress_tcp.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists insert_select_progress_tcp; create table insert_select_progress_tcp(s UInt16) engine = MergeTree order by s; " diff --git a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh index 4c850a6ec9e..92f519a9f8a 100755 --- a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh +++ b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( @@ -74,7 +74,7 @@ $CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, arrayMap(lambda_argument -> l $CLICKHOUSE_CLIENT -q "WITH 1 AS constant_value SELECT (SELECT constant_valu) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS test_table_compound; CREATE TABLE test_table_compound ( @@ -142,7 +142,7 @@ $CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constan $CLICKHOUSE_CLIENT -q "WITH cast(tuple(1), 'Tuple(value_1 String)') AS constant_value SELECT (SELECT constant_value.value_) SETTINGS enable_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS test_table_1; CREATE TABLE test_table_1 ( @@ -185,7 +185,7 @@ $CLICKHOUSE_CLIENT -q "SELECT ((1))::Tuple(a Tuple(b UInt32)) AS t, t.a.c SETTIN $CLICKHOUSE_CLIENT -q "SELECT 1"; -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE test_table; DROP TABLE test_table_compound; DROP TABLE test_table_1; diff --git a/tests/queries/0_stateless/02480_tets_show_full.sh b/tests/queries/0_stateless/02480_tets_show_full.sh index 5f5040ba128..50184857a1f 100755 --- a/tests/queries/0_stateless/02480_tets_show_full.sh +++ b/tests/queries/0_stateless/02480_tets_show_full.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) database=$($CLICKHOUSE_CLIENT -q 'SELECT currentDatabase()') -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS test_02480_table; DROP VIEW IF EXISTS test_02480_view; CREATE TABLE test_02480_table (id Int64) ENGINE=MergeTree ORDER BY id; diff --git a/tests/queries/0_stateless/02482_load_parts_refcounts.sh b/tests/queries/0_stateless/02482_load_parts_refcounts.sh index 5303824d97c..4dc7a7fd99b 100755 --- a/tests/queries/0_stateless/02482_load_parts_refcounts.sh +++ b/tests/queries/0_stateless/02482_load_parts_refcounts.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS load_parts_refcounts SYNC; CREATE TABLE load_parts_refcounts (id UInt32) diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh index c9bd242e429..6e132c55628 100755 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.sh +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.sh @@ -26,15 +26,15 @@ FROM ORDER BY number DESC ) ORDER BY number ASC" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;EXPLAIN $query" +$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;EXPLAIN $query" function run_query { echo "-- query" echo "$1" echo "-- explain" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;EXPLAIN $1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;EXPLAIN $1" echo "-- execute" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;$1" } echo "-- Enabled query_plan_remove_redundant_sorting" diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index aa43e81f131..27243dd47fa 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -13,6 +13,6 @@ $CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SEL $CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap" && echo 'Fail' || echo 0 $CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 1 || echo 'Fail' -$CLICKHOUSE_CLIENT --storage_file_read_method=mmap -nq "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }" +$CLICKHOUSE_CLIENT --storage_file_read_method=mmap -q "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }" rm $DATA_FILE diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh index 3c06119e8d2..6fd42fa940a 100755 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh @@ -24,15 +24,15 @@ FROM ) )" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;EXPLAIN $query" +$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;EXPLAIN $query" function run_query { echo "-- query" echo "$1" echo "-- explain" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;EXPLAIN $1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;EXPLAIN $1" echo "-- execute" - $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$1" + $CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;$1" } echo "-- Enabled $OPTIMIZATION_SETTING" diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index a0f228e6af4..c1aa24943c1 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -14,8 +14,8 @@ ${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings databa ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.sh b/tests/queries/0_stateless/02532_send_logs_level_test.sh index 506ac2331f2..a50539311cb 100755 --- a/tests/queries/0_stateless/02532_send_logs_level_test.sh +++ b/tests/queries/0_stateless/02532_send_logs_level_test.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key Int) engine=MergeTree order by tuple() settings min_bytes_for_wide_part = '1G', compress_marks = 1; insert into data values (1); diff --git a/tests/queries/0_stateless/02555_davengers_rename_chain.sh b/tests/queries/0_stateless/02555_davengers_rename_chain.sh index 660a95846c4..196507dc72e 100755 --- a/tests/queries/0_stateless/02555_davengers_rename_chain.sh +++ b/tests/queries/0_stateless/02555_davengers_rename_chain.sh @@ -46,7 +46,7 @@ tables["wrong_metadata_compact"]="min_bytes_for_wide_part = 10000000" for table in "${!tables[@]}"; do settings="${tables[$table]}" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS $table; CREATE TABLE $table( @@ -69,7 +69,7 @@ for table in "${!tables[@]}"; do wait_column "$table" "\`a1\` UInt64" || exit 2 - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" -- { echoOn } SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null; @@ -82,7 +82,7 @@ for table in "${!tables[@]}"; do wait_mutation_loaded "$table" "b1 TO a" || exit 2 - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" -- { echoOn } SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null; @@ -94,7 +94,7 @@ for table in "${!tables[@]}"; do wait_for_all_mutations "$table" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" -- { echoOn } SELECT 'ECHO_ALIGNMENT_FIX' FORMAT Null; diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh index 509cd03f6c2..22b94e09b58 100755 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sh +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "drop table if exists buffer_02572; +${CLICKHOUSE_CLIENT} --ignore-error --query "drop table if exists buffer_02572; drop table if exists data_02572; drop table if exists copy_02572; drop table if exists mv_02572;" ${CLICKHOUSE_CLIENT} --query="create table copy_02572 (key Int) engine=Memory();" @@ -21,7 +21,7 @@ ${CLICKHOUSE_CLIENT} --query="insert into buffer_02572 values (1);" if [ $(( $(date +%s) - start )) -gt 6 ]; then # clickhouse test cluster is overloaded, will skip # ensure that the flush was not direct - ${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" + ${CLICKHOUSE_CLIENT} --ignore-error --query "select * from data_02572; select * from copy_02572;" fi # we cannot use OPTIMIZE, this will attach query context, so let's wait @@ -31,11 +31,11 @@ for _ in {1..100}; do done -${CLICKHOUSE_CLIENT} --ignore-error --multiquery --query "select * from data_02572; select * from copy_02572;" +${CLICKHOUSE_CLIENT} --ignore-error --query "select * from data_02572; select * from copy_02572;" ${CLICKHOUSE_CLIENT} --query="system flush logs;" ${CLICKHOUSE_CLIENT} --query="select count() > 0, lower(status::String), errorCodeToName(exception_code) from system.query_views_log where view_name = concatWithSeparator('.', currentDatabase(), 'mv_02572') and view_target = concatWithSeparator('.', currentDatabase(), 'copy_02572') - group by 2, 3;" \ No newline at end of file + group by 2, 3;" diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index 03e0f363d71..79253648475 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; " @@ -26,7 +26,7 @@ read_methods=( for read_method in "${read_methods[@]}"; do query_id=$(random_str 10) $CLICKHOUSE_CLIENT --query_id "$query_id" -q "select * from data format Null settings max_local_read_bandwidth='1M', local_filesystem_read_method='$read_method'" - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT '$read_method', diff --git a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh index 4f6a300c5b3..c5776134673 100755 --- a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; " @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -nm -q " query_id=$(random_str 10) # writes 1e6*8 bytes with 1M bandwith it should take (8-1)/1=7 seconds $CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data select * from numbers(1e6) settings max_local_write_bandwidth='1M'" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT query_duration_ms >= 7e3, diff --git a/tests/queries/0_stateless/02704_max_backup_bandwidth.sh b/tests/queries/0_stateless/02704_max_backup_bandwidth.sh index 8cb03a93a7a..7e914c4c539 100755 --- a/tests/queries/0_stateless/02704_max_backup_bandwidth.sh +++ b/tests/queries/0_stateless/02704_max_backup_bandwidth.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data; create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; " @@ -15,7 +15,7 @@ $CLICKHOUSE_CLIENT -q "insert into data select * from numbers(1e6)" query_id=$(random_str 10) $CLICKHOUSE_CLIENT --query_id "$query_id" -q "backup table data to Disk('backups', '$CLICKHOUSE_DATABASE/data/backup1')" --max_backup_bandwidth=1M > /dev/null -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT query_duration_ms >= 7e3, diff --git a/tests/queries/0_stateless/02724_limit_num_mutations.sh b/tests/queries/0_stateless/02724_limit_num_mutations.sh index 60888db0e2e..604cc9ff08e 100755 --- a/tests/queries/0_stateless/02724_limit_num_mutations.sh +++ b/tests/queries/0_stateless/02724_limit_num_mutations.sh @@ -23,7 +23,7 @@ function wait_for_alter() done } -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t_limit_mutations SYNC; CREATE TABLE t_limit_mutations (id UInt64, v UInt64) @@ -48,14 +48,14 @@ SELECT count() FROM system.mutations WHERE database = currentDatabase() AND tabl SHOW CREATE TABLE t_limit_mutations; " -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " ALTER TABLE t_limit_mutations UPDATE v = 6 WHERE 1 SETTINGS number_of_mutations_to_throw = 100; ALTER TABLE t_limit_mutations MODIFY COLUMN v String SETTINGS number_of_mutations_to_throw = 100, alter_sync = 0; " wait_for_alter "String" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t_limit_mutations ORDER BY id; SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_limit_mutations' AND NOT is_done; SHOW CREATE TABLE t_limit_mutations; @@ -65,7 +65,7 @@ ${CLICKHOUSE_CLIENT} --query "SYSTEM START MERGES t_limit_mutations" wait_for_mutation "t_limit_mutations" "0000000003" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t_limit_mutations ORDER BY id; SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_limit_mutations' AND NOT is_done; SHOW CREATE TABLE t_limit_mutations; diff --git a/tests/queries/0_stateless/02725_async_insert_table_setting.sh b/tests/queries/0_stateless/02725_async_insert_table_setting.sh index 13911e8d677..14c2d335275 100755 --- a/tests/queries/0_stateless/02725_async_insert_table_setting.sh +++ b/tests/queries/0_stateless/02725_async_insert_table_setting.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t_mt_async_insert; DROP TABLE IF EXISTS t_mt_sync_insert; @@ -19,7 +19,7 @@ url="${CLICKHOUSE_URL}&async_insert=0&wait_for_async_insert=1" ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_mt_async_insert VALUES (1, 'aa'), (2, 'bb')" ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_mt_sync_insert VALUES (1, 'aa'), (2, 'bb')" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT count() FROM t_mt_async_insert; SELECT count() FROM t_mt_sync_insert; diff --git a/tests/queries/0_stateless/02841_local_assert.sh b/tests/queries/0_stateless/02841_local_assert.sh index a167c09da1f..dc49007b0f6 100755 --- a/tests/queries/0_stateless/02841_local_assert.sh +++ b/tests/queries/0_stateless/02841_local_assert.sh @@ -7,12 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo "create table test (x UInt64) engine=Memory; -insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -nm +insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -m echo "create table test (x UInt64) engine=Memory; -insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -nm --ignore-error +insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -m --ignore-error echo "create table test (x UInt64) engine=Memory; insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS} -select 1" | $CLICKHOUSE_LOCAL -nm +select 1" | $CLICKHOUSE_LOCAL -m diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh index 4bc29ce4233..be0ef4e2648 100755 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm -q """ +${CLICKHOUSE_CLIENT} -m -q """ DROP TABLE IF EXISTS with_lonely; CREATE TABLE with_lonely @@ -23,7 +23,7 @@ ORDER BY (id); """ create_optimize_partition() { - ${CLICKHOUSE_CLIENT} -nm -q """ + ${CLICKHOUSE_CLIENT} -m -q """ INSERT INTO with_lonely SELECT number, '$1', number*10, 0 FROM numbers(10); INSERT INTO with_lonely SELECT number+500000, '$1', number*10, 1 FROM numbers(10); """ @@ -39,7 +39,7 @@ create_optimize_partition "2022-10-29" create_optimize_partition "2022-10-30" create_optimize_partition "2022-10-31" -${CLICKHOUSE_CLIENT} -nm -q """ +${CLICKHOUSE_CLIENT} -m -q """ SYSTEM STOP MERGES with_lonely; INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(10); diff --git a/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh b/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh index cc4ce9b122e..418e439e44b 100755 --- a/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh +++ b/tests/queries/0_stateless/02871_clickhouse_client_restart_pager.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # head by default print 10 rows, but it is not enough to query 11 rows, since # we need to overflow the default pipe size, hence just 1 million of rows (it # should be around 6 MiB in text representation, should be definitelly enough). -$CLICKHOUSE_CLIENT --ignore-error -nm --pager head -q " +$CLICKHOUSE_CLIENT --ignore-error -m --pager head -q " select * from numbers(1e6); -- { clientError CANNOT_WRITE_TO_FILE_DESCRIPTOR } select * from numbers(1e6); -- { clientError CANNOT_WRITE_TO_FILE_DESCRIPTOR } " diff --git a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh index 3f2b732e71b..3a7d861262e 100755 --- a/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh +++ b/tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "select 1; select 2;" $CLICKHOUSE_LOCAL -q "select 1; select 2;" # -n is a no-op -$CLICKHOUSE_CLIENT -n -q "select 1; select 2;" -$CLICKHOUSE_LOCAL -n -q "select 1; select 2;" +$CLICKHOUSE_CLIENT -q "select 1; select 2;" +$CLICKHOUSE_LOCAL -q "select 1; select 2;" exit 0 diff --git a/tests/queries/0_stateless/02875_merge_engine_set_index.sh b/tests/queries/0_stateless/02875_merge_engine_set_index.sh index 355d83167a6..f40696c31a9 100755 --- a/tests/queries/0_stateless/02875_merge_engine_set_index.sh +++ b/tests/queries/0_stateless/02875_merge_engine_set_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " CREATE TABLE t1 ( a UInt32, @@ -57,7 +57,7 @@ ORDER BY b DESC FORMAT Null;" -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectedMarks'] diff --git a/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh index 8bdaa47c111..dd08724456b 100755 --- a/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh +++ b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_LOCAL -q "select 42 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native -$CLICKHOUSE_LOCAL -n -q " +$CLICKHOUSE_LOCAL -q " create table test (x UInt64, y UInt64) engine=Memory; insert into test (x) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); insert into test (y) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); diff --git a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh index 315bbcd544f..c7270b65e19 100755 --- a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh +++ b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_LOCAL -q "select 1 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native -$CLICKHOUSE_LOCAL -n -q " +$CLICKHOUSE_LOCAL -q " create table test (x UInt64, y UInt64 default 42) engine=Memory; insert into test select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); select * from test; diff --git a/tests/queries/0_stateless/02883_named_collections_override.sh b/tests/queries/0_stateless/02883_named_collections_override.sh index a08c795127d..915ce280226 100755 --- a/tests/queries/0_stateless/02883_named_collections_override.sh +++ b/tests/queries/0_stateless/02883_named_collections_override.sh @@ -8,7 +8,7 @@ u1="${CLICKHOUSE_TEST_UNIQUE_NAME}_collection1" u2="${CLICKHOUSE_TEST_UNIQUE_NAME}_collection2" u3="${CLICKHOUSE_TEST_UNIQUE_NAME}_collection3" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP NAMED COLLECTION IF EXISTS $u1; DROP NAMED COLLECTION IF EXISTS $u2; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh index 7f583087336..791515c82d6 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_1; CREATE TABLE t_async_insert_native_1 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; " @@ -22,7 +22,7 @@ echo '{"id": 1, "s": "aaa"}' \ | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_1 FORMAT JSONEachRow {"id": 2, "s": "bbb"}' 2>&1 \ | grep -o "NOT_IMPLEMENTED" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT sum(length(entries.bytes)) FROM system.asynchronous_inserts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_1'; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh index b9b1854eaef..a8a9209ee68 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_2; CREATE TABLE t_async_insert_native_2 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; " @@ -18,7 +18,7 @@ echo "(3, 'ccc') (4, 'ddd') (5, 'eee')" | $CLICKHOUSE_CLIENT $async_insert_optio wait -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT * FROM t_async_insert_native_2 ORDER BY id; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh index c9d399607d0..229f13eb821 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_3; CREATE TABLE t_async_insert_native_3 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; " @@ -21,7 +21,7 @@ $CLICKHOUSE_CLIENT $async_insert_options -q "INSERT INTO t_async_insert_native_3 wait -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT format, length(entries.bytes) FROM system.asynchronous_inserts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_3' ORDER BY format; diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh index 9118c11315c..e84c1ca8899 100755 --- a/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_async_insert_native_4; CREATE TABLE t_async_insert_native_4 (id UInt64) ENGINE = MergeTree ORDER BY id; " @@ -20,7 +20,7 @@ echo "(2) (3) (4) (5)" | $CLICKHOUSE_CLIENT_WITH_LOG $async_insert_options --asy -q 'INSERT INTO t_async_insert_native_4 FORMAT Values' 2>&1 \ | grep -c "too much data" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " SELECT * FROM t_async_insert_native_4 ORDER BY id; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh b/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh index 2917ec86957..065658d4d56 100755 --- a/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh +++ b/tests/queries/0_stateless/02885_ephemeral_columns_from_file.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl', a $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl', auto, 'x UInt64 Alias y, y UInt64')" 2>&1 | grep -c "BAD_ARGUMENTS" $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl', auto, 'x UInt64 Materialized 42, y UInt64')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -n -q " +$CLICKHOUSE_LOCAL -q " create table test (x UInt64 Ephemeral, y UInt64 default x + 1) engine=Memory; insert into test (x, y) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.jsonl'); select * from test; diff --git a/tests/queries/0_stateless/02895_npy_output_format.sh b/tests/queries/0_stateless/02895_npy_output_format.sh index a364e447062..74000bc298f 100755 --- a/tests/queries/0_stateless/02895_npy_output_format.sh +++ b/tests/queries/0_stateless/02895_npy_output_format.sh @@ -9,7 +9,7 @@ mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* chmod 777 ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -${CLICKHOUSE_CLIENT} -n -q --ignore-error " +${CLICKHOUSE_CLIENT} -q --ignore-error " DROP DATABASE IF EXISTS npy_output_02895; CREATE DATABASE IF NOT EXISTS npy_output_02895; diff --git a/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh b/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh index d6775927f35..b4656c9e321 100755 --- a/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh +++ b/tests/queries/0_stateless/02895_peak_memory_usage_http_headers_regression.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS data; DROP TABLE IF EXISTS data2; DROP VIEW IF EXISTS mv1; diff --git a/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh b/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh index 64f5dd1a987..ef631d9ed1b 100755 --- a/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh +++ b/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh @@ -5,13 +5,13 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # setting disabled and no order by or primary key; expect error -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() SETTINGS index_granularity = 8192; " 2>&1 \ | grep -F -q "You must provide an ORDER BY or PRIMARY KEY expression in the table definition." && echo 'OK' || echo 'FAIL' # setting disabled and primary key in table definition -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() PRIMARY KEY a SETTINGS index_granularity = 8192; SHOW CREATE TABLE test_empty_order_by; diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh index 5065da371a8..b77e5b0b402 100755 --- a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh @@ -15,7 +15,7 @@ if [[ $($CLICKHOUSE_CLIENT -q "select count()>0 from system.clusters where clust cluster=test_cluster_database_replicated fi -$CLICKHOUSE_CLIENT -nm --distributed_ddl_output_mode=none -q " +$CLICKHOUSE_CLIENT -m --distributed_ddl_output_mode=none -q " drop table if exists rmt1; drop table if exists rmt2; @@ -46,7 +46,7 @@ part_name='%' # wait while there be at least one 'No active replica has part all_0_1_1 or covering part' in logs for _ in {0..50}; do - no_active_repilica_messages=$($CLICKHOUSE_CLIENT -nm -q " + no_active_repilica_messages=$($CLICKHOUSE_CLIENT -m -q " system flush logs; select count() @@ -65,7 +65,7 @@ for _ in {0..50}; do sleep 1 done -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system start pulling replication log rmt2; system flush logs; diff --git a/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh b/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh index 7ac9b488be5..5f9dc6ea077 100755 --- a/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh +++ b/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # setting enabled and no order by or primary key -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() SETTINGS index_granularity = 8192; @@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} -n --query=" " 2>&1 \ | grep -F -q "ORDER BY tuple()" && echo 'OK' || echo 'FAIL' # setting enabled and per-column primary key -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8 PRIMARY KEY, b String PRIMARY KEY) ENGINE = MergeTree() SETTINGS index_granularity = 8192; @@ -21,7 +21,7 @@ ${CLICKHOUSE_CLIENT} -n --query=" " 2>&1 \ | grep -F -q "ORDER BY (a, b)" && echo 'OK' || echo 'FAIL' # setting enabled and primary key in table definition (not per-column or order by) -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8, b String) ENGINE = MergeTree() PRIMARY KEY (a) SETTINGS index_granularity = 8192; @@ -29,7 +29,7 @@ ${CLICKHOUSE_CLIENT} -n --query=" " 2>&1 \ | grep -F -q "ORDER BY a" && echo 'OK' || echo 'FAIL' # setting enabled and order by in table definition (no primary key) -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET create_table_empty_primary_key_by_default = true; DROP TABLE IF EXISTS test_empty_order_by; CREATE TABLE test_empty_order_by(a UInt8, b String) ENGINE = MergeTree() ORDER BY (a, b) SETTINGS index_granularity = 8192; diff --git a/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh b/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh index 30ec50fa20f..e37f1e51c74 100755 --- a/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh +++ b/tests/queries/0_stateless/02907_backup_mv_with_no_inner_table.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists src; create table src (a Int32) engine = MergeTree() order by tuple(); @@ -15,14 +15,14 @@ create materialized view mv (a Int32) engine = MergeTree() order by tuple() as s uuid=$(${CLICKHOUSE_CLIENT} --query "select uuid from system.tables where table='mv' and database == currentDatabase()") inner_table=".inner_id.${uuid}" -${CLICKHOUSE_CLIENT} -nm --query "drop table \`$inner_table\` sync" +${CLICKHOUSE_CLIENT} -m --query "drop table \`$inner_table\` sync" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " set send_logs_level = 'error'; backup table ${CLICKHOUSE_DATABASE}.\`mv\` to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table mv; restore table ${CLICKHOUSE_DATABASE}.\`mv\` from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" diff --git a/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh b/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh index d59ebe400ee..f950954941f 100755 --- a/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh +++ b/tests/queries/0_stateless/02907_backup_mv_with_no_source_table.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists src; create table src (a Int32) engine = MergeTree() order by tuple(); @@ -15,18 +15,18 @@ drop table if exists mv; create materialized view mv to dst (a Int32) as select * from src; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table src; backup database ${CLICKHOUSE_DATABASE} on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table mv; set allow_deprecated_database_ordinary=1; restore table ${CLICKHOUSE_DATABASE}.mv on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists src; create table src (a Int32) engine = MergeTree() order by tuple(); @@ -37,13 +37,13 @@ drop table if exists mv; create materialized view mv to dst (a Int32) as select * from src; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table src; drop table dst; backup database ${CLICKHOUSE_DATABASE} on cluster test_shard_localhost to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}2'); " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table mv; set allow_deprecated_database_ordinary=1; restore table ${CLICKHOUSE_DATABASE}.mv on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}2'); diff --git a/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh b/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh index 8ed36a7edd7..dc5793d1638 100755 --- a/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh +++ b/tests/queries/0_stateless/02907_backup_restore_default_nullable.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test; set data_type_default_nullable = 0; create table test (test String) ENGINE = MergeTree() ORDER BY tuple(); @@ -13,7 +13,7 @@ backup table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost to Disk ${CLICKHOUSE_CLIENT} --query "show create table test" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table test sync; set data_type_default_nullable = 1; restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh index 742d24a97eb..eae307add10 100755 --- a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test; set flatten_nested = 0; create table test (test Array(Tuple(foo String, bar Float64))) ENGINE = MergeTree() ORDER BY tuple(); @@ -13,7 +13,7 @@ backup table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost to Disk ${CLICKHOUSE_CLIENT} --query "show create table test" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test2; set flatten_nested = 0; create table test2 (test Nested(foo String, bar Float64)) ENGINE = MergeTree() ORDER BY tuple(); @@ -22,7 +22,7 @@ backup table ${CLICKHOUSE_DATABASE}.test2 on cluster test_shard_localhost to Dis ${CLICKHOUSE_CLIENT} --query "show create table test2" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table test sync; set flatten_nested = 1; restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); @@ -30,7 +30,7 @@ restore table ${CLICKHOUSE_DATABASE}.test on cluster test_shard_localhost from D ${CLICKHOUSE_CLIENT} --query "show create table test" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table test2 sync; set flatten_nested = 1; restore table ${CLICKHOUSE_DATABASE}.test2 on cluster test_shard_localhost from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}2'); diff --git a/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh b/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh index 57182050534..2cad15c6fcb 100755 --- a/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh +++ b/tests/queries/0_stateless/02907_clickhouse_dictionary_bug.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP DICTIONARY IF EXISTS 02907_dictionary; DROP TABLE IF EXISTS 02907_table; diff --git a/tests/queries/0_stateless/02907_system_backups_profile_events.sh b/tests/queries/0_stateless/02907_system_backups_profile_events.sh index 801056a2844..9a1d5a3db11 100755 --- a/tests/queries/0_stateless/02907_system_backups_profile_events.sh +++ b/tests/queries/0_stateless/02907_system_backups_profile_events.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " drop table if exists test; create table test (a Int32) engine = MergeTree() order by tuple(); " @@ -12,10 +12,10 @@ create table test (a Int32) engine = MergeTree() order by tuple(); backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME} backup_name="Disk('backups', '$backup_id')"; -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " backup table ${CLICKHOUSE_DATABASE}.test to $backup_name; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " select ProfileEvents['BackupEntriesCollectorMicroseconds'] > 10 from system.backups where name='Disk(\'backups\', \'$backup_id\')' " diff --git a/tests/queries/0_stateless/02908_Npy_files_caching.sh b/tests/queries/0_stateless/02908_Npy_files_caching.sh index 4845f740972..218e13efb95 100755 --- a/tests/queries/0_stateless/02908_Npy_files_caching.sh +++ b/tests/queries/0_stateless/02908_Npy_files_caching.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=0" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy') settings optimize_count_from_files=1" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/one_dim.npy', auto, 'array Int64') settings optimize_count_from_files=1" -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " desc file('$CURDIR/data_npy/one_dim.npy'); select number_of_rows from system.schema_inference_cache where format='Npy'; " $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/npy_big.npy') settings optimize_count_from_files=0" $CLICKHOUSE_LOCAL -q "select count() from file('$CURDIR/data_npy/npy_big.npy') settings optimize_count_from_files=1" -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " desc file('$CURDIR/data_npy/npy_big.npy'); select number_of_rows from system.schema_inference_cache where format='Npy'; " diff --git a/tests/queries/0_stateless/02908_table_ttl_dependency.sh b/tests/queries/0_stateless/02908_table_ttl_dependency.sh index 70136b4a42b..0bc02426f61 100755 --- a/tests/queries/0_stateless/02908_table_ttl_dependency.sh +++ b/tests/queries/0_stateless/02908_table_ttl_dependency.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS 02908_dependent; DROP TABLE IF EXISTS 02908_main; @@ -14,11 +14,11 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = MergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM ${CLICKHOUSE_DATABASE}.02908_main); " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE 02908_main; " 2>&1 | grep -F -q "HAVE_DEPENDENT_OBJECTS" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE 02908_dependent; DROP TABLE 02908_main; " diff --git a/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh b/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh index 8da144f90ca..75d491642ea 100755 --- a/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh +++ b/tests/queries/0_stateless/02909_settings_in_json_schema_cache.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo '{"x" : 42}' > $CLICKHOUSE_TEST_UNIQUE_NAME.json -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS schema_inference_make_columns_nullable=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS schema_inference_make_columns_nullable=0; SELECT count() from system.schema_inference_cache where format = 'JSON' and additional_format_info like '%schema_inference_make_columns_nullable%';" diff --git a/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh b/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh index 80e38338751..7ad38e11e96 100755 --- a/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh +++ b/tests/queries/0_stateless/02915_input_table_function_in_subquery.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " CREATE TABLE IF NOT EXISTS ts_data_double_raw ( device_id UInt32 NOT NULL CODEC(ZSTD), diff --git a/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh b/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh index 5f0f41a956b..b6d6ca57768 100755 --- a/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh +++ b/tests/queries/0_stateless/02915_lazy_loading_of_base_backups.sh @@ -13,40 +13,40 @@ b_backup="Disk('backups', '$b_backup_id')" c_backup_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_c c_backup="Disk('backups', '$c_backup_id')" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE IF EXISTS tbl1; DROP TABLE IF EXISTS tbl2; DROP TABLE IF EXISTS tbl3; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE TABLE tbl1 (a Int32) ENGINE = MergeTree() ORDER BY tuple(); " # The following BACKUP command must write backup 'a'. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $a_backup SETTINGS id='$a_backup_id'; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE TABLE tbl2 (a Int32) ENGINE = MergeTree() ORDER BY tuple(); " # The following BACKUP command must read backup 'a' and write backup 'b'. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $b_backup SETTINGS id='$b_backup_id', base_backup=$a_backup; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE TABLE tbl3 (a Int32) ENGINE = MergeTree() ORDER BY tuple(); " # The following BACKUP command must read only backup 'b' (and not 'a') and write backup 'c'. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " BACKUP DATABASE ${CLICKHOUSE_DATABASE} TO $c_backup SETTINGS id='$c_backup_id', base_backup=$b_backup; " | grep -o "BACKUP_CREATED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE tbl1; DROP TABLE tbl2; DROP TABLE tbl3; @@ -57,28 +57,28 @@ r2_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r2 r3_restore_id=${CLICKHOUSE_TEST_UNIQUE_NAME}_r3 # The following RESTORE command must read all 3 backups 'a', 'b', c' because the table 'tbl1' was in the first backup. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl1 FROM $c_backup SETTINGS id='$r1_restore_id'; " | grep -o "RESTORED" # The following RESTORE command must read only 2 backups 'b', c' (and not 'a') because the table 'tbl2' was in the second backup. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl2 FROM $c_backup SETTINGS id='$r2_restore_id'; " | grep -o "RESTORED" # The following RESTORE command must read only 1 backup 'c' (and not 'a' or 'b') because the table 'tbl3' was in the third backup. -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " RESTORE TABLE ${CLICKHOUSE_DATABASE}.tbl3 FROM $c_backup SETTINGS id='$r3_restore_id'; " | grep -o "RESTORED" all_ids="['$a_backup_id', '$b_backup_id', '$c_backup_id', '$r1_restore_id', '$r2_restore_id', '$r3_restore_id']" id_prefix_len=`expr "${CLICKHOUSE_TEST_UNIQUE_NAME}_" : '.*'` -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " SELECT substr(id, 1 + $id_prefix_len) as short_id, ProfileEvents['BackupsOpenedForRead'], ProfileEvents['BackupsOpenedForWrite'] FROM system.backups WHERE id IN ${all_ids} ORDER BY short_id " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE tbl1; DROP TABLE tbl2; DROP TABLE tbl3; diff --git a/tests/queries/0_stateless/02916_dictionary_access.sh b/tests/queries/0_stateless/02916_dictionary_access.sh index 08ee517ab3b..be62cc027ef 100755 --- a/tests/queries/0_stateless/02916_dictionary_access.sh +++ b/tests/queries/0_stateless/02916_dictionary_access.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE DICTIONARY IF NOT EXISTS ${dictname} ( id UInt64, @@ -23,15 +23,15 @@ ${CLICKHOUSE_CLIENT} -nm --query " SELECT dictGet(${dictname}, 'value', 1); " -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT * FROM dictionary(${dictname}); " 2>&1 | grep -o ACCESS_DENIED | uniq -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT dictGet(${dictname}, 'value', 1); " 2>&1 | grep -o ACCESS_DENIED | uniq -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP DICTIONARY IF EXISTS ${dictname}; DROP USER IF EXISTS ${username}; " diff --git a/tests/queries/0_stateless/02916_joinget_dependency.sh b/tests/queries/0_stateless/02916_joinget_dependency.sh index 6477ae8c967..ff9332cb57f 100755 --- a/tests/queries/0_stateless/02916_joinget_dependency.sh +++ b/tests/queries/0_stateless/02916_joinget_dependency.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We test the dependency on the DROP -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS Sub_distributed; DROP TABLE IF EXISTS Sub; DROP TABLE IF EXISTS Mapping; @@ -20,8 +20,8 @@ $CLICKHOUSE_CLIENT -q " DROP TABLE Mapping; " 2>&1 | grep -cm1 "HAVE_DEPENDENT_OBJECTS" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE Sub_distributed; DROP TABLE Sub; DROP TABLE Mapping; -" \ No newline at end of file +" diff --git a/tests/queries/0_stateless/02930_client_file_log_comment.sh b/tests/queries/0_stateless/02930_client_file_log_comment.sh index 50cd587e4b5..393bffcaf59 100755 --- a/tests/queries/0_stateless/02930_client_file_log_comment.sh +++ b/tests/queries/0_stateless/02930_client_file_log_comment.sh @@ -14,7 +14,7 @@ echo -n 'select 4242' >> "$file2" $CLICKHOUSE_CLIENT --queries-file "$file1" "$file2" <<<'select 42' $CLICKHOUSE_CLIENT --log_comment foo --queries-file /dev/stdin <<<'select 424242' -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system flush logs; select query, log_comment from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and event_date >= yesterday() and query = 'select 42' and type != 'QueryStart'; select query, log_comment from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and event_date >= yesterday() and query = 'select 4242' and type != 'QueryStart'; diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh index d9e4a2c8f8b..76ab56a4570 100755 --- a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh +++ b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " create table test (x UInt64, y UInt32, size UInt64) engine=Memory; insert into test select c1, c2, _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; select * from test; diff --git a/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh b/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh index 0e23bb6c42b..9d1faf301d3 100755 --- a/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh +++ b/tests/queries/0_stateless/02940_system_stacktrace_optimizations.sh @@ -9,12 +9,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # no message at all echo "thread = 0" -$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_id = 0" |& grep -F -o 'Send signal to' +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -m -q "select * from system.stack_trace where thread_id = 0" |& grep -F -o 'Send signal to' # send messages to some threads echo "thread != 0" -$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_id != 0 format Null" |& grep -F -o 'Send signal to' | grep -v 'Send signal to 0 threads (total)' +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -m -q "select * from system.stack_trace where thread_id != 0 format Null" |& grep -F -o 'Send signal to' | grep -v 'Send signal to 0 threads (total)' # there is no thread with comm="foo", so no signals will be sent echo "thread_name = 'foo'" -$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -nm -q "select * from system.stack_trace where thread_name = 'foo' format Null" |& grep -F -o 'Send signal to 0 threads (total)' +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level=test -m -q "select * from system.stack_trace where thread_name = 'foo' format Null" |& grep -F -o 'Send signal to 0 threads (total)' diff --git a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh index 27950866e81..44af2dbf26f 100755 --- a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh +++ b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh @@ -26,7 +26,7 @@ function wait_part() function restore_failpoints() { # restore entry error with failpoints (to avoid endless errors in logs) - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " system enable failpoint replicated_queue_unfail_entries; system sync replica $failed_replica; system disable failpoint replicated_queue_unfail_entries; @@ -34,7 +34,7 @@ function restore_failpoints() } trap restore_failpoints EXIT -$CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " +$CLICKHOUSE_CLIENT -m --insert_keeper_fault_injection_probability=0 -q " drop table if exists data_r1; drop table if exists data_r2; @@ -45,7 +45,7 @@ $CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " " # will fail ALTER_METADATA on one of replicas -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system enable failpoint replicated_queue_fail_next_entry; alter table data_r1 drop index value_idx settings alter_sync=0; -- part all_0_0_0_1 @@ -80,7 +80,7 @@ fi # This will create MERGE_PARTS, on failed replica it will be fetched from source replica (since it does not have all parts to execute merge) $CLICKHOUSE_CLIENT -q "optimize table $success_replica final settings optimize_throw_if_noop=1, alter_sync=1" # part all_0_0_1_1 -$CLICKHOUSE_CLIENT -nm --insert_keeper_fault_injection_probability=0 -q " +$CLICKHOUSE_CLIENT -m --insert_keeper_fault_injection_probability=0 -q " insert into $success_replica (key) values (2); -- part all_2_2_0 -- Avoid 'Cannot select parts for optimization: Entry for part all_2_2_0 hasn't been read from the replication log yet' system sync replica $success_replica pull; diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh b/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh index 6cb184cb1fe..ec699d974d4 100755 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_3.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) user_name="${CLICKHOUSE_DATABASE}_test_user_02947" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_merge_tree_index; DROP USER IF EXISTS $user_name; @@ -44,7 +44,7 @@ $CLICKHOUSE_CLIENT --user "$user_name" --password "password" -q "SELECT arr.size $CLICKHOUSE_CLIENT --user "$user_name" --password "password" -q "SELECT b FROM mergeTreeIndex(currentDatabase(), t_merge_tree_index, with_marks = true)" 2>&1 | grep -m1 -o "ACCESS_DENIED" || echo "OK" $CLICKHOUSE_CLIENT --user "$user_name" --password "password" -q "SELECT b.mark FROM mergeTreeIndex(currentDatabase(), t_merge_tree_index, with_marks = true)" 2>&1 | grep -m1 -o "ACCESS_DENIED" || echo "OK" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS t_merge_tree_index; DROP USER IF EXISTS $user_name; " diff --git a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh index daa9c571a5d..d06aba8a4b6 100755 --- a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh +++ b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE source_table ( id UInt64, diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh index 7f690a681c4..737d5c6b41e 100755 --- a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # CREATE TABLE local (x UInt8) Engine=Memory; # CREATE TABLE distributed ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), x) -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS local; DROP TABLE IF EXISTS distributed; CREATE TABLE local (x UInt8) Engine=Memory; diff --git a/tests/queries/0_stateless/02974_backup_query_format_null.sh b/tests/queries/0_stateless/02974_backup_query_format_null.sh index ddba2f6de16..345a4f47b20 100755 --- a/tests/queries/0_stateless/02974_backup_query_format_null.sh +++ b/tests/queries/0_stateless/02974_backup_query_format_null.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE IF EXISTS tbl; CREATE TABLE tbl (a Int32) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO tbl VALUES (2), (80), (-12345); @@ -14,7 +14,7 @@ backup_name="Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}')" ${CLICKHOUSE_CLIENT} --query "BACKUP TABLE tbl TO ${backup_name} FORMAT Null" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE tbl; RESTORE ALL FROM ${backup_name} FORMAT Null " diff --git a/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh b/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh index e65c9654c9c..7ad5a2179f9 100755 --- a/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh +++ b/tests/queries/0_stateless/02981_insert_select_resize_to_max_insert_threads.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq """ +${CLICKHOUSE_CLIENT} -q """ CREATE TABLE t1_local ( n UInt64, diff --git a/tests/queries/0_stateless/03008_deduplication_random_setttings.sh b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh index e9f59138177..07b99eb4e99 100755 --- a/tests/queries/0_stateless/03008_deduplication_random_setttings.sh +++ b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh @@ -35,7 +35,7 @@ THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" -$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -mq " $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ --insert-method $insert_method \ --table-engine $engine \ @@ -48,7 +48,7 @@ $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " ) " 1>/dev/null 2>&1 && echo 'insert_several_blocks_into_table OK' || echo "FAIL: insert_several_blocks_into_table ${THIS_RUN}" -$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -mq " $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ --insert-method $insert_method \ --table-engine $engine \ @@ -61,7 +61,7 @@ $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " ) " 1>/dev/null 2>&1 && echo 'mv_generates_several_blocks OK' || echo "FAIL: mv_generates_several_blocks ${THIS_RUN}" -$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -mq " $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ --insert-method $insert_method \ --table-engine $engine \ diff --git a/tests/queries/0_stateless/03008_local_plain_rewritable.sh b/tests/queries/0_stateless/03008_local_plain_rewritable.sh index d51e180efc9..e61f9061297 100755 --- a/tests/queries/0_stateless/03008_local_plain_rewritable.sh +++ b/tests/queries/0_stateless/03008_local_plain_rewritable.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query "drop table if exists 03008_test_local_mt sync" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " create table 03008_test_local_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) settings disk = disk( @@ -19,35 +19,35 @@ settings disk = disk( path = '/var/lib/clickhouse/disks/local_plain_rewritable/') " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " insert into 03008_test_local_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5); insert into 03008_test_local_mt (*) select number, number, number from numbers_mt(10000); " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " select count(*) from 03008_test_local_mt; select (*) from 03008_test_local_mt order by tuple(a, b) limit 10; " ${CLICKHOUSE_CLIENT} --query "optimize table 03008_test_local_mt final;" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " alter table 03008_test_local_mt modify setting disk = '03008_local_plain_rewritable', old_parts_lifetime = 3600; select engine_full from system.tables WHERE database = currentDatabase() AND name = '03008_test_local_mt'; " | grep -c "old_parts_lifetime = 3600" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " select count(*) from 03008_test_local_mt; select (*) from 03008_test_local_mt order by tuple(a, b) limit 10; " -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " alter table 03008_test_local_mt update c = 0 where a % 2 = 1; alter table 03008_test_local_mt add column d Int64 after c; alter table 03008_test_local_mt drop column c; " 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " truncate table 03008_test_local_mt; select count(*) from 03008_test_local_mt; " diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.sh b/tests/queries/0_stateless/03031_clickhouse_local_input.sh index 6f59e9b9703..e2f9cf48108 100755 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.sh +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.sh @@ -6,15 +6,15 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) tmp_file="$CUR_DIR/$CLICKHOUSE_DATABASE.txt" echo '# foo' -$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -n -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') format LineAsString" << "$INPUT_FILE" -$CLICKHOUSE_CLIENT --external --file="$INPUT_FILE" --name=t --structure='x String' -nm -q " +$CLICKHOUSE_CLIENT --external --file="$INPUT_FILE" --name=t --structure='x String' -m -q " select * from t; select * from t; " diff --git a/tests/queries/0_stateless/03143_prewhere_profile_events.sh b/tests/queries/0_stateless/03143_prewhere_profile_events.sh index 00daa0fe7cc..6a6b993e5f8 100755 --- a/tests/queries/0_stateless/03143_prewhere_profile_events.sh +++ b/tests/queries/0_stateless/03143_prewhere_profile_events.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS t; CREATE TABLE t(a UInt32, b UInt32, c UInt32, d UInt32) ENGINE=MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part=0, min_rows_for_wide_part=0; @@ -25,7 +25,7 @@ client_opts=( --max_threads 8 ) -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_1" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_1" -q " SELECT * FROM t PREWHERE (b % 8192) = 42 @@ -33,7 +33,7 @@ PREWHERE (b % 8192) = 42 FORMAT Null " -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_2" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_2" -q " SELECT * FROM t PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 @@ -42,7 +42,7 @@ PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 settings enable_multiple_prewhere_read_steps=1; " -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_3" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_3" -q " SELECT * FROM t PREWHERE (b % 8192) = 42 AND (c % 16384) = 42 @@ -51,7 +51,7 @@ PREWHERE (b % 8192) = 42 AND (c % 16384) = 42 settings enable_multiple_prewhere_read_steps=0; " -${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_4" -nq " +${CLICKHOUSE_CLIENT} "${client_opts[@]}" --query_id "$query_id_4" -q " SELECT b, c FROM t PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 @@ -59,7 +59,7 @@ PREWHERE (b % 8192) = 42 AND (c % 8192) = 42 settings enable_multiple_prewhere_read_steps=1; " -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; -- 52503 which is 43 * number of granules, 10000000 diff --git a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh index 95aef9bbc5b..4e7b318e202 100755 --- a/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh +++ b/tests/queries/0_stateless/03145_non_loaded_projection_backup.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists tp_1; create table tp_1 (x Int32, y Int32, projection p (select x, y order by x)) engine = MergeTree order by y partition by intDiv(y, 100) settings max_parts_to_merge_at_once=1; insert into tp_1 select number, number from numbers(3); @@ -25,7 +25,7 @@ alter table tp_1 drop projection pp; alter table tp_1 attach partition '0'; " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found unexpected projection directories: pp.proj" @@ -34,19 +34,19 @@ $CLICKHOUSE_CLIENT -q " backup table tp_1 to Disk('backups', '$backup_id'); " | grep -o "BACKUP_CREATED" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; drop table tp_1; restore table tp_1 from Disk('backups', '$backup_id'); " | grep -o "RESTORED" $CLICKHOUSE_CLIENT -q "select count() from tp_1;" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; check table tp_1 settings check_query_single_value_result = 0;" | grep -o "Found unexpected projection directories: pp.proj" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; check table tp_1" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " set send_logs_level='fatal'; drop table tp_1" diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh index b6980b3a23a..f33a6b3ef27 100755 --- a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " CREATE TABLE event_envoy ( timestamp_interval DateTime CODEC(DoubleDelta), @@ -18,7 +18,7 @@ ${CLICKHOUSE_CLIENT} -nq " INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5); " -${CLICKHOUSE_CLIENT} -nq " +${CLICKHOUSE_CLIENT} -q " CREATE TABLE event_envoy_remote ( timestamp_interval DateTime CODEC(DoubleDelta), diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.sh b/tests/queries/0_stateless/03156_default_multiquery_split.sh index 8ba2f46b786..d849fb5a162 100755 --- a/tests/queries/0_stateless/03156_default_multiquery_split.sh +++ b/tests/queries/0_stateless/03156_default_multiquery_split.sh @@ -53,6 +53,6 @@ SELECT * FROM TEST2 ORDER BY value; DROP TABLE TEST1; DROP TABLE TEST2; EOF -$CLICKHOUSE_CLIENT -m -n < "$SQL_FILE_NAME" +$CLICKHOUSE_CLIENT -m < "$SQL_FILE_NAME" rm "$SQL_FILE_NAME" diff --git a/tests/queries/0_stateless/03169_time_virtual_column.sh b/tests/queries/0_stateless/03169_time_virtual_column.sh index fef1de8c6f2..b289f39accb 100755 --- a/tests/queries/0_stateless/03169_time_virtual_column.sh +++ b/tests/queries/0_stateless/03169_time_virtual_column.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv sleep 1 -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " select _size, (dateDiff('millisecond', _time, now()) < 600000 AND dateDiff('millisecond', _time, now()) > 0) from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv'); " rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index 7c567c0f58f..af702569794 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); diff --git a/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh index ce53f467823..583257d8fd3 100755 --- a/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh +++ b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh @@ -5,25 +5,25 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo -e 'a,b,c\n1,2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_skip_first_lines=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_skip_first_lines=0; SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%skip_first_lines%';" echo -e 'a,b,c\n"1",2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_numbers_from_strings=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_numbers_from_strings=0; SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%try_infer_numbers_from_strings%';" echo -e 'a,b,c\n"(1,2,3)",2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_strings_from_quoted_tuples=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_strings_from_quoted_tuples=0; SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%try_infer_strings_from_quoted_tuples%';" echo -e 'a\tb\tc\n1\t2\t3' > $CLICKHOUSE_TEST_UNIQUE_NAME.tsv -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.tsv') SETTINGS input_format_tsv_skip_first_lines=1; DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.tsv') SETTINGS input_format_tsv_skip_first_lines=0; SELECT count() from system.schema_inference_cache where format = 'TSV' and additional_format_info like '%skip_first_lines%';" diff --git a/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh b/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh index 4f217935123..c759cc34425 100755 --- a/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh +++ b/tests/queries/0_stateless/03198_unload_primary_key_outdated.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n " +$CLICKHOUSE_CLIENT " DROP TABLE IF EXISTS t_unload_primary_key; CREATE TABLE t_unload_primary_key (a UInt64, b UInt64) @@ -26,7 +26,7 @@ for _ in {1..100}; do sleep 0.3 done -$CLICKHOUSE_CLIENT -n " +$CLICKHOUSE_CLIENT " SELECT name, active, primary_key_bytes_in_memory FROM system.parts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_unload_primary_key' ORDER BY name; DROP TABLE IF EXISTS t_unload_primary_key; " diff --git a/tests/queries/0_stateless/03199_dictionary_table_access.sh b/tests/queries/0_stateless/03199_dictionary_table_access.sh index 952b466b5da..14f017c7fbc 100755 --- a/tests/queries/0_stateless/03199_dictionary_table_access.sh +++ b/tests/queries/0_stateless/03199_dictionary_table_access.sh @@ -8,7 +8,7 @@ username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" dicttablename="dict_table_${CLICKHOUSE_TEST_UNIQUE_NAME}" -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " CREATE DICTIONARY IF NOT EXISTS ${dictname} ( id UInt64, @@ -26,15 +26,15 @@ ${CLICKHOUSE_CLIENT} -nm --query " SELECT * FROM ${dicttablename}; " -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT * FROM ${dictname}; " 2>&1 | grep -o ACCESS_DENIED | uniq -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " +$CLICKHOUSE_CLIENT -m --user="${username}" --query " SELECT * FROM ${dicttablename}; " 2>&1 | grep -o ACCESS_DENIED | uniq -${CLICKHOUSE_CLIENT} -nm --query " +${CLICKHOUSE_CLIENT} -m --query " DROP TABLE IF EXISTS ${dicttablename} SYNC; DROP DICTIONARY IF EXISTS ${dictname}; DROP USER IF EXISTS ${username}; diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh index 8a77538f592..adbb0cb6de0 100755 --- a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh echo '{"x" : 42}' > $CLICKHOUSE_TEST_UNIQUE_NAME.json -$CLICKHOUSE_LOCAL -nm -q " +$CLICKHOUSE_LOCAL -m -q " DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.json') SETTINGS input_format_max_bytes_to_read_for_schema_inference=1000; SELECT additional_format_info from system.schema_inference_cache" From 5c54c7025bd87f1e4239354b5f3c0adff188dd3a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 12 Aug 2024 08:25:54 +0000 Subject: [PATCH 0772/1722] Followup for #56996 --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 4 ++-- src/Interpreters/HashJoin/JoinFeatures.h | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index aedd24630d1..39ba9fc6e93 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -354,8 +354,8 @@ size_t HashJoinMethods::joinRightColumns( { if (unlikely(current_offset >= max_joined_block_rows)) { - added_columns.offsets_to_replicate->resize_assume_reserved(i); - added_columns.filter.resize_assume_reserved(i); + added_columns.offsets_to_replicate->resize(i); + added_columns.filter.resize(i); break; } } diff --git a/src/Interpreters/HashJoin/JoinFeatures.h b/src/Interpreters/HashJoin/JoinFeatures.h index b8de606c51e..b39593e7cac 100644 --- a/src/Interpreters/HashJoin/JoinFeatures.h +++ b/src/Interpreters/HashJoin/JoinFeatures.h @@ -18,11 +18,25 @@ struct JoinFeatures static constexpr bool inner = KIND == JoinKind::Inner; static constexpr bool full = KIND == JoinKind::Full; + /** Whether we may need duplicate rows from the left table. + * For example, when we have row (key1, attr1) in left table + * and rows (key1, attr2), (key1, attr3) in right table, + * then we need to duplicate row (key1, attr1) for each of joined rows from right table, so result will be + * (key1, attr1, key1, attr2) + * (key1, attr1, key1, attr3) + */ static constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); + + /// Whether we need to filter rows from the left table that do not have matches in the right table. static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); + + /// Whether we need to add default values for columns from the left table. static constexpr bool add_missing = (left || full) && !is_semi_join; + /// Whether we need to store flags for rows from the right table table + /// that indicates if they have matches in the left table. static constexpr bool need_flags = MapGetter, HashJoin::MapsAll>>::flagged; + static constexpr bool is_maps_all = std::is_same_v, HashJoin::MapsAll>; }; From bc5d7933824d535bfbf2c672e5d7405c8bde86c8 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 09:13:24 +0000 Subject: [PATCH 0773/1722] Replace asyncronouos buffer with syncronouos --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/IO/createReadBufferFromFileBase.cpp | 9 --------- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 11 +++-------- src/IO/ReadBuffer.h | 8 +------- 4 files changed, 4 insertions(+), 25 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 80dcc1baa14..01eb3d86696 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -231,7 +231,6 @@ void ReadBufferFromRemoteFSGather::reset() { current_object = StoredObject(); current_buf_idx = {}; - // buffer_cemetery_.push_back(current_buf); current_buf.reset(); } diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index c6152543a41..76588bcb5b1 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -78,7 +78,6 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 1"); res = std::make_unique( filename, buffer_size, @@ -90,8 +89,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 2"); - res = std::make_unique( filename, buffer_size, @@ -103,7 +100,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 3"); #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); @@ -123,8 +119,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 4"); - auto & reader = getThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -139,7 +133,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 5"); auto & reader = getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -154,8 +147,6 @@ std::unique_ptr createReadBufferFromFileBase( } else { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 6"); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); } return res; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index d291dcd65cf..3b650adb71f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -43,18 +43,13 @@ bool LocalObjectStorage::exists(const StoredObject & object) const std::unique_ptr LocalObjectStorage::readObjects( /// NOLINT const StoredObjects & objects, const ReadSettings & read_settings, - std::optional read_hint, - std::optional file_size) const + std::optional, + std::optional) const { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr - { - LOG_DEBUG(&Poco::Logger::get("Get object path"), "Remote Path: {}", object.remote_path); - auto kek = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); - LOG_DEBUG(&Poco::Logger::get("Buffer created"), "Remote Path: {}", object.remote_path); - return kek; - }; + { return std::make_unique(object.remote_path); }; return std::make_unique( std::move(read_buffer_creator), diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index ea41aab84f0..98f0c8c9059 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -101,11 +101,7 @@ public: * * Try to read after the end should throw an exception. */ - bool ALWAYS_INLINE eof() - { - LOG_DEBUG(); - return !hasPendingData() && !next(); - } + bool ALWAYS_INLINE eof() { return !hasPendingData() && !next(); } void ignore() { @@ -183,8 +179,6 @@ public: while (bytes_copied < n && !eof()) { - auto k = *pos; - LOG_DEBUG(&Poco::Logger::get("Next symbol in read"), "Symbol: {}", k); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); ::memcpy(to + bytes_copied, pos, bytes_to_copy); pos += bytes_to_copy; From 8f124710ef97801f020fb88e31bcd94529a112fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 10:27:21 +0000 Subject: [PATCH 0774/1722] Remove Log engine from Kafka integration tests It doesn't work well when `thread_per_consumer` is used as writer can make readers starve when `shared_time_mutex` prefers writes over reads. --- tests/integration/test_storage_kafka/test.py | 26 ++++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 4b6c9922d74..52d6054c12a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1019,7 +1019,7 @@ def test_kafka_formats(kafka_cluster, create_query_generator): DROP TABLE IF EXISTS test.kafka_{format_name}_mv; - CREATE MATERIALIZED VIEW test.kafka_{format_name}_mv Engine=Log AS + CREATE MATERIALIZED VIEW test.kafka_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS SELECT *, _topic, _partition, _offset FROM test.kafka_{format_name}; """.format( topic_name=topic_name, @@ -2460,7 +2460,7 @@ def test_kafka_commit_on_block_write(kafka_cluster, create_query_generator): (generate_old_create_table_query, "kafka.*Committed offset 2.*virt2_[01]"), ( generate_new_create_table_query, - r"kafka.*Saved offset 2[0-9]* for topic-partition \[virt2_[01]:[0-9]+", + r"kafka.*Saved offset 2 for topic-partition \[virt2_[01]:[0-9]+", ), ], ) @@ -2494,7 +2494,7 @@ def test_kafka_virtual_columns2(kafka_cluster, create_query_generator, log_line) f""" {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; """ ) @@ -2729,7 +2729,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster, create_query_generator, log_ DROP TABLE IF EXISTS test.consumer; {writer_create_query}; {reader_create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; """ ) @@ -2865,7 +2865,7 @@ def test_kafka_produce_consume_avro(kafka_cluster, create_query_generator): {writer_create_query}; {reader_create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() AS SELECT key, value FROM test.kafka; """ ) @@ -3537,7 +3537,7 @@ def test_bad_reschedule(kafka_cluster, create_query_generator): f""" {create_query}; - CREATE MATERIALIZED VIEW test.destination Engine=Log AS + CREATE MATERIALIZED VIEW test.destination ENGINE=MergeTree ORDER BY tuple() AS SELECT key, now() as consume_ts, @@ -3745,7 +3745,7 @@ def test_kafka_unavailable(kafka_cluster, create_query_generator, do_direct_read f""" {create_query}; - CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_unavailable ENGINE=MergeTree ORDER BY tuple() AS SELECT key, now() as consume_ts, @@ -4267,12 +4267,12 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator {create_query}; DROP TABLE IF EXISTS test.kafka_data_{format_name}_mv; - CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv Engine=Log AS + CREATE MATERIALIZED VIEW test.kafka_data_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS SELECT *, _topic, _partition, _offset FROM test.kafka_{format_name} WHERE length(_error) = 0; DROP TABLE IF EXISTS test.kafka_errors_{format_name}_mv; - CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS + CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv ENGINE=MergeTree ORDER BY tuple() AS SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} WHERE length(_error) > 0; """ @@ -4796,7 +4796,7 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator): DROP TABLE IF EXISTS test.kafka; {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.kafka; """ ) @@ -4875,7 +4875,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); @@ -4982,7 +4982,7 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; @@ -5362,7 +5362,7 @@ def test_formats_errors(kafka_cluster): input_format_with_names_use_header=0, format_schema='key_value_message:Message'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS SELECT key, value FROM test.{table_name}; """ ) From 06ceaee50218507f49bfc714903240ad4b5d81a0 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 11:09:45 +0000 Subject: [PATCH 0775/1722] Fix test 01903_correct_block_size_prediction_with_default - Don't allow random settings that affect the memory usage - Run two queries and compare the memory usage, rather than having an arbitrary hardcoded value --- ...ock_size_prediction_with_default.reference | 6 ++++ ...rect_block_size_prediction_with_default.sh | 36 +++++++++++++++++++ ...ect_block_size_prediction_with_default.sql | 13 ------- 3 files changed, 42 insertions(+), 13 deletions(-) create mode 100755 tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh delete mode 100644 tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference index b70a1cb7c75..2c66db91737 100644 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.reference @@ -1,3 +1,9 @@ 8 +8 +1 4 4 +1 +4 +4 +1 diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh new file mode 100755 index 00000000000..922dcb957e5 --- /dev/null +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-random-merge-tree-settings, no-random-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +sql="toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1]))" + +# Create the table and fill it +$CLICKHOUSE_CLIENT -n --query=" + CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=(\"[^\"]*?\"|[^\",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple(); + INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000); + ALTER TABLE test_extract ADD COLUMN 15Id Nullable(UInt16) DEFAULT $sql;" + +function test() +{ + # Execute two queries and compare if they have similar memory usage: + # The first query uses the default column value, while the second explicitly uses the same SQL as the default value. + # Follow https://github.com/ClickHouse/ClickHouse/issues/17317 for more info about the issue + where=$1 + + uuid_1=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + $CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1 + uuid_2=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 + $CLICKHOUSE_CLIENT -n --query=" + SYSTEM FLUSH LOGS; + WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), + memory_2 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" +} + +test "" +test "PREWHERE 15Id < 4" +test "WHERE 15Id < 4" diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql deleted file mode 100644 index 2eec08635eb..00000000000 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sql +++ /dev/null @@ -1,13 +0,0 @@ --- Tags: no-random-merge-tree-settings - -CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=("[^"]*?"|[^",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple(); - -INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000); - -ALTER TABLE test_extract ADD COLUMN `15Id` Nullable(UInt16) DEFAULT toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1])); - -SELECT uniq(15Id) FROM test_extract SETTINGS max_threads=1, max_memory_usage=100000000; - -SELECT uniq(15Id) FROM test_extract PREWHERE 15Id < 4 SETTINGS max_threads=1, max_memory_usage=100000000; - -SELECT uniq(15Id) FROM test_extract WHERE 15Id < 4 SETTINGS max_threads=1, max_memory_usage=100000000; From 38f3131e11d1d777101a975361eb585fd6263300 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 11:39:36 +0000 Subject: [PATCH 0776/1722] Fix review comments --- src/Columns/ColumnDynamic.cpp | 43 +++++++++++-------- src/Columns/ColumnDynamic.h | 12 ++++++ src/DataTypes/DataTypeDynamic.cpp | 4 +- .../Serializations/SerializationDynamic.cpp | 2 +- src/Functions/FunctionsConversion.cpp | 2 +- ..._read_shared_subcolumns_small.reference.j2 | 3 ++ 6 files changed, 43 insertions(+), 23 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 7246be29592..b1d28342a28 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -115,8 +115,8 @@ bool ColumnDynamic::addNewVariant(const DataTypePtr & new_variant, const String if (variant_info.variant_name_to_discriminator.contains(new_variant_name)) return true; - /// Check if we reached maximum number of variants (don't count shared variant). - if (variant_info.variant_names.size() - 1 == max_dynamic_types) + /// Check if we reached maximum number of variants. + if (!canAddNewVariant()) { /// Dynamic column should always have shared variant. if (!variant_info.variant_name_to_discriminator.contains(getSharedVariantTypeName())) @@ -194,8 +194,8 @@ std::vector * ColumnDynamic::combineVariants(const { const DataTypes & current_variants = assert_cast(*variant_info.variant_type).getVariants(); - /// We cannot combine Variants if total number of variants exceeds max_dynamic_types (don't count shared variant). - if (current_variants.size() + num_new_variants - 1 > max_dynamic_types) + /// We cannot combine Variants if total number of variants exceeds max_dynamic_types. + if (!canAddNewVariants(num_new_variants)) { /// Remember that we cannot combine our variant with this one, so we will not try to do it again. variants_with_failed_combination.insert(other_variant_info.variant_name); @@ -403,11 +403,11 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t auto shared_variant_discr = getSharedVariantDiscriminator(); variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length, *global_discriminators_mapping, shared_variant_discr); - /// We should process insertion from srs shared variant separately, because it can contain + /// We should process insertion from src shared variant separately, because it can contain /// values that should be extracted into our variants. insertRangeFrom above didn't insert /// values into our shared variant (we specified shared_variant_discr as special skip discriminator). - /// Check if srs shared variant is empty, nothing to do in this case. + /// Check if src shared variant is empty, nothing to do in this case. if (dynamic_src.getSharedVariant().empty()) return; @@ -466,7 +466,7 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t other_to_new_discriminators.reserve(dynamic_src.variant_info.variant_names.size()); /// Check if we cannot add any more new variants. In this case we will insert all new variants into shared variant. - if (variant_info.variant_names.size() - 1 == max_dynamic_types) + if (!canAddNewVariant()) { auto shared_variant_discr = getSharedVariantDiscriminator(); for (const auto & variant_name : dynamic_src.variant_info.variant_names) @@ -496,7 +496,7 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t /// Add new variants from sorted list until we reach max_dynamic_types. for (const auto & [_, discr] : new_variants_with_sizes) { - if (new_variants.size() - 1 == max_dynamic_types) + if (!canAddNewVariant(new_variants.size())) break; new_variants.push_back(src_variants[discr]); } @@ -846,13 +846,17 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// Check if both values are in shared variant. if (left_discr == left_shared_variant_discr && right_discr == right_shared_variant_discr) { - /// Extract type names from both values. + /// First check if both type and value are equal. auto left_value = getSharedVariant().getDataAt(left_variant.offsetAt(n)); + auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); + if (left_value == right_value) + return 0; + + /// Extract type names from both values. ReadBufferFromMemory buf_left(left_value.data, left_value.size); auto left_data_type = decodeDataType(buf_left); auto left_data_type_name = left_data_type->getName(); - auto right_value = right_dynamic.getSharedVariant().getDataAt(right_variant.offsetAt(m)); ReadBufferFromMemory buf_right(right_value.data, right_value.size); auto right_data_type = decodeDataType(buf_right); auto right_data_type_name = right_data_type->getName(); @@ -977,8 +981,6 @@ ColumnPtr ColumnDynamic::compress() const void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) { - LOG_DEBUG(getLogger("ColumnDynamic"), "takeDynamicStructureFromSourceColumns"); - if (!empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Dynamic column"); @@ -1050,8 +1052,10 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source DataTypePtr result_variant_type; Statistics new_statistics(Statistics::Source::MERGE); + /// Reset max_dynamic_types to global_max_dynamic_types. + max_dynamic_types = global_max_dynamic_types; /// Check if the number of all dynamic types exceeds the limit. - if (all_variants.size() - 1 > global_max_dynamic_types) + if (!canAddNewVariants(0, all_variants.size())) { /// Create list of variants with their sizes and sort it. std::vector> variants_with_sizes; @@ -1065,11 +1069,13 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source /// Take first max_dynamic_types variants from sorted list and fill shared_variants_statistics with the rest. DataTypes result_variants; - result_variants.reserve(global_max_dynamic_types + 1); + result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant. + /// Add shared variant. + result_variants.push_back(getSharedVariantDataType()); for (const auto & [size, variant] : variants_with_sizes) { /// Add variant to the resulting variants list until we reach max_dynamic_types. - if (result_variants.size() < global_max_dynamic_types) + if (canAddNewVariant(result_variants.size())) result_variants.push_back(variant); /// Add all remaining variants into shared_variants_statistics until we reach its max size. else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) @@ -1078,8 +1084,6 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source break; } - /// Add shared variant. - result_variants.push_back(getSharedVariantDataType()); result_variant_type = std::make_shared(result_variants); } else @@ -1094,8 +1098,9 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source new_statistics.variants_statistics[variant_name] = total_sizes[variant_name]; statistics = std::make_shared(std::move(new_statistics)); - /// Reduce max_dynamic_types to the number of selected variants (without shared variant), so there will be no possibility + /// Reduce max_dynamic_types to the number of selected variants, so there will be no possibility /// to extend selected variants on inerts into this column during merges. + /// -1 because we don't count shared variant in the limit. max_dynamic_types = variant_info.variant_names.size() - 1; /// Now we have the resulting Variant that will be used in all merged columns. @@ -1112,7 +1117,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source { /// Try to find this variant in current source column. auto it = source_variant_info.variant_name_to_discriminator.find(variant_info.variant_names[i]); - if (it != source_variant_info.variant_name_to_discriminator.end()) + if (it != source_variant_info.variant_name_to_discriminator.end()) /// Add shared variant. variants_source_columns[i].push_back(source_dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(it->second)); } } diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index a595a990964..1f050c9079e 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -27,6 +27,10 @@ namespace DB class ColumnDynamic final : public COWHelper, ColumnDynamic> { public: + /// Maximum limit on dynamic types. We use ColumnVariant to store all the types, + /// so the limit cannot be greater then ColumnVariant::MAX_NESTED_COLUMNS. + /// We also always have reserved variant for shared variant. + static constexpr size_t MAX_DYNAMIC_TYPES_LIMIT = ColumnVariant::MAX_NESTED_COLUMNS - 1; static constexpr const char * SHARED_VARIANT_TYPE_NAME = "SharedVariant"; struct Statistics @@ -359,6 +363,14 @@ public: size_t getMaxDynamicTypes() const { return max_dynamic_types; } + /// Check if we can add new variant types. + /// Shared variant doesn't count in the limit but always presents, + /// so we should subtract 1 from the total types count. + bool canAddNewVariants(size_t current_variants_count, size_t new_variants_count) { return current_variants_count + new_variants_count - 1 <= max_dynamic_types; } + bool canAddNewVariant(size_t current_variants_count) { return canAddNewVariants(current_variants_count, 1); } + bool canAddNewVariants(size_t new_variants_count) { return canAddNewVariants(variant_info.variant_names.size(), new_variants_count); } + bool canAddNewVariant() { return canAddNewVariants(variant_info.variant_names.size(), 1); } + void setVariantType(const DataTypePtr & variant_type); void setMaxDynamicPaths(size_t max_dynamic_type_); diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index e00638a50ab..04e76df57fe 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -73,8 +73,8 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS - 1) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 0 and 254"); + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() > ColumnDynamic::MAX_DYNAMIC_TYPES_LIMIT) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 0 and {}", ColumnDynamic::MAX_DYNAMIC_TYPES_LIMIT); return std::make_shared(literal->value.get()); } diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 5fadb6e4de4..9cd0adcc2ed 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -288,7 +288,7 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD statistics.variants_statistics[variant->getName()] = variant_size; } - /// Second, rend statistics for shared variants. + /// Second, read statistics for shared variants. size_t statistics_size; readVarUInt(statistics_size, *structure_stream); String variant_name; diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 21b98cf505c..660efb46b37 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4335,7 +4335,7 @@ private: auto type = decodeDataType(buf); auto type_name = type->getName(); auto it = shared_variant_to_index.find(type_name); - /// Check if didn't created column for this variant yet. + /// Check if we didn't create column for this variant yet. if (it == shared_variant_to_index.end()) { it = shared_variant_to_index.emplace(type_name, variant_columns_from_shared_variant.size()).first; diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 index 9c1f8fa45e8..de12c6b8737 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.reference.j2 @@ -1,3 +1,4 @@ +Memory Array(Array(Dynamic)) Array(Variant(String, UInt64)) LowCardinality(String) @@ -818,6 +819,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000 Array(Array(Dynamic)) Array(Variant(String, UInt64)) LowCardinality(String) @@ -1638,6 +1640,7 @@ str_79 0 [] [] [] [] [] [] [] [] [] [] [] +MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1 Array(Array(Dynamic)) Array(Variant(String, UInt64)) LowCardinality(String) From e9d16bc0549fe2a1fc44e50bd173ebc9358ed838 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Aug 2024 08:32:14 +0200 Subject: [PATCH 0777/1722] Use new mc restart --- docker/test/stateless/run.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 69052cf4771..c70cbe1fe45 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -195,16 +195,18 @@ ORDER BY tuple()" # use async inserts to avoid creating too many parts ./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 ./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500 + max_retries=100 retry=1 - while [ $retry -le $max_retries ]; do echo "clickminio restart attempt $retry:" - output=$(./mc admin service restart clickminio 2>&1) - echo "$output" + output=$(./mc admin service restart clickminio --wait --json 2>&1 | jq -r .status) + echo "Output of restart status: $output" - if echo "$output" | grep -q "Restarted \`clickminio\` successfully"; then + expected_output="success +success" + if [ "$output" = "$expected_output" ]; then echo "Restarted clickminio successfully." break fi @@ -218,7 +220,6 @@ if [ $retry -gt $max_retries ]; then echo "Failed to restart clickminio after $max_retries attempts." fi -./mc admin service restart clickminio ./mc admin trace clickminio > /test_output/minio.log & MC_ADMIN_PID=$! From b58a22aba7f590ada33bfce95dd525c4c8a414ae Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 11:41:42 +0000 Subject: [PATCH 0778/1722] Update test --- .../03036_dynamic_read_shared_subcolumns_small.sql.j2 | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 index 0c123d5f6fe..dde4f3f53c3 100644 --- a/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 +++ b/tests/queries/0_stateless/03036_dynamic_read_shared_subcolumns_small.sql.j2 @@ -6,6 +6,7 @@ drop table if exists test; {% for engine in ['Memory', 'MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000', 'MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1'] -%} +select '{{ engine }}'; create table test (id UInt64, d Dynamic(max_types=2)) engine={{ engine }}; insert into test select number, number from numbers(10); From 737948470d6f2cd69f1842396984ec17aea49b65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 11:54:34 +0000 Subject: [PATCH 0779/1722] Fix syntax --- tests/integration/test_storage_kafka/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 52d6054c12a..bef90e1b9d3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4796,7 +4796,7 @@ def test_max_rows_per_message(kafka_cluster, create_query_generator): DROP TABLE IF EXISTS test.kafka; {create_query}; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.kafka; """ ) @@ -4875,7 +4875,7 @@ def test_row_based_formats(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); @@ -4982,7 +4982,7 @@ def test_block_based_formats_2(kafka_cluster, create_query_generator): {create_query}; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.{table_name}; INSERT INTO test.{table_name} SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; @@ -5362,7 +5362,7 @@ def test_formats_errors(kafka_cluster): input_format_with_names_use_header=0, format_schema='key_value_message:Message'; - CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY key, value AS + CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY (key, value) AS SELECT key, value FROM test.{table_name}; """ ) From bbf10088869b0490fb81957b3417c7776f2c3089 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:15:54 +0000 Subject: [PATCH 0780/1722] Fix test --- tests/integration/helpers/cloud_tools.py | 59 ++++-- tests/integration/helpers/cluster.py | 1 + .../configs/config.d/named_collections.xml | 2 + .../integration/test_storage_iceberg/test.py | 194 ++++++++++-------- 4 files changed, 150 insertions(+), 106 deletions(-) diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/cloud_tools.py index 0d009b5f3fe..39c56e0eb85 100644 --- a/tests/integration/helpers/cloud_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -9,7 +9,9 @@ from enum import Enum class CloudUploader: - def upload_directory(self, local_path, remote_blob_path): + + def upload_directory(self, local_path, remote_blob_path, **kwargs): + print(kwargs) result_files = [] # print(f"Arguments: {local_path}, {s3_path}") # for local_file in glob.glob(local_path + "/**"): @@ -18,12 +20,11 @@ class CloudUploader: result_local_path = os.path.join(local_path, local_file) result_remote_blob_path = os.path.join(remote_blob_path, local_file) if os.path.isfile(local_file): - self.upload_file(result_local_path, result_remote_blob_path) + self.upload_file(result_local_path, result_remote_blob_path, **kwargs) result_files.append(result_remote_blob_path) else: files = self.upload_directory( - result_local_path, - result_remote_blob_path, + result_local_path, result_remote_blob_path, **kwargs ) result_files.extend(files) return result_files @@ -34,37 +35,61 @@ class S3Uploader(CloudUploader): self.minio_client = minio_client self.bucket_name = bucket_name - def upload_file(self, local_path, remote_blob_path): + def upload_file(self, local_path, remote_blob_path, bucket=None): + print(f"Upload to bucket: {bucket}") + if bucket is None: + bucket = self.bucket_name self.minio_client.fput_object( - bucket_name=self.bucket_name, + bucket_name=bucket, object_name=remote_blob_path, file_path=local_path, ) class LocalUploader(CloudUploader): - def __init__(self): - pass + + def __init__(self, clickhouse_node): + self.clickhouse_node = clickhouse_node def upload_file(self, local_path, remote_blob_path): - if local_path != remote_blob_path: - shutil.copyfile(local_path, remote_blob_path) + dir_path = os.path.dirname(remote_blob_path) + if dir_path != "": + self.clickhouse_node.exec_in_container( + [ + "bash", + "-c", + "mkdir -p {}".format(dir_path), + ] + ) + self.clickhouse_node.copy_file_to_container(local_path, remote_blob_path) class AzureUploader(CloudUploader): - def __init__(self, container_client): - self.container_client = container_client + def __init__(self, blob_service_client, container_name): + self.blob_service_client = blob_service_client + self.container_client = self.blob_service_client.get_container_client( + container_name + ) - def upload_file(self, local_path, remote_blob_path): - # print("Local path", local_path) - # print("Remote blob path", remote_blob_path) - blob_client = self.container_client.get_blob_client(remote_blob_path) + def upload_file(self, local_path, remote_blob_path, container_name=None): + if container_name is None: + container_client = self.container_client + else: + container_client = self.blob_service_client.get_container_client( + container_name + ) + blob_client = container_client.get_blob_client(remote_blob_path) with open(local_path, "rb") as data: - # print("Data", data) blob_client.upload_blob(data, overwrite=True) +def upload_directory(minio_client, bucket, local_path, remote_path): + S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( + local_path, remote_path + ) + + def get_file_contents(minio_client, bucket, s3_path): data = minio_client.get_object(bucket, s3_path) data_str = b"" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 215718463e8..7e85bcf2f3c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2204,6 +2204,7 @@ class ClickHouseCluster: data = fdata.read() encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") + self.exec_in_container( container_id, [ diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index 3d80fa2ecf9..b488638dd19 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -9,5 +9,7 @@ devstoreaccount1 Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b8a67600785..b1d9b7f66bf 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -95,24 +95,19 @@ def started_cluster(): cluster.azure_container_name = "mycontainer" - # connection_string = ( - # f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - # f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - # f"BlobEndpoint=http://azurite1:{cluster.env_variables['AZURITE_PORT']}/devstoreaccount1;" - # ) - # local_blob_service_client = BlobServiceClient.from_connection_string( - # cluster.env_variables["AZURITE_CONNECTION_STRING"] - # ) + cluster.blob_service_client = cluster.blob_service_client - local_blob_service_client = cluster.blob_service_client - - container_client = local_blob_service_client.create_container( + container_client = cluster.blob_service_client.create_container( cluster.azure_container_name ) cluster.container_client = container_client - cluster.default_azure_uploader = AzureUploader(container_client) + cluster.default_azure_uploader = AzureUploader( + cluster.blob_service_client, cluster.azure_container_name + ) + + cluster.default_local_uploader = LocalUploader(cluster.instances["node1"]) yield cluster @@ -187,13 +182,12 @@ def create_iceberg_table( table_function=False, **kwargs, ): - if storage_type == "local": - pass - elif storage_type == "s3": + if storage_type == "s3": if "bucket" in kwargs: bucket = kwargs["bucket"] else: bucket = cluster.minio_bucket + print(bucket) if table_function: return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" node.query( @@ -213,8 +207,19 @@ def create_iceberg_table( CREATE TABLE {table_name} ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" ) + elif storage_type == "local": + if table_function: + return f""" + icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) + """ + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" + ) else: - raise Exception("Unknown iceberg storage type: {}", storage_type) + raise Exception(f"Unknown iceberg storage type: {storage_type}") def create_initial_data_file( @@ -236,23 +241,28 @@ def create_initial_data_file( return result_path -def default_upload_directory(started_cluster, storage_type, local_path, remote_path): +def default_upload_directory( + started_cluster, storage_type, local_path, remote_path, **kwargs +): if storage_type == "local": - return LocalUploader().upload_directory(local_path, remote_path) + return started_cluster.default_local_uploader.upload_directory( + local_path, remote_path, **kwargs + ) elif storage_type == "s3": + print(kwargs) return started_cluster.default_s3_uploader.upload_directory( - local_path, remote_path + local_path, remote_path, **kwargs ) elif storage_type == "azure": return started_cluster.default_azure_uploader.upload_directory( - local_path, remote_path + local_path, remote_path, **kwargs ) else: - raise Exception("Unknown iceberg storage type: {}", storage_type) + raise Exception(f"Unknown iceberg storage type: {storage_type}") @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -264,7 +274,7 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -273,11 +283,9 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): "SELECT number, toString(number + 1) FROM numbers(100)" ) - # assert 0 == 1 - @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -293,7 +301,10 @@ def test_partition_by(started_cluster, format_version, storage_type): ) files = default_upload_directory( - started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", "" + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", ) assert len(files) == 14 # 10 partitiions + 4 metadata files @@ -302,7 +313,7 @@ def test_partition_by(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_multiple_iceberg_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -322,7 +333,7 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', @@ -357,7 +368,7 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -391,7 +402,7 @@ def test_types(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -421,7 +432,7 @@ def test_types(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_delete_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -441,7 +452,7 @@ def test_delete_files(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -487,7 +498,7 @@ def test_delete_files(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_evolved_schema(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -507,7 +518,7 @@ def test_evolved_schema(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -533,7 +544,7 @@ def test_evolved_schema(started_cluster, format_version, storage_type): assert data == expected_data -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_row_based_deletes(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -552,7 +563,7 @@ def test_row_based_deletes(started_cluster, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -572,7 +583,7 @@ def test_row_based_deletes(started_cluster, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_schema_inference(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -598,7 +609,7 @@ def test_schema_inference(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table( @@ -638,7 +649,7 @@ def test_schema_inference(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -657,7 +668,7 @@ def test_metadata_file_selection(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -666,7 +677,7 @@ def test_metadata_file_selection(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -693,7 +704,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -701,58 +712,63 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 -# @pytest.mark.parametrize("storage_type", ["s3", "azure"]) -# def test_restart_broken(started_cluster): -# instance = started_cluster.instances["node1"] -# spark = started_cluster.spark_session -# minio_client = started_cluster.minio_client -# bucket = "broken2" -# TABLE_NAME = "test_restart_broken_table_function" +def test_restart_broken_s3(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + TABLE_NAME = "test_restart_broken_table_function_s3" -# if not minio_client.bucket_exists(bucket): -# minio_client.make_bucket(bucket) + minio_client = started_cluster.minio_client + bucket = "broken2" -# parquet_data_path = create_initial_data_file( -# started_cluster, -# instance, -# "SELECT number, toString(number) FROM numbers(100)", -# TABLE_NAME, -# ) + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) -# write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") -# files = default_upload_directory( -# started_cluster, -# storage_type, -# f"/iceberg_data/default/{TABLE_NAME}/", -# "", -# ) -# create_iceberg_table(instance, TABLE_NAME, bucket=bucket) -# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + write_iceberg_from_df( + spark, + generate_data(spark, 0, 100), + TABLE_NAME, + mode="overwrite", + format_version="1", + ) -# s3_objects = list_s3_objects(minio_client, bucket, prefix="") -# assert ( -# len( -# list( -# minio_client.remove_objects( -# bucket, -# [DeleteObject(obj) for obj in s3_objects], -# ) -# ) -# ) -# == 0 -# ) -# minio_client.remove_bucket(bucket) + files = default_upload_directory( + started_cluster, + "s3", + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + bucket=bucket, + ) + create_iceberg_table("s3", instance, TABLE_NAME, started_cluster, bucket=bucket) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 -# instance.restart_clickhouse() + s3_objects = list_s3_objects(minio_client, bucket, prefix="") + assert ( + len( + list( + minio_client.remove_objects( + bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + minio_client.remove_bucket(bucket) -# assert "NoSuchBucket" in instance.query_and_get_error( -# f"SELECT count() FROM {TABLE_NAME}" -# ) + instance.restart_clickhouse() -# minio_client.make_bucket(bucket) + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) -# files = default_upload_directory( -# S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" -# ) + minio_client.make_bucket(bucket) -# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + files = default_upload_directory( + started_cluster, + "s3", + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + bucket=bucket, + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 3172bf8d76534bb46ce54ae6af96e14443d2b59b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 12 Aug 2024 12:23:32 +0000 Subject: [PATCH 0781/1722] better accounting of time for merge of projections --- .../Transforms/MergeJoinTransform.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 24 ++++++++++++++----- .../03221_merge_profile_events.reference | 2 +- .../03221_merge_profile_events.sql | 8 ++++--- 4 files changed, 25 insertions(+), 11 deletions(-) diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index ec7f567ea57..6abfa0fccd0 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -1282,7 +1282,7 @@ MergeJoinTransform::MergeJoinTransform( void MergeJoinTransform::onFinish() { - algorithm.logElapsed(merging_elapsed_ns / 1000000000ULL); + algorithm.logElapsed(static_cast(merging_elapsed_ns) / 1000000000ULL); } } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 39bac8f7c24..cb1921ede2b 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -461,8 +461,12 @@ void MergeTask::addGatheringColumn(GlobalRuntimeContextPtr global_ctx, const Str MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::getContextForNextStage() { - ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); - ProfileEvents::increment(ProfileEvents::MergeHorizontalStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + /// Do not increment for projection stage because time is already accounted in main task. + if (global_ctx->parent_part == nullptr) + { + ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + ProfileEvents::increment(ProfileEvents::MergeHorizontalStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + } auto new_ctx = std::make_shared(); @@ -481,8 +485,12 @@ MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::g MergeTask::StageRuntimeContextPtr MergeTask::VerticalMergeStage::getContextForNextStage() { - ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); - ProfileEvents::increment(ProfileEvents::MergeVerticalStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + /// Do not increment for projection stage because time is already accounted in main task. + if (global_ctx->parent_part == nullptr) + { + ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + ProfileEvents::increment(ProfileEvents::MergeVerticalStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + } auto new_ctx = std::make_shared(); new_ctx->need_sync = std::move(ctx->need_sync); @@ -1026,8 +1034,12 @@ bool MergeTask::execute() UInt64 stage_elapsed_ms = current_elapsed_ms - global_ctx->prev_elapsed_ms; global_ctx->prev_elapsed_ms = current_elapsed_ms; - ProfileEvents::increment(current_stage->getTotalTimeProfileEvent(), stage_elapsed_ms); - ProfileEvents::increment(ProfileEvents::MergeTotalMilliseconds, stage_elapsed_ms); + /// Do not increment for projection stage because time is already accounted in main task. + if (global_ctx->parent_part == nullptr) + { + ProfileEvents::increment(current_stage->getTotalTimeProfileEvent(), stage_elapsed_ms); + ProfileEvents::increment(ProfileEvents::MergeTotalMilliseconds, stage_elapsed_ms); + } auto next_stage_context = current_stage->getContextForNextStage(); diff --git a/tests/queries/0_stateless/03221_merge_profile_events.reference b/tests/queries/0_stateless/03221_merge_profile_events.reference index 729e53eae79..d969717336b 100644 --- a/tests/queries/0_stateless/03221_merge_profile_events.reference +++ b/tests/queries/0_stateless/03221_merge_profile_events.reference @@ -1,3 +1,3 @@ Horizontal 1 20000 3 0 480000 1 1 1 1 Vertical 1 20000 1 2 480000 1 1 1 1 1 1 -Vertical 2 20020 4 2 480660 1 1 1 1 1 1 1 1 +Vertical 2 400000 2 6 12800000 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03221_merge_profile_events.sql b/tests/queries/0_stateless/03221_merge_profile_events.sql index 787aff93ffc..1aa3dd266f8 100644 --- a/tests/queries/0_stateless/03221_merge_profile_events.sql +++ b/tests/queries/0_stateless/03221_merge_profile_events.sql @@ -58,12 +58,12 @@ DROP TABLE IF EXISTS t_merge_profile_events_2; DROP TABLE IF EXISTS t_merge_profile_events_3; -CREATE TABLE t_merge_profile_events_3 (id UInt64, v1 UInt64, v2 UInt64, PROJECTION p (SELECT sum(v1), sum(v2) GROUP BY id % 10)) +CREATE TABLE t_merge_profile_events_3 (id UInt64, v1 UInt64, v2 UInt64, PROJECTION p (SELECT v2, v2 * v2, v2 * 2, v2 * 10, v1 ORDER BY v1)) ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; -INSERT INTO t_merge_profile_events_3 SELECT number, number, number FROM numbers(10000); -INSERT INTO t_merge_profile_events_3 SELECT number, number, number FROM numbers(10000); +INSERT INTO t_merge_profile_events_3 SELECT number, number, number FROM numbers(100000); +INSERT INTO t_merge_profile_events_3 SELECT number, number, number FROM numbers(100000); OPTIMIZE TABLE t_merge_profile_events_3 FINAL; SYSTEM FLUSH LOGS; @@ -83,6 +83,8 @@ SELECT ProfileEvents['MergeVerticalStageExecuteMilliseconds'] > 0, ProfileEvents['MergeProjectionStageTotalMilliseconds'] > 0, ProfileEvents['MergeProjectionStageExecuteMilliseconds'] > 0, + ProfileEvents['MergeExecuteMilliseconds'] <= duration_ms, + ProfileEvents['MergeTotalMilliseconds'] <= duration_ms FROM system.part_log WHERE database = currentDatabase() AND table = 't_merge_profile_events_3' AND event_type = 'MergeParts' AND part_name = 'all_1_2_1'; DROP TABLE IF EXISTS t_merge_profile_events_3; From 0810703d6b162135a6f076e96052c433ef5b25d7 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:38:38 +0000 Subject: [PATCH 0782/1722] Roll out strange changes --- .../test/integration/runner/requirements.txt | 1 - src/Core/SettingsChangesHistory.cpp | 271 +----------------- .../IO/CachedOnDiskReadBufferFromFile.cpp | 17 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 - src/Disks/IO/createReadBufferFromFileBase.cpp | 19 +- src/IO/ReadBuffer.h | 5 +- .../DataLakes/IStorageDataLake.h | 49 +--- .../helpers/{cloud_tools.py => s3_tools.py} | 0 .../test_iceberg_azure_storage/test.py | 2 +- .../test_local_storage/__init__.py | 0 .../test_local_storage/configs/config.xml | 3 - .../test_local_storage/files/example2.csv | 3 - tests/integration/test_local_storage/test.py | 148 ---------- tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- .../integration/test_storage_iceberg/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 18 files changed, 26 insertions(+), 503 deletions(-) rename tests/integration/helpers/{cloud_tools.py => s3_tools.py} (100%) delete mode 100644 tests/integration/test_local_storage/__init__.py delete mode 100644 tests/integration/test_local_storage/configs/config.xml delete mode 100644 tests/integration/test_local_storage/files/example2.csv delete mode 100644 tests/integration/test_local_storage/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8bbb6a798ad..428986b5562 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -64,7 +64,6 @@ minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 oauthlib==3.2.0 -pandas==2.2.1 packaging==24.0 paramiko==3.4.0 pika==1.2.0 diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6637989202c..8f73e10c44f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"optimize_functions_to_subcolumns", false, true, "Enable optimization by default"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"local_create_new_file_on_insert", false, false, "Enables or disables creating a new file on each insert in local object storage engine tables"} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -335,16 +75,7 @@ static std::initializer_listgetOrSet( - cache_key, file_offset_of_buffer_end, size, file_size.value(), - create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); } - return !file_segments->empty(); } @@ -161,8 +158,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current read range: [{}, {})", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); + "Having {} file segments to read: {}, current offset: {}", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); initialized = true; } @@ -1046,10 +1043,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; - - LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", - remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); - size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1062,8 +1055,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); + fmt::format("Expected {} <= {} (size: {}, read range: {})", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); } swap(*implementation_buffer); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 01eb3d86696..bb9761a3905 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -169,7 +169,6 @@ void ReadBufferFromRemoteFSGather::initialize() bool ReadBufferFromRemoteFSGather::nextImpl() { - // LOG_DEBU /// Find first available buffer that fits to given offset. if (!current_buf) initialize(); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index a8d5bd1797e..9f1cb681f1a 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -85,8 +85,6 @@ private: size_t current_buf_idx = 0; SeekableReadBufferPtr current_buf; - std::deque buffer_cemetery_; - LoggerPtr log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index 76588bcb5b1..b132e25ac6b 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -1,15 +1,14 @@ -#include -#include #include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include #include -#include "ReadBufferFromRemoteFSGather.h" #include "config.h" namespace ProfileEvents @@ -100,7 +99,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { - #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( @@ -146,9 +144,8 @@ std::unique_ptr createReadBufferFromFileBase( settings.local_throttler); } else - { throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); - } + return res; }; diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 98f0c8c9059..73f5335411f 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -101,7 +101,10 @@ public: * * Try to read after the end should throw an exception. */ - bool ALWAYS_INLINE eof() { return !hasPendingData() && !next(); } + bool ALWAYS_INLINE eof() + { + return !hasPendingData() && !next(); + } void ignore() { diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 123a629f395..c8603fccb86 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -48,20 +48,10 @@ public: ConfigurationPtr configuration = base_configuration->clone(); - try { - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start1"); metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish1"); - - auto data_files = metadata->getDataFiles(); - LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data file name: {}", data_file); - } - configuration->setPaths(data_files); + configuration->setPaths(metadata->getDataFiles()); if (use_schema_from_metadata) schema_from_metadata = metadata->getTableSchema(); } @@ -90,20 +80,9 @@ public: const std::optional & format_settings_, ContextPtr local_context) { - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start2"); - auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish2"); - - auto schema_from_metadata = metadata->getTableSchema(); - auto data_files = metadata->getDataFiles(); - LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data file name: {}", data_file); - } if (!schema_from_metadata.empty()) { return ColumnsDescription(std::move(schema_from_metadata)); @@ -111,7 +90,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(data_files); + configuration->setPaths(metadata->getDataFiles()); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } @@ -120,33 +99,14 @@ public: void updateConfiguration(ContextPtr local_context) override { Storage::updateConfiguration(local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start3"); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish3"); - if (current_metadata && *current_metadata == *new_metadata) return; - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 1"); - - current_metadata = std::move(new_metadata); - - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 2"); - - auto data_files = current_metadata->getDataFiles(); - - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 3"); - - LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data file name: {}", data_file); - } auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(data_files); + updated_configuration->setPaths(current_metadata->getDataFiles()); updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); Storage::configuration = updated_configuration; @@ -188,10 +148,7 @@ private: if (!current_metadata) { Storage::updateConfiguration(local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start4"); - current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish4"); } auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); if (!column_mapping.empty()) diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/s3_tools.py similarity index 100% rename from tests/integration/helpers/cloud_tools.py rename to tests/integration/helpers/s3_tools.py diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py index f96f8acfaaf..0b7179c3cc4 100644 --- a/tests/integration/test_iceberg_azure_storage/test.py +++ b/tests/integration/test_iceberg_azure_storage/test.py @@ -37,7 +37,7 @@ from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject -from tests.integration.helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_local_storage/__init__.py b/tests/integration/test_local_storage/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml deleted file mode 100644 index b4179ee51df..00000000000 --- a/tests/integration/test_local_storage/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - /var/lib/clickhouse/ - diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv deleted file mode 100644 index 7b6e6d6bab1..00000000000 --- a/tests/integration/test_local_storage/files/example2.csv +++ /dev/null @@ -1,3 +0,0 @@ -id,data -1,Str1 -2,Str2 diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py deleted file mode 100644 index 89c3c17e1f0..00000000000 --- a/tests/integration/test_local_storage/test.py +++ /dev/null @@ -1,148 +0,0 @@ -import logging -import os -import random -import string - -import pytest - -from helpers.cluster import ClickHouseCluster - -from pathlib import Path - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "test_local_storage", main_configs=["configs/config.xml"], stay_alive=True - ) - - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def process_result(line: str): - return sorted( - list( - map( - lambda x: (int(x.split("\t")[0]), x.split("\t")[1]), - filter(lambda x: len(x) > 0, line.split("\n")), - ) - ) - ) - - -def test_local_engine(started_cluster): - node = started_cluster.instances["test_local_storage"] - node.query( - """ - CREATE TABLE test_0 ( - id Int64, - data String - ) ENGINE=Local('/data/example.csv', 'CSV'); - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); - """ - ) - - result = node.query( - """ - select * from test_0; - """ - ) - - assert [(-1, "7"), (1, "3"), (4, "abc")] == process_result(result) - - error_got = node.query_and_get_error( - """ - INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); - """ - ) - - node.query( - """ - SET engine_file_truncate_on_insert = 1; - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); - """, - settings={"engine_file_truncate_on_insert": 1}, - ) - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) - - node.query( - """ - SET local_create_new_file_on_insert = 1; - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); - """, - settings={"local_create_new_file_on_insert": 1}, - ) - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [ - (-1, "7"), - (0, "15"), - (1, "3"), - (4, "abc"), - (5, "arr"), - (9, "ty"), - ] == process_result(result) - - node.restart_clickhouse() - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) - - -def test_table_function(started_cluster): - node = started_cluster.instances["test_local_storage"] - - node.copy_file_to_container( - "test_local_storage/files/example2.csv", "/data/example2.csv" - ) - - result = node.query( - """ - SELECT * FROM local('/data/example2.csv', 'CSV', 'id Int64, data String'); - """ - ) - - print("Res5", result) - - assert [(1, "Str1"), (2, "Str2")] == process_result(result) - - # assert False diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 1c1a7decdc3..35fe9d1bd6e 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -30,7 +30,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 750b77b29f3..95825c2cd17 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from helpers.cloud_tools import prepare_s3_bucket, upload_directory, get_file_contents +from tests.integration.helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b1d9b7f66bf..013a016efcd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -38,7 +38,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject from azure.storage.blob import BlobServiceClient -from helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, get_file_contents, list_s3_objects, diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 31b1584507d..1c590824fd9 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from helpers.cloud_tools import prepare_s3_bucket +from tests.integration.helpers.s3_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 411e8f7cfb73d39df0614611766c3946603979ea Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:44:22 +0000 Subject: [PATCH 0783/1722] Remove unnecessary changes --- tests/integration/helpers/cluster.py | 1 - .../test_iceberg_azure_storage/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 - .../configs/users.d/users.xml | 9 - .../test_iceberg_azure_storage/test.py | 291 ------------------ .../test_storage_azure_blob_storage/test.py | 2 - tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 9 files changed, 3 insertions(+), 315 deletions(-) delete mode 100644 tests/integration/test_iceberg_azure_storage/__init__.py delete mode 100644 tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml delete mode 100644 tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml delete mode 100644 tests/integration/test_iceberg_azure_storage/test.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7e85bcf2f3c..215718463e8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2204,7 +2204,6 @@ class ClickHouseCluster: data = fdata.read() encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") - self.exec_in_container( container_id, [ diff --git a/tests/integration/test_iceberg_azure_storage/__init__.py b/tests/integration/test_iceberg_azure_storage/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml deleted file mode 100644 index d4c54e2d13d..00000000000 --- a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - http://minio1:9001/root/ - minio - minio123 - - - diff --git a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py deleted file mode 100644 index 0b7179c3cc4..00000000000 --- a/tests/integration/test_iceberg_azure_storage/test.py +++ /dev/null @@ -1,291 +0,0 @@ -import helpers.client -from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.test_tools import TSV - -import pyspark -import logging -import os -import json -import pytest -import time -import glob -import uuid -import os - -import tempfile - -import io -import avro.schema -import avro.io -import avro.datafile -import pandas as pd - -from pyspark.sql.types import ( - StructType, - StructField, - StringType, - IntegerType, - DateType, - TimestampType, - BooleanType, - ArrayType, -) -from pyspark.sql.functions import current_timestamp -from datetime import datetime -from pyspark.sql.functions import monotonically_increasing_id, row_number -from pyspark.sql.window import Window -from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 -from minio.deleteobjects import DeleteObject - -from tests.integration.helpers.s3_tools import ( - prepare_s3_bucket, - upload_directory, - get_file_contents, - list_s3_objects, -) - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -def get_spark(): - builder = ( - pyspark.sql.SparkSession.builder.appName("spark_test") - .config( - "spark.sql.catalog.spark_catalog", - "org.apache.iceberg.spark.SparkSessionCatalog", - ) - .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") - .config("spark.sql.catalog.spark_catalog.type", "hadoop") - .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") - .config( - "spark.sql.extensions", - "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", - ) - .master("local") - ) - return builder.master("local").getOrCreate() - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__, with_spark=True) - cluster.add_instance( - "node1", - main_configs=["configs/config.d/named_collections.xml"], - user_configs=["configs/users.d/users.xml"], - with_minio=True, - stay_alive=True, - ) - - logging.info("Starting cluster...") - cluster.start() - - prepare_s3_bucket(cluster) - logging.info("S3 bucket created") - - cluster.spark_session = get_spark() - - yield cluster - - finally: - cluster.shutdown() - - -def run_query(instance, query, stdin=None, settings=None): - # type: (ClickHouseInstance, str, object, dict) -> str - - logging.info("Running query '{}'...".format(query)) - result = instance.query(query, stdin=stdin, settings=settings) - logging.info("Query finished") - - return result - - -def write_iceberg_from_file( - spark, path, table_name, mode="overwrite", format_version="1", partition_by=None -): - if mode == "overwrite": - if partition_by is None: - spark.read.load(f"file://{path}").writeTo(table_name).tableProperty( - "format-version", format_version - ).using("iceberg").create() - else: - spark.read.load(f"file://{path}").writeTo(table_name).partitionedBy( - partition_by - ).tableProperty("format-version", format_version).using("iceberg").create() - else: - spark.read.load(f"file://{path}").writeTo(table_name).append() - - -def write_iceberg_from_df( - spark, df, table_name, mode="overwrite", format_version="1", partition_by=None -): - if mode == "overwrite": - if partition_by is None: - df.writeTo(table_name).tableProperty( - "format-version", format_version - ).using("iceberg").create() - else: - df.writeTo(table_name).tableProperty( - "format-version", format_version - ).partitionedBy(partition_by).using("iceberg").create() - else: - df.writeTo(table_name).append() - - -def generate_data(spark, start, end): - a = spark.range(start, end, 1).toDF("a") - b = spark.range(start + 1, end + 1, 1).toDF("b") - b = b.withColumn("b", b["b"].cast(StringType())) - - a = a.withColumn( - "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) - ) - b = b.withColumn( - "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) - ) - - df = a.join(b, on=["row_index"]).drop("row_index") - return df - - -def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) - - -def create_initial_data_file( - cluster, node, query, table_name, compression_method="none" -): - node.query( - f""" - INSERT INTO TABLE FUNCTION - file('{table_name}.parquet') - SETTINGS - output_format_parquet_compression_method='{compression_method}', - s3_truncate_on_insert=1 {query} - FORMAT Parquet""" - ) - user_files_path = os.path.join( - SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files" - ) - result_path = f"{user_files_path}/{table_name}.parquet" - return result_path - - -@pytest.mark.parametrize("format_version", ["1", "2"]) -def test_single_iceberg_file(started_cluster, format_version): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_single_iceberg_file_" + format_version - - inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" - parquet_data_path = create_initial_data_file( - started_cluster, instance, inserted_data, TABLE_NAME - ) - - write_iceberg_from_file( - spark, parquet_data_path, TABLE_NAME, format_version=format_version - ) - - files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" - ) - - for bucket in minio_client.list_buckets(): - for object in minio_client.list_objects(bucket.name, recursive=True): - print("Object: ", object.object_name) - extension = object.object_name.split(".")[-1] - print("File extension: ", extension) - try: - response = minio_client.get_object( - object.bucket_name, object.object_name - ) - - if extension == "avro": - avro_bytes = response.read() - - # Use BytesIO to create a file-like object from the byte string - avro_file = io.BytesIO(avro_bytes) - - # Read the Avro data - reader = avro.datafile.DataFileReader( - avro_file, avro.io.DatumReader() - ) - records = [record for record in reader] - - # Close the reader - reader.close() - - # Now you can work with the records - for record in records: - # print(json.dumps(record, indent=4, sort_keys=True)) - print(str(record)) - # my_json = ( - # str(record) - # .replace("'", '"') - # .replace("None", "null") - # .replace('b"', '"') - # ) - # print(my_json) - # data = json.loads(my_json) - # s = json.dumps(data, indent=4, sort_keys=True) - # print(s) - elif extension == "json": - my_bytes_value = response.read() - my_json = my_bytes_value.decode("utf8").replace("'", '"') - data = json.loads(my_json) - s = json.dumps(data, indent=4, sort_keys=True) - print(s) - elif extension == "parquet": - # print("To be continued...") - # # Your byte string containing the Parquet data - # parquet_bytes = response.read() - - # # Use BytesIO to create a file-like object from the byte string - # parquet_file = io.BytesIO(parquet_bytes) - - # # Read the Parquet data into a PyArrow Table - # table = pq.read_table(parquet_file) - - # # Convert the PyArrow Table to a Pandas DataFrame - # df = table.to_pandas() - - # # Now you can work with s DataFrame - # print(df) - parquet_bytes = ( - response.read() - ) # Replace with your actual byte string - - # Create a temporary file and write the byte string to it - with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - tmp_file.write(parquet_bytes) - tmp_file_path = tmp_file.name - - # Read the Parquet file using PySpark - df = spark.read.parquet(tmp_file_path) - - # Show the DataFrame - print(df.toPandas()) - else: - print(response.read()) - - finally: - print("----------------") - response.close() - response.release_conn() - - create_iceberg_table(instance, TABLE_NAME) - - assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( - inserted_data - ) - - assert 0 == 1 diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index c5599d20519..6fbe7634642 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -550,7 +550,6 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] - print("Account URL: ", cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " @@ -560,7 +559,6 @@ def test_simple_write_account_string_table_function(cluster): ) print(get_azure_file_content("test_simple_write_tf.csv", port)) assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' - assert 0 == 1 def test_simple_write_connection_string_table_function(cluster): diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 35fe9d1bd6e..054b79ff6fe 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -30,7 +30,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from tests.integration.helpers.s3_tools import ( +from helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 95825c2cd17..0c3fbfb3cda 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from tests.integration.helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 1c590824fd9..ab327afe90b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from tests.integration.helpers.s3_tools import prepare_s3_bucket +from helpers.s3_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 42d20f2a8d34640233c2dc4002d5fe611f2f9c77 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:52:11 +0000 Subject: [PATCH 0784/1722] Remove table registration --- .../StorageObjectStorageSource.cpp | 2 -- .../registerStorageObjectStorage.cpp | 26 ++++--------------- .../TableFunctionObjectStorage.cpp | 17 ++++-------- .../TableFunctionObjectStorage.h | 24 ++++------------- 4 files changed, 15 insertions(+), 54 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 74fcd0418ab..ed68dd61967 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -437,8 +437,6 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Get path"), "Path: {}", object_info.getPath()); - auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 4bc8cfa6a2f..b5f4cf5bb54 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,15 +1,16 @@ #include #include #include -#include -#include #include +#include #include #include namespace DB { +#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS + namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -69,6 +70,8 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } +#endif + #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { @@ -136,27 +139,8 @@ void registerStorageHDFS(StorageFactory & factory) } #endif -void registerStorageLocal(StorageFactory & factory) -{ - factory.registerStorage( - "Local", - [=](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - return createStorageObjectStorage(args, configuration, args.getLocalContext()); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::FILE, - }); -} - - void registerStorageObjectStorage(StorageFactory & factory) { - registerStorageLocal(factory); #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index d156afb81a1..550d9cc799b 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -12,13 +12,12 @@ #include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include namespace DB @@ -204,11 +203,6 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif - factory.registerFunction>( - {.documentation - = {.description = R"(The table function can be used to read the data stored locally.)", - .examples{{"local", "SELECT * FROM local(file_path, [format, compression, structure])", ""}}}, - .allow_readonly = false}); } #if USE_AZURE_BLOB_STORAGE @@ -229,5 +223,4 @@ template class TableFunctionObjectStorage; #endif -template class TableFunctionObjectStorage; } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 3468e5c5007..86b8f0d5e14 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include -#include -#include -#include -#include #include "config.h" +#include +#include +#include +#include +#include namespace DB { @@ -14,7 +14,6 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; -class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -91,17 +90,6 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; -struct LocalDefinition -{ - static constexpr auto name = "local"; - static constexpr auto storage_type_name = "Local"; - static constexpr auto signature = " - path\n" - " - path, format\n" - " - path, format, structure\n" - " - path, format, structure, compression_method\n"; - static constexpr auto max_number_of_arguments = 4; -}; - template class TableFunctionObjectStorage : public ITableFunction { @@ -181,6 +169,4 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif - -using TableFunctionLocal = TableFunctionObjectStorage; } From 34643ee16c5452a08feac63aaaea605a8c912b37 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 13:30:25 +0000 Subject: [PATCH 0785/1722] Run test only from modified files --- tests/ci/integration_tests_runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 84718462ab5..7b9e7d1f63e 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -69,9 +69,9 @@ def get_changed_tests_to_run(pr_info, repo_path): return [] for fpath in changed_files: - if "tests/integration/test_" in fpath: + if re.search("tests/integration/test_.*/test.*\.py", fpath) is not None: logging.info("File %s changed and seems like integration test", fpath) - result.add(fpath.split("/")[2]) + result.add("/".join(fpath.split("/")[2:])) return filter_existing_tests(result, repo_path) From 83d20bee00a4973cbffc3bcd9ba4073c79efb073 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Mon, 12 Aug 2024 07:42:55 -0600 Subject: [PATCH 0786/1722] Update 03221_create_if_not_exists_setting test to a .sql test --- ...221_create_if_not_exists_setting.reference | 4 --- ...=> 03221_create_if_not_exists_setting.sql} | 32 ++++--------------- 2 files changed, 6 insertions(+), 30 deletions(-) rename tests/queries/0_stateless/{03221_create_if_not_exists_setting.sh => 03221_create_if_not_exists_setting.sql} (51%) mode change 100755 => 100644 diff --git a/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference b/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference index 8740b05c9ca..e69de29bb2d 100644 --- a/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference +++ b/tests/queries/0_stateless/03221_create_if_not_exists_setting.reference @@ -1,4 +0,0 @@ -57 -82 -0 -0 diff --git a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sql old mode 100755 new mode 100644 similarity index 51% rename from tests/queries/0_stateless/03221_create_if_not_exists_setting.sh rename to tests/queries/0_stateless/03221_create_if_not_exists_setting.sql index 8dcde8977bc..59535981e7a --- a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sh +++ b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sql @@ -1,43 +1,23 @@ -#!/usr/bin/env bash -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh +SET create_if_not_exists=0; -- Default -# $CLICKHOUSE_CLIENT -mn -q "SET create_if_not_exists=0;" # Default -$CLICKHOUSE_CLIENT -mn -q " DROP TABLE IF EXISTS example_table; CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; -CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; -" 2> /dev/null -# ensure failed error code -echo $? -$CLICKHOUSE_CLIENT -mn -q " +CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; -- { serverError TABLE_ALREADY_EXISTS } + DROP DATABASE IF EXISTS example_database; CREATE DATABASE example_database; -CREATE DATABASE example_database; -" 2> /dev/null -echo $? +CREATE DATABASE example_database; -- { serverError DATABASE_ALREADY_EXISTS } -$CLICKHOUSE_CLIENT -mn -q " SET create_if_not_exists=1; + DROP TABLE IF EXISTS example_table; CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; CREATE TABLE example_table (id UInt32) ENGINE=MergeTree() ORDER BY id; -" -# ensure successful error code -echo $? - -$CLICKHOUSE_CLIENT -mn -q " -SET create_if_not_exists=1; DROP DATABASE IF EXISTS example_database; CREATE DATABASE example_database; CREATE DATABASE example_database; -" -echo $? -$CLICKHOUSE_CLIENT -mn -q " DROP DATABASE IF EXISTS example_database; -DROP TABLE IF EXISTS example_table; -" \ No newline at end of file +DROP TABLE IF EXISTS example_table; \ No newline at end of file From f7c6eabb498b47b21c13dbf55efbda551902d09c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 13:44:05 +0000 Subject: [PATCH 0787/1722] Small fix to filter by current_database in system.query_log --- ...01903_correct_block_size_prediction_with_default.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index 922dcb957e5..e898a9d5ee8 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -20,15 +20,15 @@ function test() # Follow https://github.com/ClickHouse/ClickHouse/issues/17317 for more info about the issue where=$1 - uuid_1=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + uuid_1=$(cat /proc/sys/kernel/random/uuid) $CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1 - uuid_2=$($CLICKHOUSE_CLIENT --query="SELECT generateUUIDv4()") + uuid_2=$(cat /proc/sys/kernel/random/uuid) $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 $CLICKHOUSE_CLIENT -n --query=" SYSTEM FLUSH LOGS; - WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), - memory_2 AS (SELECT memory_usage FROM system.query_log WHERE event_time > now() - INTERVAL 5 MINUTE AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) - SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" + WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), + memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" } test "" From 58dd01d8b61e76c378a45bc84967b990bf355e4e Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 13:45:00 +0000 Subject: [PATCH 0788/1722] Add docs --- .../table-engines/integrations/iceberg.md | 30 +++++++++++----- .../sql-reference/table-functions/iceberg.md | 34 +++++++++++-------- .../integration/test_storage_iceberg/test.py | 10 +----- 3 files changed, 42 insertions(+), 32 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 21fdbc0b1a5..40f3e7c22fd 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -6,28 +6,34 @@ sidebar_label: Iceberg # Iceberg Table Engine -This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. +This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure and locally stored tables. ## Create Table -Note that the Iceberg table must already exist in S3, this command does not take DDL parameters to create a new table. +Note that the Iceberg table must already exist in the storage, this command does not take DDL parameters to create a new table. ``` sql -CREATE TABLE iceberg_table - ENGINE = Iceberg(url, [aws_access_key_id, aws_secret_access_key,]) +CREATE TABLE iceberg_table_s3 + ENGINE = IcebergS3(url, [, NOSIGN | access_key_id, secret_access_key, [session_token]], format, [,compression]) + +CREATE TABLE iceberg_table_azure + ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + +CREATE TABLE iceberg_table_local + ENGINE = ICebergLocal(path_to_table, [,format] [,compression_method]) ``` -**Engine parameters** +**Engine arguments** -- `url` — url with the path to an existing Iceberg table. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. +Description of the arguments coincides with decription of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. +`format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) **Example** ```sql -CREATE TABLE iceberg_table ENGINE=Iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +CREATE TABLE iceberg_table ENGINE=IcebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` Using named collections: @@ -45,9 +51,15 @@ Using named collections: ``` ```sql -CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table') +CREATE TABLE iceberg_table ENGINE=IcebergS3(iceberg_conf, filename = 'test_table') + ``` +**Aliases** + + +Table engine `Iceberg` is an alias to `IcebergS3` now. + ## See also - [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index fa86b436a5e..dfe8d6f2d0a 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -6,35 +6,37 @@ sidebar_label: iceberg # iceberg Table Function -Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. +Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure or locally stored. ## Syntax ``` sql -iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure]) +icebergS3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,compression_method]) +icebergS3(named_collection[, option=value [,..]]) + +icebergAzure(connection_string|storage_account_url, container_name, blobpath, [,account_name], [,account_key] [,format] [,compression_method]) +icebergAzure(named_collection[, option=value [,..]]) + +icebergLocal(path_to_table, [,format] [,compression_method]) +icebergLocal(named_collection[, option=value [,..]]) ``` ## Arguments -- `url` — Bucket url with the path to an existing Iceberg table in S3. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). -- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used. -- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. - -Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md). +Description of the arguments coincides with decription of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. +`format` stands for the format of data files in the Iceberg table. **Returned value** - -A table with the specified structure for reading data in the specified Iceberg table in S3. +A table with the specified structure for reading data in the specified Iceberg table. **Example** ```sql -SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +SELECT * FROM icebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` :::important -ClickHouse currently supports reading v1 (v2 support is coming soon!) of the Iceberg format via the `iceberg` table function and `Iceberg` table engine. +ClickHouse currently supports reading v1 and v2 of the Iceberg format via the `icebergS3`, `icebergAzure` and `icebergLocal` table functions and `IcebergS3`, `icebergAzure` ans `icebergLocal` table engines. ::: ## Defining a named collection @@ -56,10 +58,14 @@ Here is an example of configuring a named collection for storing the URL and cre ``` ```sql -SELECT * FROM iceberg(iceberg_conf, filename = 'test_table') -DESCRIBE iceberg(iceberg_conf, filename = 'test_table') +SELECT * FROM icebergS3(iceberg_conf, filename = 'test_table') +DESCRIBE icebergS3(iceberg_conf, filename = 'test_table') ``` +**Aliases** + +Table function `iceberg` is an alias to `icebergS3` now. + **See Also** - [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 013a016efcd..b66133f5562 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -12,14 +12,6 @@ import glob import uuid import os -import tempfile - -import io -import avro.schema -import avro.io -import avro.datafile -import pandas as pd - from pyspark.sql.types import ( StructType, StructField, @@ -38,7 +30,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject from azure.storage.blob import BlobServiceClient -from tests.integration.helpers.s3_tools import ( +from helpers.s3_tools import ( prepare_s3_bucket, get_file_contents, list_s3_objects, From c817a4e8adfba37b23156ed75e1a501068e10cc1 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Mon, 12 Aug 2024 07:45:51 -0600 Subject: [PATCH 0789/1722] Update settings.md to clarify create_if_not_exists behavior Co-authored-by: Nikita Taranov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 22f73a03729..b9d5dde8522 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5640,6 +5640,6 @@ Default value: `0`. ## create_if_not_exists -Enable IF NOT EXISTS for CREATE statements by default. If either this setting or IF NOT EXISTS is specified, then no Exception will be thrown when trying to create a new table. +Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. Default value: `false`. From a32945614607c7043d2c04b0d431b781380ee946 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 13:47:10 +0000 Subject: [PATCH 0790/1722] Fix review comments --- src/Columns/ColumnObject.cpp | 18 ++++++------- src/DataTypes/DataTypeObject.cpp | 26 ++++++++++++------- src/DataTypes/DataTypeObject.h | 6 ++--- .../Serializations/ISerialization.cpp | 2 +- src/DataTypes/Serializations/ISerialization.h | 2 +- .../Serializations/SerializationJSON.cpp | 9 ++++--- src/Formats/JSONExtractTree.cpp | 9 ++++--- src/Functions/FunctionsJSON.cpp | 2 +- src/Functions/JSONPaths.cpp | 16 ++++++------ .../MergeTree/MergeTreeDataPartWriterWide.cpp | 16 ++++++------ .../MergeTree/MergeTreeReaderWide.cpp | 8 +++--- src/Storages/MergeTree/checkDataPart.cpp | 4 +-- 12 files changed, 65 insertions(+), 53 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index e7bb7639dd2..1f16c12f6ba 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -193,7 +193,7 @@ Field ColumnObject::operator[](size_t n) const const auto & shared_data_offsets = getSharedDataOffsets(); const auto [shared_paths, shared_values] = getSharedDataPathsAndValues(); - size_t start = shared_data_offsets[ssize_t(n) - 1]; + size_t start = shared_data_offsets[static_cast(n) - 1]; size_t end = shared_data_offsets[n]; for (size_t i = start; i != end; ++i) { @@ -495,7 +495,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co /// Check if src object doesn't have any paths in shared data in specified range. const auto & src_shared_data_offsets = src_object_column.getSharedDataOffsets(); - if (src_shared_data_offsets[ssize_t(start) - 1] == src_shared_data_offsets[ssize_t(start) + length - 1]) + if (src_shared_data_offsets[static_cast(start) - 1] == src_shared_data_offsets[static_cast(start) + length - 1]) { size_t current_size = size(); @@ -537,7 +537,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co size_t current_size = shared_data_offsets.size(); /// Use separate index to iterate over sorted src_dynamic_paths_for_shared_data. size_t src_dynamic_paths_for_shared_data_index = 0; - size_t offset = src_shared_data_offsets[ssize_t(row) - 1]; + size_t offset = src_shared_data_offsets[static_cast(row) - 1]; size_t end = src_shared_data_offsets[row]; for (size_t i = offset; i != end; ++i) { @@ -639,8 +639,8 @@ StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const c StringRef res(begin, 0); // Serialize all paths and values in binary format. const auto & shared_data_offsets = getSharedDataOffsets(); - size_t offset = shared_data_offsets[ssize_t(n) - 1]; - size_t end = shared_data_offsets[ssize_t(n)]; + size_t offset = shared_data_offsets[static_cast(n) - 1]; + size_t end = shared_data_offsets[static_cast(n)]; size_t num_paths = typed_paths.size() + dynamic_paths.size() + (end - offset); char * pos = arena.allocContinue(sizeof(size_t), begin); memcpy(pos, &num_paths, sizeof(size_t)); @@ -1259,8 +1259,8 @@ void ColumnObject::fillPathColumnFromSharedData(IColumn & path_column, StringRef { const auto & shared_data_array = assert_cast(*shared_data_column); const auto & shared_data_offsets = shared_data_array.getOffsets(); - size_t first_offset = shared_data_offsets[ssize_t(start) - 1]; - size_t last_offset = shared_data_offsets[ssize_t(end) - 1]; + size_t first_offset = shared_data_offsets[static_cast(start) - 1]; + size_t last_offset = shared_data_offsets[static_cast(end) - 1]; /// Check if we have at least one row with data. if (first_offset == last_offset) { @@ -1274,8 +1274,8 @@ void ColumnObject::fillPathColumnFromSharedData(IColumn & path_column, StringRef const auto & dynamic_serialization = getDynamicSerialization(); for (size_t i = start; i != end; ++i) { - size_t paths_start = shared_data_offsets[ssize_t(i) - 1]; - size_t paths_end = shared_data_offsets[ssize_t(i)]; + size_t paths_start = shared_data_offsets[static_cast(i) - 1]; + size_t paths_end = shared_data_offsets[static_cast(i)]; auto lower_bound_path_index = ColumnObject::findPathLowerBoundInSharedData(path, shared_data_paths, paths_start, paths_end); if (lower_bound_path_index != paths_end && shared_data_paths.getDataAt(lower_bound_path_index) == path) { diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 86041e580c4..7d05dded13d 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -39,9 +39,9 @@ namespace ErrorCodes DataTypeObject::DataTypeObject( const SchemaFormat & schema_format_, - const std::unordered_map & typed_paths_, - const std::unordered_set & paths_to_skip_, - const std::vector & path_regexps_to_skip_, + std::unordered_map typed_paths_, + std::unordered_set paths_to_skip_, + std::vector path_regexps_to_skip_, size_t max_dynamic_paths_, size_t max_dynamic_types_) : schema_format(schema_format_) @@ -59,10 +59,10 @@ DataTypeObject::DataTypeObject( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified with the data type ('{}') and matches the SKIP path prefix '{}'", typed_path, type->getName(), path_to_skip); } - for (const auto & path_regext_to_skip : paths_to_skip) + for (const auto & path_regex_to_skip : path_regexps_to_skip) { - if (re2::RE2::FullMatch(typed_path, re2::RE2(path_regext_to_skip))) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified with the data type ('{}') and matches the SKIP REGEXP '{}'", typed_path, type->getName(), path_regext_to_skip); + if (re2::RE2::FullMatch(typed_path, re2::RE2(path_regex_to_skip))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path '{}' is specified with the data type ('{}') and matches the SKIP REGEXP '{}'", typed_path, type->getName(), path_regex_to_skip); } } } @@ -220,7 +220,13 @@ void replaceJSONTypeNameIfNeeded(String & type_name, size_t max_dynamic_paths, s { /// Replace only if we don't already have parameters in JSON type declaration. if (pos + 4 == type_name.size() || type_name[pos + 4] != '(') - type_name.replace(pos, 4, fmt::format("JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); + type_name.replace( + pos, + 4, + fmt::format( + "JSON(max_dynamic_paths={}, max_dynamic_types={})", + max_dynamic_paths / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, + std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); pos = type_name.find("JSON", pos + 4); } } @@ -275,7 +281,7 @@ String getSubPath(const String & path, const String & prefix) return path.substr(prefix.size() + 1); } -std::string_view getSubPath(const std::string_view & path, const String & prefix) +std::string_view getSubPath(std::string_view path, const String & prefix) { return path.substr(prefix.size() + 1); } @@ -305,7 +311,7 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu } std::unique_ptr res = std::make_unique(std::make_shared(prefix, typed_paths_serializations)); - /// Keep all current constrains like limits and skip paths/prefixes/regexps. + /// Keep all current constraints like limits and skip paths/prefixes/regexps. res->type = std::make_shared(schema_format, typed_sub_paths, paths_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); /// If column was provided, we should create a column for the requested subcolumn. if (data.column) @@ -491,7 +497,7 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: } std::sort(path_regexps_to_skip.begin(), path_regexps_to_skip.end()); - return std::make_shared(schema_format, typed_paths, paths_to_skip, path_regexps_to_skip, max_dynamic_paths, max_dynamic_types); + return std::make_shared(schema_format, std::move(typed_paths), std::move(paths_to_skip), std::move(path_regexps_to_skip), max_dynamic_paths, max_dynamic_types); } static DataTypePtr createJSON(const ASTPtr & arguments) diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 6eea777ed26..7eb2e7729de 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -25,9 +25,9 @@ public: explicit DataTypeObject( const SchemaFormat & schema_format_, - const std::unordered_map & typed_paths_ = {}, - const std::unordered_set & paths_to_skip_ = {}, - const std::vector & path_regexps_to_skip_ = {}, + std::unordered_map typed_paths_ = {}, + std::unordered_set paths_to_skip_ = {}, + std::vector path_regexps_to_skip_ = {}, size_t max_dynamic_paths_ = DEFAULT_MAX_SEPARATELY_STORED_PATHS, size_t max_dynamic_types_ = DataTypeDynamic::DEFAULT_MAX_DYNAMIC_TYPES); diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 9d0d9f7c6eb..338edc3a144 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -411,7 +411,7 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref || path[last_elem].type == Substream::ObjectTypedPath; } -bool ISerialization::isFictitiousSubcolumn(const DB::ISerialization::SubstreamPath & path, size_t prefix_len) +bool ISerialization::isEphemeralSubcolumn(const DB::ISerialization::SubstreamPath & path, size_t prefix_len) { if (prefix_len == 0 || prefix_len > path.size()) return false; diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 52f636bcc86..480d5a4f7c4 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -451,7 +451,7 @@ public: /// Returns true if subcolumn doesn't actually stores any data in column and doesn't require a separate stream /// for writing/reading data. For example, it's a null-map subcolumn of Variant type (it's always constructed from discriminators);. - static bool isFictitiousSubcolumn(const SubstreamPath & path, size_t prefix_len); + static bool isEphemeralSubcolumn(const SubstreamPath & path, size_t prefix_len); protected: template diff --git a/src/DataTypes/Serializations/SerializationJSON.cpp b/src/DataTypes/Serializations/SerializationJSON.cpp index 346e6785abc..6ad326c65da 100644 --- a/src/DataTypes/Serializations/SerializationJSON.cpp +++ b/src/DataTypes/Serializations/SerializationJSON.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #if USE_SIMDJSON #include @@ -51,6 +52,8 @@ struct PathElements } } + boost::split(elements, path, boost::is_any_of(".")); + elements.emplace_back(last_dot_pos + 1, size_t(pos - last_dot_pos - 1)); } @@ -108,12 +111,12 @@ void SerializationJSON::serializeTextImpl(const IColumn & column, size_t const auto & dynamic_paths = column_object.getDynamicPaths(); const auto & shared_data_offsets = column_object.getSharedDataOffsets(); const auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); - size_t shared_data_offset = shared_data_offsets[ssize_t(row_num) - 1]; - size_t shared_data_end = shared_data_offsets[ssize_t(row_num)]; + size_t shared_data_offset = shared_data_offsets[static_cast(row_num) - 1]; + size_t shared_data_end = shared_data_offsets[static_cast(row_num)]; /// We need to convert the set of paths in this row to a JSON object. /// To do it, we first collect all the paths from current row, then we sort them - /// and construct the resulting JSON object by iterating over sorter list of paths. + /// and construct the resulting JSON object by iterating over sorted list of paths. /// For example: /// b.c, a.b, a.a, b.e, g, h.u.t -> a.a, a.b, b.c, b.e, g, h.u.t -> {"a" : {"a" : ..., "b" : ...}, "b" : {"c" : ..., "e" : ...}, "g" : ..., "h" : {"u" : {"t" : ...}}}. std::vector sorted_paths; diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 8203ccb5862..6340cd6f8e3 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1761,9 +1761,12 @@ private: if (paths_to_skip.contains(path)) return true; - auto it = std::lower_bound(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end(), path); - if (it != sorted_paths_to_skip.end() && it != sorted_paths_to_skip.begin() && path.starts_with(*std::prev(it))) - return true; + if (!sorted_paths_to_skip.empty()) + { + auto it = std::lower_bound(sorted_paths_to_skip.begin(), sorted_paths_to_skip.end(), path); + if (it != sorted_paths_to_skip.begin() && path.starts_with(*std::prev(it))) + return true; + } for (const auto & regexp : path_regexps_to_skip) { diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 9bf329f6dbb..ccdf4e6df81 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -739,7 +739,7 @@ public: { NumberType value; - if (!tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, true, error)) + if (!tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, /*allow_type_conversion=*/true, error)) return false; auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index 4f29846319d..4a84cec711b 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -178,8 +178,8 @@ private: const auto [shared_data_paths, _] = column_object.getSharedDataPathsAndValues(); for (size_t i = 0; i != shared_data_offsets.size(); ++i) { - size_t start = shared_data_offsets[ssize_t(i) - 1]; - size_t end = shared_data_offsets[ssize_t(i)]; + size_t start = shared_data_offsets[static_cast(i) - 1]; + size_t end = shared_data_offsets[static_cast(i)]; /// Merge sorted list of paths from shared data and sorted_dynamic_and_typed_paths size_t sorted_paths_index = 0; for (size_t j = start; j != end; ++j) @@ -187,7 +187,7 @@ private: auto shared_data_path = shared_data_paths->getDataAt(j); while (sorted_paths_index != sorted_dynamic_and_typed_paths.size() && sorted_dynamic_and_typed_paths[sorted_paths_index] < shared_data_path) { - auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; + const auto & path = sorted_dynamic_and_typed_paths[sorted_paths_index]; /// If it's dynamic path include it only if it's not NULL. if (auto it = dynamic_path_columns.find(path); it == dynamic_path_columns.end() || !it->second->isNullAt(i)) data.insertData(path.data(), path.size()); @@ -199,7 +199,7 @@ private: for (; sorted_paths_index != sorted_dynamic_and_typed_paths.size(); ++sorted_paths_index) { - auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; + const auto & path = sorted_dynamic_and_typed_paths[sorted_paths_index]; if (auto it = dynamic_path_columns.find(path); it == dynamic_path_columns.end() || !it->second->isNullAt(i)) data.insertData(path.data(), path.size()); } @@ -254,8 +254,8 @@ private: /// Iterate over all rows and extract types from dynamic values in shared data. for (size_t i = 0; i != shared_data_offsets.size(); ++i) { - size_t start = shared_data_offsets[ssize_t(i) - 1]; - size_t end = shared_data_offsets[ssize_t(i)]; + size_t start = shared_data_offsets[static_cast(i) - 1]; + size_t end = shared_data_offsets[static_cast(i)]; for (size_t j = start; j != end; ++j) { if (auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j))) @@ -288,8 +288,8 @@ private: const auto [shared_data_paths, shared_data_values] = column_object.getSharedDataPathsAndValues(); for (size_t i = 0; i != shared_data_offsets.size(); ++i) { - size_t start = shared_data_offsets[ssize_t(i) - 1]; - size_t end = shared_data_offsets[ssize_t(i)]; + size_t start = shared_data_offsets[static_cast(i) - 1]; + size_t end = shared_data_offsets[static_cast(i)]; /// Merge sorted list of paths and values from shared data and sorted_typed_and_dynamic_paths_with_types size_t sorted_paths_index = 0; for (size_t j = start; j != end; ++j) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index d507468f3ea..3edcce74b09 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -132,8 +132,8 @@ void MergeTreeDataPartWriterWide::addStreams( { assert(!substream_path.empty()); - /// Don't create streams for fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Don't create streams for ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return; auto full_stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); @@ -209,8 +209,8 @@ ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGett { return [&, this] (const ISerialization::SubstreamPath & substream_path) -> WriteBuffer * { - /// Skip fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Skip ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return nullptr; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -375,8 +375,8 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( min_compress_block_size = settings.min_compress_block_size; getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { - /// Skip fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Skip ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -417,8 +417,8 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { - /// Skip fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Skip ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 9ab45ec7b56..898bf5a2933 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -213,8 +213,8 @@ void MergeTreeReaderWide::addStreams( ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) { - /// Don't create streams for fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Don't create streams for ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return; auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); @@ -352,8 +352,8 @@ void MergeTreeReaderWide::prefetchForColumn( deserializePrefix(serialization, name_and_type, current_task_last_mark, cache, deserialize_states_cache); auto callback = [&](const ISerialization::SubstreamPath & substream_path) { - /// Skip fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Skip ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return; auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 6cc2b947424..3a22daa0011 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -215,8 +215,8 @@ static IMergeTreeDataPart::Checksums checkDataPart( { get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { - /// Skip fictitious subcolumns that don't store any real data. - if (ISerialization::isFictitiousSubcolumn(substream_path, substream_path.size())) + /// Skip ephemeral subcolumns that don't store any real data. + if (ISerialization::isEphemeralSubcolumn(substream_path, substream_path.size())) return; auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column, substream_path, ".bin", data_part_storage); From 9834457c26011bb785d4c62fbbb758afd6a6ef12 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 13:48:18 +0000 Subject: [PATCH 0791/1722] Fix copying arguments --- src/DataTypes/DataTypeObject.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 7d05dded13d..9f76d04d7da 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -45,9 +45,9 @@ DataTypeObject::DataTypeObject( size_t max_dynamic_paths_, size_t max_dynamic_types_) : schema_format(schema_format_) - , typed_paths(typed_paths_) - , paths_to_skip(paths_to_skip_) - , path_regexps_to_skip(path_regexps_to_skip_) + , typed_paths(std::move(typed_paths_)) + , paths_to_skip(std::move(paths_to_skip_)) + , path_regexps_to_skip(std::move(path_regexps_to_skip_)) , max_dynamic_paths(max_dynamic_paths_) , max_dynamic_types(max_dynamic_types_) { From 6cde029ed9cd6a22937193e12863974f1fa8f160 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 13:48:44 +0000 Subject: [PATCH 0792/1722] Fix style --- tests/ci/integration_tests_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 7b9e7d1f63e..f5dbef4f6db 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -69,7 +69,7 @@ def get_changed_tests_to_run(pr_info, repo_path): return [] for fpath in changed_files: - if re.search("tests/integration/test_.*/test.*\.py", fpath) is not None: + if re.search(r"tests/integration/test_.*/test.*\.py", fpath) is not None: logging.info("File %s changed and seems like integration test", fpath) result.add("/".join(fpath.split("/")[2:])) return filter_existing_tests(result, repo_path) From 8522776c33b334745e8743a075f0456aee3720c9 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 13:49:01 +0000 Subject: [PATCH 0793/1722] Remove unused code --- src/DataTypes/Serializations/SerializationJSON.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationJSON.cpp b/src/DataTypes/Serializations/SerializationJSON.cpp index 6ad326c65da..092ccd1c5a5 100644 --- a/src/DataTypes/Serializations/SerializationJSON.cpp +++ b/src/DataTypes/Serializations/SerializationJSON.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #if USE_SIMDJSON #include @@ -52,8 +51,6 @@ struct PathElements } } - boost::split(elements, path, boost::is_any_of(".")); - elements.emplace_back(last_dot_pos + 1, size_t(pos - last_dot_pos - 1)); } From da5b9582a990f7a2c05c1a3dede3739fb9cbfcae Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 13:54:17 +0000 Subject: [PATCH 0794/1722] Fix indent --- .../01903_correct_block_size_prediction_with_default.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index e898a9d5ee8..075d9a1dacf 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -26,9 +26,9 @@ function test() $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 $CLICKHOUSE_CLIENT -n --query=" SYSTEM FLUSH LOGS; - WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), - memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) - SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" + WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), + memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" } test "" From 61e616204eee76289e3da63dd663ce39a0316d0c Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 14:21:42 +0000 Subject: [PATCH 0795/1722] Docs correct typos --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- docs/en/sql-reference/table-functions/iceberg.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 40f3e7c22fd..78e076a7427 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -25,7 +25,7 @@ CREATE TABLE iceberg_table_local **Engine arguments** -Description of the arguments coincides with decription of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. +Description of the arguments coincides with description of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. `format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index dfe8d6f2d0a..784fd646860 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -23,7 +23,7 @@ icebergLocal(named_collection[, option=value [,..]]) ## Arguments -Description of the arguments coincides with decription of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. +Description of the arguments coincides with description of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. `format` stands for the format of data files in the Iceberg table. **Returned value** From 1cc845726842f388c4524d55b248f210e28d979d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 12 Aug 2024 16:57:47 +0200 Subject: [PATCH 0796/1722] fix reviews, fix crash in fuzzer --- src/DataTypes/getLeastSupertype.cpp | 2 +- src/DataTypes/getLeastSupertype.h | 2 +- src/Functions/FunctionsConversion.cpp | 11 +- .../03223_interval_data_type_comparison.sql | 198 +++++++++--------- 4 files changed, 106 insertions(+), 107 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 8bcec49815f..65df529e78b 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -251,7 +251,7 @@ DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet & } if (is_higher_interval && min_interval <= IntervalKind::Kind::Week) - throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName()); + throw Exception(ErrorCodes::NO_COMMON_TYPE, "Cannot compare intervals {} and {} because the number of days in a month is not fixed", types[0]->getName(), types[1]->getName()); if (smallest_type) { diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 8dd1685e6e9..55d8e8fff0d 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -51,6 +51,6 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types); DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types); /// A vector that shows the conversion rates to the next Interval type starting from NanoSecond -static std::vector interval_conversions = {1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4, 1}; +static std::vector interval_conversions = {1, 1000, 1000, 1000, 60, 60, 24, 7, 4, 3, 4}; } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 25c6bbcbfef..b6102cb7ecf 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1586,17 +1586,17 @@ struct ConvertImpl Int64 result_value; int from_position = static_cast(from.kind); - int to_position = static_cast(to.kind); // Positions of each interval according to granurality map + int to_position = static_cast(to.kind); /// Positions of each interval according to granularity map if (from_position < to_position) { - for (int i = from_position - 1; i <= to_position; ++i) + for (int i = from_position; i < to_position; ++i) conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) / conversion_factor; } else { - for (int i = from_position - 1; i >= to_position; --i) + for (int i = from_position; i > to_position; --i) conversion_factor *= interval_conversions[i]; result_value = arguments[0].column->getInt(0) * conversion_factor; } @@ -2366,9 +2366,8 @@ private: } if constexpr (std::is_same_v) - { - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); - } + if (WhichDataType(from_type).isInterval()) + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); } if (!done) diff --git a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql index 5d01addae45..77b6e2fa3dc 100644 --- a/tests/queries/0_stateless/03223_interval_data_type_comparison.sql +++ b/tests/queries/0_stateless/03223_interval_data_type_comparison.sql @@ -1,142 +1,142 @@ SELECT('Comparing nanoseconds'); -SELECT toIntervalNanosecond(500) > toIntervalNanosecond(300); -SELECT toIntervalNanosecond(1000) < toIntervalNanosecond(1500); -SELECT toIntervalNanosecond(2000) = toIntervalNanosecond(2000); -SELECT toIntervalNanosecond(1000) >= toIntervalMicrosecond(1); -SELECT toIntervalNanosecond(1000001) > toIntervalMillisecond(1); -SELECT toIntervalNanosecond(2000000001) > toIntervalSecond(2); -SELECT toIntervalNanosecond(60000000000) = toIntervalMinute(1); -SELECT toIntervalNanosecond(7199999999999) < toIntervalHour(2); -SELECT toIntervalNanosecond(1) < toIntervalDay(2); -SELECT toIntervalNanosecond(5) < toIntervalWeek(1); +SELECT INTERVAL 500 NANOSECOND > INTERVAL 300 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND < INTERVAL 1500 NANOSECOND; +SELECT INTERVAL 2000 NANOSECOND = INTERVAL 2000 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND >= INTERVAL 1 MICROSECOND; +SELECT INTERVAL 1000001 NANOSECOND > INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000001 NANOSECOND > INTERVAL 2 SECOND; +SELECT INTERVAL 60000000000 NANOSECOND = INTERVAL 1 MINUTE; +SELECT INTERVAL 7199999999999 NANOSECOND < INTERVAL 2 HOUR; +SELECT INTERVAL 1 NANOSECOND < INTERVAL 2 DAY; +SELECT INTERVAL 5 NANOSECOND < INTERVAL 1 WEEK; -SELECT toIntervalNanosecond(500) < toIntervalNanosecond(300); -SELECT toIntervalNanosecond(1000) > toIntervalNanosecond(1500); -SELECT toIntervalNanosecond(2000) != toIntervalNanosecond(2000); -SELECT toIntervalNanosecond(1000) < toIntervalMicrosecond(1); -SELECT toIntervalNanosecond(1000001) < toIntervalMillisecond(1); -SELECT toIntervalNanosecond(2000000001) < toIntervalSecond(2); -SELECT toIntervalNanosecond(60000000000) != toIntervalMinute(1); -SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2); -SELECT toIntervalNanosecond(1) > toIntervalDay(2); -SELECT toIntervalNanosecond(5) > toIntervalWeek(1); +SELECT INTERVAL 500 NANOSECOND < INTERVAL 300 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND > INTERVAL 1500 NANOSECOND; +SELECT INTERVAL 2000 NANOSECOND != INTERVAL 2000 NANOSECOND; +SELECT INTERVAL 1000 NANOSECOND < INTERVAL 1 MICROSECOND; +SELECT INTERVAL 1000001 NANOSECOND < INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000001 NANOSECOND < INTERVAL 2 SECOND; +SELECT INTERVAL 60000000000 NANOSECOND != INTERVAL 1 MINUTE; +SELECT INTERVAL 7199999999999 NANOSECOND > INTERVAL 2 HOUR; +SELECT INTERVAL 1 NANOSECOND > INTERVAL 2 DAY; +SELECT INTERVAL 5 NANOSECOND > INTERVAL 1 WEEK; -SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 1 NANOSECOND < INTERVAL 2 MONTH; -- { serverError NO_COMMON_TYPE } SELECT('Comparing microseconds'); -SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999); -SELECT toIntervalMicrosecond(1001) > toIntervalMillisecond(1); -SELECT toIntervalMicrosecond(2000000) = toIntervalSecond(2); -SELECT toIntervalMicrosecond(179999999) < toIntervalMinute(3); -SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1); -SELECT toIntervalMicrosecond(36000000000000) > toIntervalDay(2); -SELECT toIntervalMicrosecond(1209600000000) = toIntervalWeek(2); +SELECT INTERVAL 1 MICROSECOND < INTERVAL 999 MICROSECOND; +SELECT INTERVAL 1001 MICROSECOND > INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000 MICROSECOND = INTERVAL 2 SECOND; +SELECT INTERVAL 179999999 MICROSECOND < INTERVAL 3 MINUTE; +SELECT INTERVAL 3600000000 MICROSECOND = INTERVAL 1 HOUR; +SELECT INTERVAL 36000000000000 MICROSECOND > INTERVAL 2 DAY; +SELECT INTERVAL 1209600000000 MICROSECOND = INTERVAL 2 WEEK; -SELECT toIntervalMicrosecond(1) > toIntervalMicrosecond(999); -SELECT toIntervalMicrosecond(1001) < toIntervalMillisecond(1); -SELECT toIntervalMicrosecond(2000000) != toIntervalSecond(2); -SELECT toIntervalMicrosecond(179999999) > toIntervalMinute(3); -SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1); -SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2); -SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2); +SELECT INTERVAL 1 MICROSECOND > INTERVAL 999 MICROSECOND; +SELECT INTERVAL 1001 MICROSECOND < INTERVAL 1 MILLISECOND; +SELECT INTERVAL 2000000 MICROSECOND != INTERVAL 2 SECOND; +SELECT INTERVAL 179999999 MICROSECOND > INTERVAL 3 MINUTE; +SELECT INTERVAL 3600000000 MICROSECOND != INTERVAL 1 HOUR; +SELECT INTERVAL 36000000000000 MICROSECOND < INTERVAL 2 DAY; +SELECT INTERVAL 1209600000000 MICROSECOND != INTERVAL 2 WEEK; -SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 36000000000000 MICROSECOND < INTERVAL 1 QUARTER; -- { serverError NO_COMMON_TYPE } SELECT('Comparing milliseconds'); -SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2); -SELECT toIntervalMillisecond(2000) = toIntervalSecond(2); -SELECT toIntervalMillisecond(170000) < toIntervalMinute(3); -SELECT toIntervalMillisecond(144000001) > toIntervalHour(40); -SELECT toIntervalMillisecond(1728000000) = toIntervalDay(20); -SELECT toIntervalMillisecond(1198599999) < toIntervalWeek(2); +SELECT INTERVAL 2000 MILLISECOND > INTERVAL 2 MILLISECOND; +SELECT INTERVAL 2000 MILLISECOND = INTERVAL 2 SECOND; +SELECT INTERVAL 170000 MILLISECOND < INTERVAL 3 MINUTE; +SELECT INTERVAL 144000001 MILLISECOND > INTERVAL 40 HOUR; +SELECT INTERVAL 1728000000 MILLISECOND = INTERVAL 20 DAY; +SELECT INTERVAL 1198599999 MILLISECOND < INTERVAL 2 WEEK; -SELECT toIntervalMillisecond(2000) < toIntervalMillisecond(2); -SELECT toIntervalMillisecond(2000) != toIntervalSecond(2); -SELECT toIntervalMillisecond(170000) > toIntervalMinute(3); -SELECT toIntervalMillisecond(144000001) < toIntervalHour(40); -SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20); -SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2); +SELECT INTERVAL 2000 MILLISECOND < INTERVAL 2 MILLISECOND; +SELECT INTERVAL 2000 MILLISECOND != INTERVAL 2 SECOND; +SELECT INTERVAL 170000 MILLISECOND > INTERVAL 3 MINUTE; +SELECT INTERVAL 144000001 MILLISECOND < INTERVAL 40 HOUR; +SELECT INTERVAL 1728000000 MILLISECOND != INTERVAL 20 DAY; +SELECT INTERVAL 1198599999 MILLISECOND > INTERVAL 2 WEEK; -SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 36000000000000 MILLISECOND < INTERVAL 1 YEAR; -- { serverError NO_COMMON_TYPE } SELECT('Comparing seconds'); -SELECT toIntervalSecond(120) > toIntervalSecond(2); -SELECT toIntervalSecond(120) = toIntervalMinute(2); -SELECT toIntervalSecond(1) < toIntervalHour(2); -SELECT toIntervalSecond(86401) >= toIntervalDay(1); -SELECT toIntervalSecond(1209600) = toIntervalWeek(2); +SELECT INTERVAL 120 SECOND > INTERVAL 2 SECOND; +SELECT INTERVAL 120 SECOND = INTERVAL 2 MINUTE; +SELECT INTERVAL 1 SECOND < INTERVAL 2 HOUR; +SELECT INTERVAL 86401 SECOND >= INTERVAL 1 DAY; +SELECT INTERVAL 1209600 SECOND = INTERVAL 2 WEEK; -SELECT toIntervalSecond(120) < toIntervalSecond(2); -SELECT toIntervalSecond(120) != toIntervalMinute(2); -SELECT toIntervalSecond(1) > toIntervalHour(2); -SELECT toIntervalSecond(86401) < toIntervalDay(1); -SELECT toIntervalSecond(1209600) != toIntervalWeek(2); +SELECT INTERVAL 120 SECOND < INTERVAL 2 SECOND; +SELECT INTERVAL 120 SECOND != INTERVAL 2 MINUTE; +SELECT INTERVAL 1 SECOND > INTERVAL 2 HOUR; +SELECT INTERVAL 86401 SECOND < INTERVAL 1 DAY; +SELECT INTERVAL 1209600 SECOND != INTERVAL 2 WEEK; -SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 36000000000000 SECOND < INTERVAL 1 MONTH; -- { serverError NO_COMMON_TYPE } SELECT('Comparing minutes'); -SELECT toIntervalMinute(1) < toIntervalMinute(59); -SELECT toIntervalMinute(1) < toIntervalHour(59); -SELECT toIntervalMinute(1440) = toIntervalDay(1); -SELECT toIntervalMinute(30241) > toIntervalWeek(3); +SELECT INTERVAL 1 MINUTE < INTERVAL 59 MINUTE; +SELECT INTERVAL 1 MINUTE < INTERVAL 59 HOUR; +SELECT INTERVAL 1440 MINUTE = INTERVAL 1 DAY; +SELECT INTERVAL 30241 MINUTE > INTERVAL 3 WEEK; -SELECT toIntervalMinute(1) > toIntervalMinute(59); -SELECT toIntervalMinute(1) > toIntervalHour(59); -SELECT toIntervalMinute(1440) != toIntervalDay(1); -SELECT toIntervalMinute(30241) < toIntervalWeek(3); +SELECT INTERVAL 1 MINUTE > INTERVAL 59 MINUTE; +SELECT INTERVAL 1 MINUTE > INTERVAL 59 HOUR; +SELECT INTERVAL 1440 MINUTE != INTERVAL 1 DAY; +SELECT INTERVAL 30241 MINUTE < INTERVAL 3 WEEK; -SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 MINUTE = INTERVAL 120 QUARTER; -- { serverError NO_COMMON_TYPE } SELECT('Comparing hours'); -SELECT toIntervalHour(48) > toIntervalHour(2); -SELECT toIntervalHour(48) >= toIntervalDay(2); -SELECT toIntervalHour(672) = toIntervalWeek(4); +SELECT INTERVAL 48 HOUR > INTERVAL 2 HOUR; +SELECT INTERVAL 48 HOUR >= INTERVAL 2 DAY; +SELECT INTERVAL 672 HOUR = INTERVAL 4 WEEK; -SELECT toIntervalHour(48) < toIntervalHour(2); -SELECT toIntervalHour(48) < toIntervalDay(2); -SELECT toIntervalHour(672) != toIntervalWeek(4); +SELECT INTERVAL 48 HOUR < INTERVAL 2 HOUR; +SELECT INTERVAL 48 HOUR < INTERVAL 2 DAY; +SELECT INTERVAL 672 HOUR != INTERVAL 4 WEEK; -SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 HOUR < INTERVAL 1 YEAR; -- { serverError NO_COMMON_TYPE } SELECT('Comparing days'); -SELECT toIntervalDay(1) < toIntervalDay(23); -SELECT toIntervalDay(25) > toIntervalWeek(3); +SELECT INTERVAL 1 DAY < INTERVAL 23 DAY; +SELECT INTERVAL 25 DAY > INTERVAL 3 WEEK; -SELECT toIntervalDay(1) > toIntervalDay(23); -SELECT toIntervalDay(25) < toIntervalWeek(3); +SELECT INTERVAL 1 DAY > INTERVAL 23 DAY; +SELECT INTERVAL 25 DAY < INTERVAL 3 WEEK; -SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 DAY = INTERVAL 48 MONTH; -- { serverError NO_COMMON_TYPE } SELECT('Comparing weeks'); -SELECT toIntervalWeek(1) < toIntervalWeek(6); +SELECT INTERVAL 1 WEEK < INTERVAL 6 WEEK; -SELECT toIntervalWeek(1) > toIntervalWeek(6); +SELECT INTERVAL 1 WEEK > INTERVAL 6 WEEK; -SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 124 WEEK > INTERVAL 8 QUARTER; -- { serverError NO_COMMON_TYPE } SELECT('Comparing months'); -SELECT toIntervalMonth(1) < toIntervalMonth(3); -SELECT toIntervalMonth(124) > toIntervalQuarter(5); -SELECT toIntervalMonth(36) = toIntervalYear(3); +SELECT INTERVAL 1 MONTH < INTERVAL 3 MONTH; +SELECT INTERVAL 124 MONTH > INTERVAL 5 QUARTER; +SELECT INTERVAL 36 MONTH = INTERVAL 3 YEAR; -SELECT toIntervalMonth(1) > toIntervalMonth(3); -SELECT toIntervalMonth(124) < toIntervalQuarter(5); -SELECT toIntervalMonth(36) != toIntervalYear(3); +SELECT INTERVAL 1 MONTH > INTERVAL 3 MONTH; +SELECT INTERVAL 124 MONTH < INTERVAL 5 QUARTER; +SELECT INTERVAL 36 MONTH != INTERVAL 3 YEAR; -SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 6 MONTH = INTERVAL 26 MICROSECOND; -- { serverError NO_COMMON_TYPE } SELECT('Comparing quarters'); -SELECT toIntervalQuarter(5) > toIntervalQuarter(4); -SELECT toIntervalQuarter(20) = toIntervalYear(5); +SELECT INTERVAL 5 QUARTER > INTERVAL 4 QUARTER; +SELECT INTERVAL 20 QUARTER = INTERVAL 5 YEAR; -SELECT toIntervalQuarter(5) < toIntervalQuarter(4); -SELECT toIntervalQuarter(20) != toIntervalYear(5); +SELECT INTERVAL 5 QUARTER < INTERVAL 4 QUARTER; +SELECT INTERVAL 20 QUARTER != INTERVAL 5 YEAR; -SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 QUARTER = INTERVAL 6 NANOSECOND; -- { serverError NO_COMMON_TYPE } SELECT('Comparing years'); -SELECT toIntervalYear(1) < toIntervalYear(3); +SELECT INTERVAL 1 YEAR < INTERVAL 3 YEAR; -SELECT toIntervalYear(1) > toIntervalYear(3); +SELECT INTERVAL 1 YEAR > INTERVAL 3 YEAR; -SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError NO_COMMON_TYPE } +SELECT INTERVAL 2 YEAR = INTERVAL 8 SECOND; -- { serverError NO_COMMON_TYPE } \ No newline at end of file From eb3ffb71847fc5a204af31665a4594b0918fc1d7 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 15:09:16 +0000 Subject: [PATCH 0797/1722] Add supportsReplication --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 625b1281c61..f925cb773f5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2351,7 +2351,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, siz /// We don't control the amount of refs for temporary parts so we cannot decide can we remove blobs /// or not. So we are not doing it bool keep_shared = false; - if (disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication) + if (disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication && supportsReplication()) { LOG_WARNING(log, "Since zero-copy replication is enabled we are not going to remove blobs from shared storage for {}", full_path); keep_shared = true; From 897b8d5a88a69b8831ab489c2bea9d32d0cf06dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Aug 2024 15:21:01 +0000 Subject: [PATCH 0798/1722] Try to give more chances to `node2` to steal some work --- tests/integration/test_storage_s3_queue/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 8f197e09e61..00ef8499594 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1300,7 +1300,7 @@ where zookeeper_path ilike '%{table_name}%' and status = 'Processed' and rows_pr pytest.param("unordered", 1), pytest.param("unordered", 8), pytest.param("ordered", 1), - pytest.param("ordered", 8), + pytest.param("ordered", 2), ], ) def test_shards_distributed(started_cluster, mode, processing_threads): @@ -1311,7 +1311,7 @@ def test_shards_distributed(started_cluster, mode, processing_threads): keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 - row_num = 50 + row_num = 300 total_rows = row_num * files_to_generate shards_num = 2 From a39b9cf643bff565728be4083eb024ff5254f363 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 11 May 2024 13:05:24 +0000 Subject: [PATCH 0799/1722] Un-screw usearch's build description No directory 'SimSIMD-map' exists, the build only worked because SimSIMD support in usearch was (accidentally?) disabled. This commit corrects the build description. SimSIMD support in usearch will be enabled by a later commit. --- contrib/CMakeLists.txt | 2 +- contrib/usearch-cmake/CMakeLists.txt | 8 +++----- src/Storages/MergeTree/MergeTreeIndexUSearch.h | 1 - 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index b33e7083e32..98b992e1080 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -207,7 +207,7 @@ if (ARCH_S390X) endif() add_contrib (annoy-cmake annoy) -option(ENABLE_USEARCH "Enable USearch (Approximate Neighborhood Search, HNSW) support" ${ENABLE_LIBRARIES}) +option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES}) if (ENABLE_USEARCH) add_contrib (FP16-cmake FP16) add_contrib (robin-map-cmake robin-map) diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 29fbe57106c..0b6f60e106b 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -1,9 +1,7 @@ -set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") -set(USEARCH_SOURCE_DIR "${USEARCH_PROJECT_DIR}/include") - set(FP16_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/FP16") set(ROBIN_MAP_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/robin-map") -set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD-map") +set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") +set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") add_library(_usearch INTERFACE) @@ -11,7 +9,7 @@ target_include_directories(_usearch SYSTEM INTERFACE ${FP16_PROJECT_DIR}/include ${ROBIN_MAP_PROJECT_DIR}/include ${SIMSIMD_PROJECT_DIR}/include - ${USEARCH_SOURCE_DIR}) + ${USEARCH_PROJECT_DIR}/include) add_library(ch_contrib::usearch ALIAS _usearch) target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index 41de94402c9..e6068790d22 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -113,4 +113,3 @@ private: #endif - From d7211f9d12d33c54929fb24991fe7e46939be67d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 09:22:38 +0000 Subject: [PATCH 0800/1722] Fix CMake integration of usearch and annoy Registers usearch and annoy properly via configure_config.cmake and config.h.in like all other 3rd party libs, instead of (mis)using target_compile_definitions. --- contrib/annoy-cmake/CMakeLists.txt | 1 - contrib/usearch-cmake/CMakeLists.txt | 1 - src/Common/config.h.in | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 ++++-- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 +++- src/Storages/MergeTree/MergeTreeIndexUSearch.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeIndexUSearch.h | 8 ++++++-- src/Storages/MergeTree/MergeTreeIndices.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndices.h | 5 +++-- src/configure_config.cmake | 6 ++++++ 11 files changed, 31 insertions(+), 16 deletions(-) diff --git a/contrib/annoy-cmake/CMakeLists.txt b/contrib/annoy-cmake/CMakeLists.txt index bdef7d92132..f6579c12412 100644 --- a/contrib/annoy-cmake/CMakeLists.txt +++ b/contrib/annoy-cmake/CMakeLists.txt @@ -20,5 +20,4 @@ add_library(_annoy INTERFACE) target_include_directories(_annoy SYSTEM INTERFACE ${ANNOY_SOURCE_DIR}) add_library(ch_contrib::annoy ALIAS _annoy) -target_compile_definitions(_annoy INTERFACE ENABLE_ANNOY) target_compile_definitions(_annoy INTERFACE ANNOYLIB_MULTITHREADED_BUILD) diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 0b6f60e106b..6be622275ae 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -12,4 +12,3 @@ target_include_directories(_usearch SYSTEM INTERFACE ${USEARCH_PROJECT_DIR}/include) add_library(ch_contrib::usearch ALIAS _usearch) -target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index e3f8882850f..0fa5f4313b2 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -58,6 +58,8 @@ #cmakedefine01 USE_FILELOG #cmakedefine01 USE_ODBC #cmakedefine01 USE_BLAKE3 +#cmakedefine01 USE_ANNOY +#cmakedefine01 USE_USEARCH #cmakedefine01 USE_SKIM #cmakedefine01 USE_PRQL #cmakedefine01 USE_ULID diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 901d7c61167..0ec7bde933c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -52,6 +52,8 @@ #include #include +#include "config.h" + using namespace DB; namespace @@ -1476,11 +1478,11 @@ static void buildIndexes( MergeTreeIndexConditionPtr condition; if (index_helper->isVectorSearch()) { -#ifdef ENABLE_ANNOY +#if USE_ANNOY if (const auto * annoy = typeid_cast(index_helper.get())) condition = annoy->createIndexCondition(query_info, context); #endif -#ifdef ENABLE_USEARCH +#if USE_USEARCH if (const auto * usearch = typeid_cast(index_helper.get())) condition = usearch->createIndexCondition(query_info, context); #endif diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b68e48eeb3a..cec0e0926f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -1,7 +1,7 @@ -#ifdef ENABLE_ANNOY - #include +#if USE_ANNOY + #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 282920c608e..8e0e0e621a0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -1,6 +1,8 @@ #pragma once -#ifdef ENABLE_ANNOY +#include "config.h" + +#if USE_ANNOY #include diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index efd9bb754e1..5a532803d84 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -1,10 +1,10 @@ -#ifdef ENABLE_USEARCH +#include + +#if USE_USEARCH #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wpass-failed" -#include - #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h index e6068790d22..6923ef2f807 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -1,12 +1,16 @@ #pragma once -#ifdef ENABLE_USEARCH +#include "config.h" -#include +#if USE_USEARCH #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wpass-failed" + +#include + #include + #pragma clang diagnostic pop namespace DB diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index bded961db8e..32ac629e706 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -127,12 +127,12 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("hypothesis", hypothesisIndexCreator); registerValidator("hypothesis", hypothesisIndexValidator); -#ifdef ENABLE_ANNOY +#if USE_ANNOY registerCreator("annoy", annoyIndexCreator); registerValidator("annoy", annoyIndexValidator); #endif -#ifdef ENABLE_USEARCH +#if USE_USEARCH registerCreator("usearch", usearchIndexCreator); registerValidator("usearch", usearchIndexValidator); #endif diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 1be73e1c811..355f1b69356 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -15,6 +15,7 @@ #include #include +#include "config.h" constexpr auto INDEX_FILE_PREFIX = "skp_idx_"; @@ -230,12 +231,12 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach); MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); void hypothesisIndexValidator(const IndexDescription & index, bool attach); -#ifdef ENABLE_ANNOY +#if USE_ANNOY MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index); void annoyIndexValidator(const IndexDescription & index, bool attach); #endif -#ifdef ENABLE_USEARCH +#if USE_USEARCH MergeTreeIndexPtr usearchIndexCreator(const IndexDescription& index); void usearchIndexValidator(const IndexDescription& index, bool attach); #endif diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 5b24f79ef6e..702875b1f40 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -164,6 +164,12 @@ endif() if (TARGET ch_contrib::bcrypt) set(USE_BCRYPT 1) endif() +if (TARGET ch_contrib::annoy) + set(USE_ANNOY 1) +endif() +if (TARGET ch_contrib::usearch) + set(USE_USEARCH 1) +endif() if (TARGET ch_contrib::ssh) set(USE_SSH 1) endif() From 7c419399216a714f9dcffe7835f951718851bceb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 09:36:39 +0000 Subject: [PATCH 0801/1722] Fix test results (no analyzer support yet ...) --- tests/queries/0_stateless/02354_vector_search_queries.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 64051aa8544..87d27be0ea4 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -8,6 +8,8 @@ SET allow_experimental_annoy_index = 1; SET allow_experimental_usearch_index = 1; +SET enable_analyzer = 0; + SELECT 'ARRAY, 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block'; DROP TABLE IF EXISTS tab_annoy; From 218421c255cadbe65406e6a040d05942cc4efc3e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 09:47:50 +0000 Subject: [PATCH 0802/1722] Remove Annoy indexes Annoy indexes fell out of favor in the community, at least when it comes to vector databases. Such indexes work okay-ish low dimensions but they suffers badly from a curse of dimensionality which makes them inapt for a high number of dimensions. Now that Annoy is gone, issue (*) also disappears and we can drop 'no-ubsan', 'no-cpu-aarch64', and 'no-asan' from tests. (*) spotify/annoy#456 --- .gitmodules | 3 - contrib/CMakeLists.txt | 1 - contrib/annoy | 1 - contrib/annoy-cmake/CMakeLists.txt | 23 - .../mergetree-family/annindexes.md | 87 +--- src/CMakeLists.txt | 4 - src/Common/config.h.in | 1 - src/Core/Settings.h | 6 +- src/Databases/DatabaseReplicated.cpp | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 2 - src/Parsers/ASTIndexDeclaration.h | 1 - src/Parsers/ParserCreateIndexQuery.cpp | 4 +- src/Parsers/ParserCreateQuery.cpp | 4 +- .../QueryPlan/ReadFromMergeTree.cpp | 5 - .../MergeTree/MergeTreeIOSettings.cpp | 1 - src/Storages/MergeTree/MergeTreeIOSettings.h | 2 - .../MergeTree/MergeTreeIndexAnnoy.cpp | 416 ------------------ src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 114 ----- src/Storages/MergeTree/MergeTreeIndices.cpp | 4 - src/Storages/MergeTree/MergeTreeIndices.h | 5 - src/configure_config.cmake | 3 - .../02354_vector_search_bugs.reference | 10 - .../0_stateless/02354_vector_search_bugs.sql | 75 +--- ...ector_search_default_granularity.reference | 6 +- ...2354_vector_search_default_granularity.sql | 21 +- ...r_search_index_creation_negative.reference | 2 +- ..._vector_search_index_creation_negative.sql | 22 +- .../02354_vector_search_queries.reference | 99 ----- .../02354_vector_search_queries.sql | 119 +---- 29 files changed, 32 insertions(+), 1010 deletions(-) delete mode 160000 contrib/annoy delete mode 100644 contrib/annoy-cmake/CMakeLists.txt delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp delete mode 100644 src/Storages/MergeTree/MergeTreeIndexAnnoy.h diff --git a/.gitmodules b/.gitmodules index 7fdfb1103c5..0a66031de8d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -230,9 +230,6 @@ [submodule "contrib/minizip-ng"] path = contrib/minizip-ng url = https://github.com/zlib-ng/minizip-ng -[submodule "contrib/annoy"] - path = contrib/annoy - url = https://github.com/ClickHouse/annoy [submodule "contrib/qpl"] path = contrib/qpl url = https://github.com/intel/qpl diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 98b992e1080..dc2ad2a3150 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -205,7 +205,6 @@ add_contrib (morton-nd-cmake morton-nd) if (ARCH_S390X) add_contrib(crc32-s390x-cmake crc32-s390x) endif() -add_contrib (annoy-cmake annoy) option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES}) if (ENABLE_USEARCH) diff --git a/contrib/annoy b/contrib/annoy deleted file mode 160000 index f2ac8e7b48f..00000000000 --- a/contrib/annoy +++ /dev/null @@ -1 +0,0 @@ -Subproject commit f2ac8e7b48f9a9cf676d3b58286e5455aba8e956 diff --git a/contrib/annoy-cmake/CMakeLists.txt b/contrib/annoy-cmake/CMakeLists.txt deleted file mode 100644 index f6579c12412..00000000000 --- a/contrib/annoy-cmake/CMakeLists.txt +++ /dev/null @@ -1,23 +0,0 @@ -option(ENABLE_ANNOY "Enable Annoy index support" ${ENABLE_LIBRARIES}) - -# Annoy index should be disabled with undefined sanitizer. Because of memory storage optimizations -# (https://github.com/ClickHouse/annoy/blob/9d8a603a4cd252448589e84c9846f94368d5a289/src/annoylib.h#L442-L463) -# UBSan fails and leads to crash. Simmilar issue is already opened in Annoy repo -# https://github.com/spotify/annoy/issues/456 -# Problem with aligment can lead to errors like -# (https://stackoverflow.com/questions/46790550/c-undefined-behavior-strict-aliasing-rule-or-incorrect-alignment) -# or will lead to crash on arm https://developer.arm.com/documentation/ka003038/latest -# This issues should be resolved before annoy became non-experimental (--> setting "allow_experimental_annoy_index") -if ((NOT ENABLE_ANNOY) OR (SANITIZE STREQUAL "undefined") OR (ARCH_AARCH64)) - message (STATUS "Not using annoy") - return() -endif() - -set(ANNOY_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/annoy") -set(ANNOY_SOURCE_DIR "${ANNOY_PROJECT_DIR}/src") - -add_library(_annoy INTERFACE) -target_include_directories(_annoy SYSTEM INTERFACE ${ANNOY_SOURCE_DIR}) - -add_library(ch_contrib::annoy ALIAS _annoy) -target_compile_definitions(_annoy INTERFACE ANNOYLIB_MULTITHREADED_BUILD) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 5a81313f62e..9a80542522e 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -126,81 +126,8 @@ was specified for ANN indexes, the default value is 100 million. # Available ANN Indexes {#available_ann_indexes} -- [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) - - [USearch](/docs/en/engines/table-engines/mergetree-family/annindexes.md#usearch-usearch) -## Annoy {#annoy} - -Annoy indexes are currently experimental, to use them you first need to `SET allow_experimental_annoy_index = 1`. They are also currently -disabled on ARM due to memory safety problems with the algorithm. - -This type of ANN index is based on the [Annoy library](https://github.com/spotify/annoy) which recursively divides the space into random -linear surfaces (lines in 2D, planes in 3D etc.). - -
- -
- -Syntax to create an Annoy index over an [Array(Float32)](../../../sql-reference/data-types/array.md) column: - -```sql -CREATE TABLE table_with_annoy_index -( - id Int64, - vectors Array(Float32), - INDEX [ann_index_name] vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] -) -ENGINE = MergeTree -ORDER BY id; -``` - -Annoy currently supports two distance functions: -- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space - ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). -- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors - ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). - -For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no -distance function was specified during index creation, `L2Distance` is used as default. - -Parameter `NumTrees` is the number of trees which the algorithm creates (default if not specified: 100). Higher values of `NumTree` mean -more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. - -:::note -All arrays must have same length. To avoid errors, you can use a -[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK -length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported. -::: - -The creation of Annoy indexes (whenever a new part is build, e.g. at the end of a merge) is a relatively slow process. You can increase -setting `max_threads_for_annoy_index_creation` (default: 4) which controls how many threads are used to create an Annoy index. Please be -careful with this setting, it is possible that multiple indexes are created in parallel in which case there can be overparallelization. - -Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger -values mean more accurate results at the cost of longer query runtime: - -```sql -SELECT * -FROM table_name -ORDER BY L2Distance(vectors, Point) -LIMIT N -SETTINGS annoy_index_search_k_nodes=100; -``` - -:::note -The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see -[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. -::: - ## USearch {#usearch} This type of ANN index is based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW @@ -211,6 +138,8 @@ that are expensive to load and compare. The library also has several hardware-sp distance computations on modern Arm (NEON and SVE) and x86 (AVX2 and AVX-512) CPUs and OS-specific optimizations to allow efficient navigation around immutable persistent files, without loading them into RAM. +USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_usearch_index = 1`. +
-
- -Syntax to create an USearch index over an [Array](../../../sql-reference/data-types/array.md) column: - -```sql -CREATE TABLE table_with_usearch_index -( - id Int64, - vectors Array(Float32), - INDEX [ann_index_name] vectors TYPE usearch([Distance[, ScalarKind]]) [GRANULARITY N] -) -ENGINE = MergeTree -ORDER BY id; -``` - -USearch currently supports two distance functions: -- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space - ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). -- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors - ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). - -USearch allows storing the vectors in reduced precision formats. Supported scalar kinds are `f64`, `f32`, `f16` or `i8`. If no scalar kind -was specified during index creation, `f16` is used as default. - -For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no -distance function was specified during index creation, `L2Distance` is used as default. - -:::note -All arrays must have same length. To avoid errors, you can use a -[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK -length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported. -::: - -:::note -The USearch index currently does not work with per-table, non-default `index_granularity` settings (see -[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. -::: - diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 23ad12bb017..e9f3b95dbc1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -907,9 +907,9 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_time_series_table, false, "Allows experimental TimeSeries table engine", 0) \ + M(Bool, allow_experimental_vector_similarity_index, false, "Allow experimental vector similarity index", 0) \ M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ - M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ @@ -1036,6 +1036,7 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_annoy_index, false) \ MAKE_OBSOLETE(M, UInt64, max_threads_for_annoy_index_creation, 4) \ MAKE_OBSOLETE(M, Int64, annoy_index_search_k_nodes, -1) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_usearch_index, false) \ MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \ MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 511723f1873..8fabd1ecf91 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,6 +85,7 @@ static std::initializer_listsetSetting("allow_experimental_object_type", 1); query_context->setSetting("allow_experimental_variant_type", 1); query_context->setSetting("allow_experimental_dynamic_type", 1); - query_context->setSetting("allow_experimental_usearch_index", 1); + query_context->setSetting("allow_experimental_vector_similarity_index", 1); query_context->setSetting("allow_experimental_bigint_types", 1); query_context->setSetting("allow_experimental_window_functions", 1); query_context->setSetting("allow_experimental_geo_types", 1); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a1ffcf07588..95143031707 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -787,8 +787,8 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Please use index type 'full_text' instead of 'inverted'"); /// ---- - if (index_desc.type == "usearch" && !settings.allow_experimental_usearch_index) - throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index is disabled. Turn on allow_experimental_usearch_index"); + if (index_desc.type == "vector_similarity" && !settings.allow_experimental_vector_similarity_index) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index is disabled. Turn on allow_experimental_vector_similarity_index"); properties.indices.push_back(index_desc); } diff --git a/src/Parsers/ASTIndexDeclaration.h b/src/Parsers/ASTIndexDeclaration.h index 90645f12b7c..72f3f017a99 100644 --- a/src/Parsers/ASTIndexDeclaration.h +++ b/src/Parsers/ASTIndexDeclaration.h @@ -13,7 +13,7 @@ class ASTIndexDeclaration : public IAST { public: static const auto DEFAULT_INDEX_GRANULARITY = 1uz; - static const auto DEFAULT_USEARCH_INDEX_GRANULARITY = 100'000'000uz; + static const auto DEFAULT_VECTOR_SIMILARITY_INDEX_GRANULARITY = 100'000'000uz; ASTIndexDeclaration(ASTPtr expression, ASTPtr type, const String & name_); diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index e7cfd753f99..ed89b80edca 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -89,8 +89,8 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected else { auto index_type = index->getType(); - if (index_type && index_type->name == "usearch") - index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY; + if (index_type && index_type->name == "vector_similarity") + index->granularity = ASTIndexDeclaration::DEFAULT_VECTOR_SIMILARITY_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index b31fe21c4cc..cc4e02f46a3 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -214,8 +214,8 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe else { auto index_type = index->getType(); - if (index_type->name == "usearch") - index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY; + if (index_type->name == "vector_similarity") + index->granularity = ASTIndexDeclaration::DEFAULT_VECTOR_SIMILARITY_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 3324cc4e42a..1f30725b4d0 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -24,7 +24,7 @@ #include #include #include -#include +#include #include #include #include @@ -1478,8 +1478,8 @@ static void buildIndexes( if (index_helper->isVectorSimilarityIndex()) { #if USE_USEARCH - if (const auto * usearch_index = typeid_cast(index_helper.get())) - condition = usearch_index->createIndexCondition(query_info, context); + if (const auto * vector_similarity_index = typeid_cast(index_helper.get())) + condition = vector_similarity_index->createIndexCondition(query_info, context); #endif if (!condition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp similarity index 76% rename from src/Storages/MergeTree/MergeTreeIndexUSearch.cpp rename to src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 1aa6c9c14d4..6f3b1b043cd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_USEARCH @@ -90,7 +90,7 @@ void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const auto result = Base::save_to_stream(callback); if (result.error) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save USearch index, error: " + String(result.error.release())); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index, error: " + String(result.error.release())); } void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) @@ -104,7 +104,7 @@ void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) auto result = Base::load_from_stream(callback); if (result.error) /// See the comment in MergeTreeIndexGranuleVectorSimilarity::deserializeBinary why we throw here - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load USearch index, error: " + String(result.error.release()) + " Please drop the index and create it again."); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index, error: " + String(result.error.release()) + " Please drop the index and create it again."); } USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStatistics() const @@ -121,16 +121,16 @@ USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStat return statistics; } -MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( +MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) - : MergeTreeIndexGranuleUSearch(index_name_, index_sample_block_, metric_kind_, scalar_kind_, nullptr) + : MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, nullptr) { } -MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( +MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, @@ -144,7 +144,7 @@ MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( { } -void MergeTreeIndexGranuleUSearch::serializeBinary(WriteBuffer & ostr) const +void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) const { if (empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty minmax index {}", backQuote(index_name)); @@ -158,18 +158,18 @@ void MergeTreeIndexGranuleUSearch::serializeBinary(WriteBuffer & ostr) const index->serialize(ostr); auto statistics = index->getStatistics(); - LOG_TRACE(logger, "Wrote USearch index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", + LOG_TRACE(logger, "Wrote vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage)); } -void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) +void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) { UInt64 file_version; readIntBinary(file_version, istr); if (file_version != FILE_FORMAT_VERSION) throw Exception( ErrorCodes::FORMAT_VERSION_TOO_OLD, - "USearch index could not be loaded because its version is too old (current version: {}, persisted version: {}). Please drop the index and create it again.", + "Vector similarity index could not be loaded because its version is too old (current version: {}, persisted version: {}). Please drop the index and create it again.", FILE_FORMAT_VERSION, file_version); /// More fancy error handling would be: Set a flag on the index that it failed to load. During usage return all granules, i.e. /// behave as if the index does not exist. Since format changes are expected to happen only rarely and it is "only" an index, keep it simple for now. @@ -181,11 +181,11 @@ void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, MergeTre index->deserialize(istr); auto statistics = index->getStatistics(); - LOG_TRACE(logger, "Loaded USearch index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", + LOG_TRACE(logger, "Loaded vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage)); } -MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch( +MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, @@ -197,14 +197,14 @@ MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch( { } -MergeTreeIndexGranulePtr MergeTreeIndexAggregatorUSearch::getGranuleAndReset() +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorVectorSimilarity::getGranuleAndReset() { - auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, index); + auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, index); index = nullptr; return granule; } -void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, size_t limit) +void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_t * pos, size_t limit) { if (*pos >= block.rows()) throw Exception( @@ -239,8 +239,8 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, if (column_array->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); - /// The Usearch algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays - /// are INSERTed into an Usearch-indexed column or if no value was specified at all in which case the arrays take on their default + /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays + /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default /// values which is also empty. if (column_array->isDefaultAt(0)) throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); @@ -262,13 +262,13 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, /// Reserving space is mandatory if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) - throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); for (size_t current_row = 0; current_row < num_rows; ++current_row) { auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]); if (!rc) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to USearch index, error: " + String(rc.error.release())); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release())); ProfileEvents::increment(ProfileEvents::USearchAddCount); ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members); @@ -281,7 +281,7 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, *pos += rows_read; } -MergeTreeIndexConditionUSearch::MergeTreeIndexConditionUSearch( +MergeTreeIndexConditionVectorSimilarity::MergeTreeIndexConditionVectorSimilarity( const IndexDescription & /*index_description*/, const SelectQueryInfo & query, unum::usearch::metric_kind_t metric_kind_, @@ -291,12 +291,12 @@ MergeTreeIndexConditionUSearch::MergeTreeIndexConditionUSearch( { } -bool MergeTreeIndexConditionUSearch::mayBeTrueOnGranule(MergeTreeIndexGranulePtr) const +bool MergeTreeIndexConditionVectorSimilarity::mayBeTrueOnGranule(MergeTreeIndexGranulePtr) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); } -bool MergeTreeIndexConditionUSearch::alwaysUnknownOrTrue() const +bool MergeTreeIndexConditionVectorSimilarity::alwaysUnknownOrTrue() const { String index_distance_function; switch (metric_kind) @@ -308,14 +308,14 @@ bool MergeTreeIndexConditionUSearch::alwaysUnknownOrTrue() const return vector_similarity_condition.alwaysUnknownOrTrue(index_distance_function); } -std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeIndexGranulePtr granule_) const +std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(MergeTreeIndexGranulePtr granule_) const { const UInt64 limit = vector_similarity_condition.getLimit(); const UInt64 index_granularity = vector_similarity_condition.getIndexGranularity(); const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); - const auto granule = std::dynamic_pointer_cast(granule_); + const auto granule = std::dynamic_pointer_cast(granule_); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); @@ -328,7 +328,7 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeInd auto result = index->search(reference_vector.data(), limit); if (result.error) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in USearch index, error: " + String(result.error.release())); + throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release())); ProfileEvents::increment(ProfileEvents::USearchSearchCount); ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members); @@ -350,34 +350,34 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeInd return granules; } -MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) +MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity(const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) : IMergeTreeIndex(index_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) { } -MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const +MergeTreeIndexGranulePtr MergeTreeIndexVectorSimilarity::createIndexGranule() const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); } -MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const +MergeTreeIndexAggregatorPtr MergeTreeIndexVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); } -MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(index, query, metric_kind, context); + return std::make_shared(index, query, metric_kind, context); }; -MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAG *, ContextPtr) const +MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(const ActionsDAG *, ContextPtr) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG"); } -MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) +MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { static constexpr auto default_metric_kind = unum::usearch::metric_kind_t::l2sq_k; auto metric_kind = default_metric_kind; @@ -389,25 +389,25 @@ MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) if (index.arguments.size() > 1) scalar_kind = quantizationToScalarKind.at(index.arguments[1].safeGet()); - return std::make_shared(index, metric_kind, scalar_kind); + return std::make_shared(index, metric_kind, scalar_kind); } -void usearchIndexValidator(const IndexDescription & index, bool /* attach */) +void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* attach */) { - /// Check number and type of USearch index arguments: + /// Check number and type of index arguments: if (index.arguments.size() > 2) - throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index must not have more than one parameters"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must not have more than one parameters"); if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of USearch index (distance function) must be of type String"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (distance function) must be of type String"); if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of USearch index (scalar type) must be of type String"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (scalar type) must be of type String"); /// Check that the index is created on a single column if (index.column_names.size() != 1 || index.data_types.size() != 1) - throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "USearch indexes must be created on a single column"); + throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column"); /// Check that a supported metric was passed as first argument @@ -420,16 +420,15 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized scalar kind (second argument) for vector index. Supported kinds are: {}", keysAsString(quantizationToScalarKind)); /// Check data type of indexed column: - DataTypePtr data_type = index.sample_block.getDataTypes()[0]; if (const auto * data_type_array = typeid_cast(data_type.get())) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "USearch can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "USearch can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexUSearch.h b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h similarity index 84% rename from src/Storages/MergeTree/MergeTreeIndexUSearch.h rename to src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h index d4df6658a90..95ea3cd5240 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUSearch.h +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h @@ -48,22 +48,22 @@ public: using USearchIndexWithSerializationPtr = std::shared_ptr; -struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranule { - MergeTreeIndexGranuleUSearch( + MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_); - MergeTreeIndexGranuleUSearch( + MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_, USearchIndexWithSerializationPtr index_); - ~MergeTreeIndexGranuleUSearch() override = default; + ~MergeTreeIndexGranuleVectorSimilarity() override = default; void serializeBinary(WriteBuffer & ostr) const override; void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; @@ -76,7 +76,7 @@ struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule const unum::usearch::scalar_kind_t scalar_kind; USearchIndexWithSerializationPtr index; - LoggerPtr logger = getLogger("USearchIndex"); + LoggerPtr logger = getLogger("VectorSimilarityIndex"); private: /// The version of the persistence format of USearch index. Increment whenever you change the format. @@ -87,15 +87,15 @@ private: }; -struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorVectorSimilarity final : IMergeTreeIndexAggregator { - MergeTreeIndexAggregatorUSearch( + MergeTreeIndexAggregatorVectorSimilarity( const String & index_name_, const Block & index_sample_block, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_); - ~MergeTreeIndexAggregatorUSearch() override = default; + ~MergeTreeIndexAggregatorVectorSimilarity() override = default; bool empty() const override { return !index || index->size() == 0; } MergeTreeIndexGranulePtr getGranuleAndReset() override; @@ -109,16 +109,16 @@ struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator }; -class MergeTreeIndexConditionUSearch final : public IMergeTreeIndexCondition +class MergeTreeIndexConditionVectorSimilarity final : public IMergeTreeIndexCondition { public: - MergeTreeIndexConditionUSearch( + MergeTreeIndexConditionVectorSimilarity( const IndexDescription & index_description, const SelectQueryInfo & query, unum::usearch::metric_kind_t metric_kind_, ContextPtr context); - ~MergeTreeIndexConditionUSearch() override = default; + ~MergeTreeIndexConditionVectorSimilarity() override = default; bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const override; @@ -130,15 +130,15 @@ private: }; -class MergeTreeIndexUSearch : public IMergeTreeIndex +class MergeTreeIndexVectorSimilarity : public IMergeTreeIndex { public: - MergeTreeIndexUSearch( + MergeTreeIndexVectorSimilarity( const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_); - ~MergeTreeIndexUSearch() override = default; + ~MergeTreeIndexVectorSimilarity() override = default; MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index f07449f762c..89aed7873a4 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -129,8 +129,8 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerValidator("hypothesis", hypothesisIndexValidator); #if USE_USEARCH - registerCreator("usearch", usearchIndexCreator); - registerValidator("usearch", usearchIndexValidator); + registerCreator("vector_similarity", vectorSimilarityIndexCreator); + registerValidator("vector_similarity", vectorSimilarityIndexValidator); #endif registerCreator("inverted", fullTextIndexCreator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 3dee79aae85..48ef2a4739e 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -239,8 +239,8 @@ MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); void hypothesisIndexValidator(const IndexDescription & index, bool attach); #if USE_USEARCH -MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index); -void usearchIndexValidator(const IndexDescription & index, bool attach); +MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index); +void vectorSimilarityIndexValidator(const IndexDescription & index, bool attach); #endif MergeTreeIndexPtr fullTextIndexCreator(const IndexDescription & index); diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index de36683ede1..2ef75d0a7fe 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -2,21 +2,21 @@ -- Tests various bugs and special cases for vector indexes. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof DROP TABLE IF EXISTS tab; SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected'; -CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree() ORDER BY id; INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA } INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA } DROP TABLE tab; SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; SYSTEM STOP MERGES tab; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]); INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]); @@ -31,7 +31,7 @@ DROP TABLE tab; SELECT 'Correctness of index with > 1 mark'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000); WITH [1.0, 0.0] AS reference_vec diff --git a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql index ff659b56033..a19a0d17536 100644 --- a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql +++ b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql @@ -2,17 +2,17 @@ -- Tests that vector search indexes use a (non-standard) index granularity of 100 mio by default. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; -- After CREATE TABLE DROP TABLE IF EXISTS tab; -CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE usearch) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; -- After ALTER TABLE DROP TABLE tab; CREATE TABLE tab (id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; -ALTER TABLE tab ADD INDEX idx(vec) TYPE usearch; +ALTER TABLE tab ADD INDEX idx(vec) TYPE vector_similarity; SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql index 92e8efd918b..36241dfabf7 100644 --- a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql +++ b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql @@ -2,10 +2,10 @@ -- Tests that vector similarity indexes can be detached/attached. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); DETACH TABLE tab SYNC; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index 60bd54d1dbe..912f7d7fcae 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -2,36 +2,36 @@ -- Tests that various conditions are checked during creation of vector search indexes. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; SELECT 'At most two index arguments'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } SELECT '1st argument (distance function) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } SELECT 'Unsupported distance functions are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT '2nd argument (scalar kind) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } SELECT 'Unsupported scalar kinds are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('L2Distance', 'invalidKind')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('L2Distance', 'invalidKind')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT 'Must be created on single column'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } SELECT 'Must be created on Array(Float32) columns'; SET allow_suspicious_low_cardinality_types = 1; -CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } SELECT 'Rejects INSERTs of Arrays with different sizes'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index 22ad46f802c..7c8e4c0ca59 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -1,9 +1,9 @@ 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block -- Usearch: ORDER-BY-type +- ORDER-BY-type 5 [0,2] 0 6 [0,2.1] 0.09999990463256836 7 [0,2.2] 0.20000004768371582 -- Usearch: ORDER-BY-type, EXPLAIN +- ORDER-BY-type, EXPLAIN Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -16,15 +16,15 @@ Expression (Projection) Granules: 1/1 Skip Name: idx - Description: usearch GRANULARITY 100000000 + Description: vector_similarity GRANULARITY 100000000 Parts: 1/1 Granules: 1/1 12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block -- Usearch: ORDER-BY-type +- ORDER-BY-type 6 [0,2] 0 7 [0,2.1] 0.09999990463256836 8 [0,2.2] 0.20000004768371582 -- Usearch: ORDER-BY-type, EXPLAIN +- ORDER-BY-type, EXPLAIN Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -37,11 +37,11 @@ Expression (Projection) Granules: 4/4 Skip Name: idx - Description: usearch GRANULARITY 2 + Description: vector_similarity GRANULARITY 2 Parts: 1/1 Granules: 2/4 Special cases -- Usearch: ORDER-BY-type +- ORDER-BY-type 6 [1,9.3] 0.005731362878640178 1 [2,3.2] 0.15200169244542905 7 [5.5,4.7] 0.3503476876550442 diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 555f47b364f..50537ad6244 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -2,7 +2,7 @@ -- Tests various simple approximate nearest neighborhood (ANN) queries that utilize vector search indexes. -SET allow_experimental_usearch_index = 1; +SET allow_experimental_vector_similarity_index = 1; SET enable_analyzer = 0; @@ -10,18 +10,18 @@ SELECT '10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -SELECT '- Usearch: ORDER-BY-type'; +SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -SELECT '- Usearch: ORDER-BY-type, EXPLAIN'; +SELECT '- ORDER-BY-type, EXPLAIN'; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) @@ -34,17 +34,17 @@ DROP TABLE tab; SELECT '12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]); -SELECT '- Usearch: ORDER-BY-type'; +SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -SELECT '- Usearch: ORDER-BY-type, EXPLAIN'; +SELECT '- ORDER-BY-type, EXPLAIN'; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) @@ -58,10 +58,10 @@ DROP TABLE tab; SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen. -- Just for jun, use metric = 'cosineDistance', scalarKind = 'f64' -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE usearch('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -SELECT '- Usearch: ORDER-BY-type'; +SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, cosineDistance(vec, reference_vec) FROM tab From cc5c64e1ede7284d91ada1f28edbb18a457f5894 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 19:48:51 +0000 Subject: [PATCH 0829/1722] Add migration helper for legacy 'annoy' and 'usearch' indexes types Index types 'annoy' and 'usearch' were removed and replaced by 'vector_similarity' indexes in an earlier commit. This means unfortuantely, that if customers have tables with these indexes and upgrade, their database might not start anymore - the system loads the metadata at startup, thinks something is wrong with such tables, and halts immediately. This commit adds support for loading and attaching such indexes back. Data insert or use (search) return an error which recommends a migration to 'vector_similarity' indexes. The implementation is generally similar to what has recently been implemented for 'full_text' indexes [1, 2]. [1] https://github.com/ClickHouse/ClickHouse/pull/64656 [2] https://github.com/ClickHouse/ClickHouse/pull/64846 --- .../QueryPlan/ReadFromMergeTree.cpp | 3 ++ .../MergeTreeIndexLegacyVectorSimilarity.cpp | 45 +++++++++++++++++++ .../MergeTreeIndexLegacyVectorSimilarity.h | 26 +++++++++++ src/Storages/MergeTree/MergeTreeIndices.cpp | 10 +++++ src/Storages/MergeTree/MergeTreeIndices.h | 3 ++ ...earch_legacy_index_compatibility.reference | 2 + ...ctor_search_legacy_index_compatibility.sql | 43 ++++++++++++++++++ 7 files changed, 132 insertions(+) create mode 100644 src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp create mode 100644 src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h create mode 100644 tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1f30725b4d0..348019d7d10 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -1481,6 +1482,8 @@ static void buildIndexes( if (const auto * vector_similarity_index = typeid_cast(index_helper.get())) condition = vector_similarity_index->createIndexCondition(query_info, context); #endif + if (const auto * legacy_vector_similarity_index = typeid_cast(index_helper.get())) + condition = legacy_vector_similarity_index->createIndexCondition(query_info, context); if (!condition) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); } diff --git a/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp new file mode 100644 index 00000000000..29de109d4fc --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.cpp @@ -0,0 +1,45 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_INDEX; +} + +MergeTreeIndexLegacyVectorSimilarity::MergeTreeIndexLegacyVectorSimilarity(const IndexDescription & index_) + : IMergeTreeIndex(index_) +{ +} + +MergeTreeIndexGranulePtr MergeTreeIndexLegacyVectorSimilarity::createIndexGranule() const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +} + +MergeTreeIndexAggregatorPtr MergeTreeIndexLegacyVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings &) const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +} + +MergeTreeIndexConditionPtr MergeTreeIndexLegacyVectorSimilarity::createIndexCondition(const SelectQueryInfo &, ContextPtr) const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +}; + +MergeTreeIndexConditionPtr MergeTreeIndexLegacyVectorSimilarity::createIndexCondition(const ActionsDAG *, ContextPtr) const +{ + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indexes of type 'annoy' or 'usearch' are no longer supported. Please drop and recreate the index as type 'vector_similarity'"); +} + +MergeTreeIndexPtr legacyVectorSimilarityIndexCreator(const IndexDescription & index) +{ + return std::make_shared(index); +} + +void legacyVectorSimilarityIndexValidator(const IndexDescription &, bool) +{ +} + +} diff --git a/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h new file mode 100644 index 00000000000..1015401823d --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexLegacyVectorSimilarity.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +/// Walking corpse implementation for removed skipping index of type "annoy" and "usearch". +/// Its only purpose is to allow loading old tables with indexes of these types. +/// Data insertion and index usage/search will throw an exception, suggesting to migrate to "vector_similarity" indexes. + +namespace DB +{ + +class MergeTreeIndexLegacyVectorSimilarity : public IMergeTreeIndex +{ +public: + explicit MergeTreeIndexLegacyVectorSimilarity(const IndexDescription & index_); + ~MergeTreeIndexLegacyVectorSimilarity() override = default; + + MergeTreeIndexGranulePtr createIndexGranule() const override; + MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings &) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo &, ContextPtr) const; + MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override; + + bool isVectorSimilarityIndex() const override { return true; } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index 89aed7873a4..d2fc0e84b56 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -132,6 +132,16 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("vector_similarity", vectorSimilarityIndexCreator); registerValidator("vector_similarity", vectorSimilarityIndexValidator); #endif + /// ------ + /// TODO: remove this block at the end of 2024. + /// Index types 'annoy' and 'usearch' are no longer supported as of June 2024. Their successor is index type 'vector_similarity'. + /// To support loading tables with old indexes during a transition period, register dummy indexes which allow load/attaching but + /// throw an exception when the user attempts to use them. + registerCreator("annoy", legacyVectorSimilarityIndexCreator); + registerValidator("annoy", legacyVectorSimilarityIndexValidator); + registerCreator("usearch", legacyVectorSimilarityIndexCreator); + registerValidator("usearch", legacyVectorSimilarityIndexValidator); + /// ------ registerCreator("inverted", fullTextIndexCreator); registerValidator("inverted", fullTextIndexValidator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 48ef2a4739e..c52d7ffe131 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -243,6 +243,9 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index); void vectorSimilarityIndexValidator(const IndexDescription & index, bool attach); #endif +MergeTreeIndexPtr legacyVectorSimilarityIndexCreator(const IndexDescription & index); +void legacyVectorSimilarityIndexValidator(const IndexDescription & index, bool attach); + MergeTreeIndexPtr fullTextIndexCreator(const IndexDescription & index); void fullTextIndexValidator(const IndexDescription & index, bool attach); diff --git a/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference new file mode 100644 index 00000000000..030bfa9b1bd --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.reference @@ -0,0 +1,2 @@ +Annoy +Usearch diff --git a/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql new file mode 100644 index 00000000000..0889aa74f7a --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_legacy_index_compatibility.sql @@ -0,0 +1,43 @@ +-- Indexes of type 'annoy' or 'usearch' are no longer supported. +-- Test what happens when ClickHouse encounters tables with the old index type. + +DROP TABLE IF EXISTS tab; + +SELECT 'Annoy'; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX vec_idx vec TYPE annoy()) ENGINE = MergeTree ORDER BY id; + +INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -- { serverError ILLEGAL_INDEX } + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; +-- (*) The search succeeds because the index contains no data (i.e. some shortcut) +-- If it had data (can't really test in SQL tests ...), this statement would also return an error, trust me. + +-- Detach and attach should work. +DETACH TABLE tab; +ATTACH TABLE tab; + +DROP TABLE tab; + +SELECT 'Usearch'; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX vec_idx vec TYPE usearch()) ENGINE = MergeTree ORDER BY id; + +INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -- { serverError ILLEGAL_INDEX } + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; +-- see above: (*) + +-- Detach and attach should work. +DETACH TABLE tab; +ATTACH TABLE tab; + +DROP TABLE tab; From d2e79f0b92936eb3ec3f6409fe6db18a3091919d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Aug 2024 15:28:38 +0000 Subject: [PATCH 0830/1722] Rework vector index parameters USearch (similar to FAISS) allows to specify the distance function, quantization, and various HNSW meta-parameters for index creation and sarch. Some users wished for greater configurability, so let's expose them. Index creation now requires either - 2 parameters (with the other 4 parameters taking on default values), or - 6 parameters for full control This commit also remove quantization `f64` (that would be upsampling). --- .../mergetree-family/annindexes.md | 12 +- .../MergeTreeIndexVectorSimilarity.cpp | 166 ++++++++++++------ .../MergeTreeIndexVectorSimilarity.h | 23 ++- .../0_stateless/02354_vector_search_bugs.sql | 6 +- ...2354_vector_search_default_granularity.sql | 4 +- .../02354_vector_search_detach_attach.sql | 2 +- ...r_search_index_creation_negative.reference | 12 +- ..._vector_search_index_creation_negative.sql | 48 +++-- ...4_vector_search_multiple_indexes.reference | 0 .../02354_vector_search_multiple_indexes.sql | 14 ++ .../02354_vector_search_queries.sql | 8 +- 11 files changed, 203 insertions(+), 92 deletions(-) create mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 63c061a0d46..354fac6ea74 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -43,12 +43,22 @@ CREATE TABLE table ( id Int64, vectors Array(Float32), - INDEX [index_name vectors TYPE vector_similarity([Distance[, ScalarKind]]) [GRANULARITY [N]] + INDEX index_name vec TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; ``` +Parameters: +- `method`: Supports currently only `hnsw`. +- `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a + line between two points in Euclidean space), or `cosineDistance` (the [cosine + distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors). +- `quantization`: either `f32`, `f16`, or `i8` for storing the vector with reduced precision (optional, default: `f32`) +- `m`: the number of neighbors per graph node (optional, default: 16) +- `ef_construction`: (optional, default: 128) +- `ef_search`: (optional, default: 64) + Vector similarity indexes are based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW algorithm](https://arxiv.org/abs/1603.09320), i.e., a hierarchical graph where each point represents a vector and the edges represent similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from the diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 6f3b1b043cd..5b0793fa0c8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -45,6 +45,9 @@ namespace ErrorCodes namespace { +/// The only indexing method currently supported by USearch +std::set methods = {"hnsw"}; + /// Maps from user-facing name to internal name std::unordered_map distanceFunctionToMetricKind = { {"L2Distance", unum::usearch::metric_kind_t::l2sq_k}, @@ -52,22 +55,37 @@ std::unordered_map distanceFunctionToMetri /// Maps from user-facing name to internal name std::unordered_map quantizationToScalarKind = { - {"f64", unum::usearch::scalar_kind_t::f64_k}, {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; +template +concept is_set = std::same_as>; + +template +concept is_unordered_map = std::same_as>; + template -String keysAsString(const T & t) +String joinByComma(const T & t) { - String result; - for (const auto & [k, _] : t) + if constexpr (is_set) { - if (!result.empty()) - result += ", "; - result += k; + return fmt::format("{}", fmt::join(t, ", ")); } - return result; + else if constexpr (is_unordered_map) + { + String joined_keys; + for (const auto & [k, _] : t) + { + if (!joined_keys.empty()) + joined_keys += ", "; + joined_keys += k; + } + return joined_keys; + } + /// TODO once our libcxx is recent enough, replace above by + /// return fmt::format("{}", fmt::join(std::views::keys(t)), ", ")); + std::unreachable(); } } @@ -75,8 +93,10 @@ String keysAsString(const T & t) USearchIndexWithSerialization::USearchIndexWithSerialization( size_t dimensions, unum::usearch::metric_kind_t metric_kind, - unum::usearch::scalar_kind_t scalar_kind) - : Base(Base::make(unum::usearch::metric_punned_t(dimensions, metric_kind, scalar_kind))) + unum::usearch::scalar_kind_t scalar_kind, + UsearchHnswParams usearch_hnsw_params) + : Base(Base::make(unum::usearch::metric_punned_t(dimensions, metric_kind, scalar_kind), + unum::usearch::index_dense_config_t(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search))) { } @@ -125,8 +145,9 @@ MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_) - : MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, nullptr) + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_) + : MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, usearch_hnsw_params_, nullptr) { } @@ -135,11 +156,13 @@ MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_, USearchIndexWithSerializationPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) + , usearch_hnsw_params(usearch_hnsw_params_) , index(std::move(index_)) { } @@ -153,8 +176,8 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) /// Number of dimensions is required in the index constructor, /// so it must be written and read separately from the other part - writeIntBinary(static_cast(index->dimensions()), ostr); // write dimension - // + writeIntBinary(static_cast(index->dimensions()), ostr); + index->serialize(ostr); auto statistics = index->getStatistics(); @@ -176,7 +199,7 @@ void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, UInt64 dimension; readIntBinary(dimension, istr); - index = std::make_shared(dimension, metric_kind, scalar_kind); + index = std::make_shared(dimension, metric_kind, scalar_kind, usearch_hnsw_params); index->deserialize(istr); @@ -189,17 +212,19 @@ MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilari const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_) + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_) : index_name(index_name_) , index_sample_block(index_sample_block_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) + , usearch_hnsw_params(usearch_hnsw_params_) { } MergeTreeIndexGranulePtr MergeTreeIndexAggregatorVectorSimilarity::getGranuleAndReset() { - auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, index); + auto granule = std::make_shared(index_name, index_sample_block, metric_kind, scalar_kind, usearch_hnsw_params, index); index = nullptr; return granule; } @@ -258,15 +283,15 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); if (!index) - index = std::make_shared(dimensions, metric_kind, scalar_kind); + index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); /// Reserving space is mandatory if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); - for (size_t current_row = 0; current_row < num_rows; ++current_row) + for (size_t row = 0; row < num_rows; ++row) { - auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[current_row - 1]]); + auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); if (!rc) throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release())); @@ -313,8 +338,6 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer const UInt64 limit = vector_similarity_condition.getLimit(); const UInt64 index_granularity = vector_similarity_condition.getIndexGranularity(); - const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); - const auto granule = std::dynamic_pointer_cast(granule_); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); @@ -326,6 +349,8 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer "does not match the dimension in the index ({})", vector_similarity_condition.getDimensions(), index->dimensions()); + const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); + auto result = index->search(reference_vector.data(), limit); if (result.error) throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release())); @@ -350,21 +375,26 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer return granules; } -MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity(const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_) +MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity( + const IndexDescription & index_, + unum::usearch::metric_kind_t metric_kind_, + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_) : IMergeTreeIndex(index_) , metric_kind(metric_kind_) , scalar_kind(scalar_kind_) + , usearch_hnsw_params(usearch_hnsw_params_) { } MergeTreeIndexGranulePtr MergeTreeIndexVectorSimilarity::createIndexGranule() const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind, usearch_hnsw_params); } MergeTreeIndexAggregatorPtr MergeTreeIndexVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const { - return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind); + return std::make_shared(index.name, index.sample_block, metric_kind, scalar_kind, usearch_hnsw_params); } MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const @@ -379,56 +409,82 @@ MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition( MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { - static constexpr auto default_metric_kind = unum::usearch::metric_kind_t::l2sq_k; - auto metric_kind = default_metric_kind; - if (!index.arguments.empty()) - metric_kind = distanceFunctionToMetricKind.at(index.arguments[0].safeGet()); + const bool has_six_args = (index.arguments.size() == 6); - static constexpr auto default_scalar_kind = unum::usearch::scalar_kind_t::f16_k; - auto scalar_kind = default_scalar_kind; - if (index.arguments.size() > 1) - scalar_kind = quantizationToScalarKind.at(index.arguments[1].safeGet()); + unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet()); - return std::make_shared(index, metric_kind, scalar_kind); + /// use defaults for the other parameters + unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k; + UsearchHnswParams usearch_hnsw_params; + + if (has_six_args) + { + scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet()); + usearch_hnsw_params = {.m = index.arguments[3].safeGet(), + .ef_construction = index.arguments[4].safeGet(), + .ef_search = index.arguments[5].safeGet()}; + } + + return std::make_shared(index, metric_kind, scalar_kind, usearch_hnsw_params); } void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* attach */) { - /// Check number and type of index arguments: + const bool has_two_args = (index.arguments.size() == 2); + const bool has_six_args = (index.arguments.size() == 6); - if (index.arguments.size() > 2) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must not have more than one parameters"); + /// Check number and type of arguments + if (!has_two_args && !has_six_args) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or six arguments"); + if (index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (method) must be of type String"); + if (index.arguments[1].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (metric) must be of type String"); + if (has_six_args) + { + if (index.arguments[2].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Third argument of vector similarity index (quantization) must be of type String"); + if (index.arguments[3].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Fourth argument of vector similarity index (M) must be of type UInt64"); + if (index.arguments[4].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Fifth argument of vector similarity index (ef_construction) must be of type UInt64"); + if (index.arguments[5].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Sixth argument of vector similarity index (ef_search) must be of type UInt64"); + } - if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (distance function) must be of type String"); - if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (scalar type) must be of type String"); + /// Check that passed arguments are supported + if (!methods.contains(index.arguments[0].safeGet())) + throw Exception(ErrorCodes::INCORRECT_DATA, "First argument (method) of vector similarity index is not supported. Supported methods are: {}", joinByComma(methods)); + if (!distanceFunctionToMetricKind.contains(index.arguments[1].safeGet())) + throw Exception(ErrorCodes::INCORRECT_DATA, "Second argument (distance function) of vector similarity index is not supported. Supported distance function are: {}", joinByComma(distanceFunctionToMetricKind)); + if (has_six_args) + { + if (!quantizationToScalarKind.contains(index.arguments[2].safeGet())) + throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind)); + if (index.arguments[3].safeGet() < 2) + throw Exception(ErrorCodes::INCORRECT_DATA, "Fourth argument (M) of vector similarity index must be > 1"); + if (index.arguments[4].safeGet() < 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Fifth argument (ef_construction) of vector similarity index must be > 0"); + if (index.arguments[5].safeGet() < 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Sixth argument (ef_search) of vector similarity index must be > 0"); + } /// Check that the index is created on a single column - if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column"); - /// Check that a supported metric was passed as first argument - - if (!index.arguments.empty() && !distanceFunctionToMetricKind.contains(index.arguments[0].safeGet())) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized metric kind (first argument) for vector index. Supported kinds are: {}", keysAsString(distanceFunctionToMetricKind)); - - /// Check that a supported kind was passed as a second argument - - if (index.arguments.size() > 1 && !quantizationToScalarKind.contains(index.arguments[1].safeGet())) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unrecognized scalar kind (second argument) for vector index. Supported kinds are: {}", keysAsString(quantizationToScalarKind)); - - /// Check data type of indexed column: + /// Check data type of the indexed column: DataTypePtr data_type = index.sample_block.getDataTypes()[0]; if (const auto * data_type_array = typeid_cast(data_type.get())) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity index can only be created on columns of type Array(Float32)"); + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); + } } } diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h index 95ea3cd5240..f7098c1626c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h @@ -14,6 +14,13 @@ namespace DB { +struct UsearchHnswParams +{ + size_t m = unum::usearch::default_connectivity(); + size_t ef_construction = unum::usearch::default_expansion_add(); + size_t ef_search = unum::usearch::default_expansion_search(); +}; + using USearchIndex = unum::usearch::index_dense_gt; class USearchIndexWithSerialization : public USearchIndex @@ -24,7 +31,8 @@ public: USearchIndexWithSerialization( size_t dimensions, unum::usearch::metric_kind_t metric_kind, - unum::usearch::scalar_kind_t scalar_kind); + unum::usearch::scalar_kind_t scalar_kind, + UsearchHnswParams usearch_hnsw_params); void serialize(WriteBuffer & ostr) const; void deserialize(ReadBuffer & istr); @@ -54,13 +62,15 @@ struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranu const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_); + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_); MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, unum::usearch::metric_kind_t metric_kind_, unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_, USearchIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleVectorSimilarity() override = default; @@ -74,6 +84,7 @@ struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranu const Block index_sample_block; const unum::usearch::metric_kind_t metric_kind; const unum::usearch::scalar_kind_t scalar_kind; + const UsearchHnswParams usearch_hnsw_params; USearchIndexWithSerializationPtr index; LoggerPtr logger = getLogger("VectorSimilarityIndex"); @@ -93,7 +104,8 @@ struct MergeTreeIndexAggregatorVectorSimilarity final : IMergeTreeIndexAggregato const String & index_name_, const Block & index_sample_block, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_); + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_); ~MergeTreeIndexAggregatorVectorSimilarity() override = default; @@ -105,6 +117,7 @@ struct MergeTreeIndexAggregatorVectorSimilarity final : IMergeTreeIndexAggregato const Block index_sample_block; const unum::usearch::metric_kind_t metric_kind; const unum::usearch::scalar_kind_t scalar_kind; + const UsearchHnswParams usearch_hnsw_params; USearchIndexWithSerializationPtr index; }; @@ -136,7 +149,8 @@ public: MergeTreeIndexVectorSimilarity( const IndexDescription & index_, unum::usearch::metric_kind_t metric_kind_, - unum::usearch::scalar_kind_t scalar_kind_); + unum::usearch::scalar_kind_t scalar_kind_, + UsearchHnswParams usearch_hnsw_params_); ~MergeTreeIndexVectorSimilarity() override = default; @@ -149,6 +163,7 @@ public: private: const unum::usearch::metric_kind_t metric_kind; const unum::usearch::scalar_kind_t scalar_kind; + const UsearchHnswParams usearch_hnsw_params; }; } diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index 2ef75d0a7fe..7c66b4b8e45 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -9,14 +9,14 @@ DROP TABLE IF EXISTS tab; SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected'; -CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id; INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA } INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA } DROP TABLE tab; SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; SYSTEM STOP MERGES tab; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]); INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]); @@ -31,7 +31,7 @@ DROP TABLE tab; SELECT 'Correctness of index with > 1 mark'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192; -- disable adaptive granularity due to bug INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000); WITH [1.0, 0.0] AS reference_vec diff --git a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql index a19a0d17536..acb69cb6ff8 100644 --- a/tests/queries/0_stateless/02354_vector_search_default_granularity.sql +++ b/tests/queries/0_stateless/02354_vector_search_default_granularity.sql @@ -6,13 +6,13 @@ SET allow_experimental_vector_similarity_index = 1; -- After CREATE TABLE DROP TABLE IF EXISTS tab; -CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab (id Int32, vec Array(Float32), INDEX idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; -- After ALTER TABLE DROP TABLE tab; CREATE TABLE tab (id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id; -ALTER TABLE tab ADD INDEX idx(vec) TYPE vector_similarity; +ALTER TABLE tab ADD INDEX idx(vec) TYPE vector_similarity('hnsw', 'L2Distance'); SELECT granularity FROM system.data_skipping_indices WHERE database = currentDatabase() AND table = 'tab' AND name = 'idx'; DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql index 36241dfabf7..f92eaddbbed 100644 --- a/tests/queries/0_stateless/02354_vector_search_detach_attach.sql +++ b/tests/queries/0_stateless/02354_vector_search_detach_attach.sql @@ -5,7 +5,7 @@ SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); DETACH TABLE tab SYNC; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference index bee3236f436..b6d034208d0 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference @@ -1,8 +1,10 @@ -At most two index arguments -1st argument (distance function) must be String -Unsupported distance functions are rejected -2nd argument (scalar kind) must be String -Unsupported scalar kinds are rejected +Two or six index arguments +1st argument (method) must be String and hnsw +2nd argument (distance function) must be String and L2Distance or cosineDistance +3nd argument (quantization), if given, must be String and f32, f16, ... +4nd argument (M), if given, must be UInt64 and > 1 +5nd argument (ef_construction), if given, must be UInt64 and > 0 +6nd argument (ef_search), if given, must be UInt64 and > 0 Must be created on single column Must be created on Array(Float32) columns Rejects INSERTs of Arrays with different sizes diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index 912f7d7fcae..7c2ddfe81fc 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -6,32 +6,46 @@ SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; -SELECT 'At most two index arguments'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('too', 'many', 'args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +SELECT 'Two or six index arguments'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant_have_one_arg')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'three_args')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cant', 'have', 'more', 'than', 'six', 'args', '!')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -SELECT '1st argument (distance function) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +SELECT '1st argument (method) must be String and hnsw'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3, 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('not_hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -SELECT 'Unsupported distance functions are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('invalidDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '2nd argument (distance function) must be String and L2Distance or cosineDistance'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'invalid_distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -SELECT '2nd argument (scalar kind) must be String'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } - -SELECT 'Unsupported scalar kinds are rejected'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('L2Distance', 'invalidKind')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '3nd argument (quantization), if given, must be String and f32, f16, ...'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 1, 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'invalid', 2, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '4nd argument (M), if given, must be UInt64 and > 1'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '5nd argument (ef_construction), if given, must be UInt64 and > 0'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 'invalid', 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 0, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +SELECT '6nd argument (ef_search), if given, must be UInt64 and > 0'; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 'invalid')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 0)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT 'Must be created on single column'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } SELECT 'Must be created on Array(Float32) columns'; SET allow_suspicious_low_cardinality_types = 1; -CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec UInt64, INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } SELECT 'Rejects INSERTs of Arrays with different sizes'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql new file mode 100644 index 00000000000..f1cfc041233 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql @@ -0,0 +1,14 @@ +-- Tags: no-fasttest, no-ordinary-database + +-- Tests that multiple vector similarity indexes can be created on the same column (even if that makes no sense) + +SET allow_experimental_vector_similarity_index = 1; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); + +ALTER TABLE tab ADD INDEX idx(vec) TYPE minmax; +ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); +ALTER TABLE tab ADD INDEX vec_idx2(vec) TYPE vector_similarity('hnsw', 'L2Distance'); -- silly but creating the same index also works for non-vector indexes ... + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 50537ad6244..dbf0fca32ab 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -10,7 +10,7 @@ SELECT '10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); @@ -34,7 +34,7 @@ DROP TABLE tab; SELECT '12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]); SELECT '- ORDER-BY-type'; @@ -56,9 +56,9 @@ DROP TABLE tab; SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen. --- Just for jun, use metric = 'cosineDistance', scalarKind = 'f64' +-- Test with non-default metric, M, ef_construction, ef_search -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('cosineDistance', 'f64') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99, 66) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); SELECT '- ORDER-BY-type'; From fb76cb90b1badef334b96b61d976136fd38d535d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Aug 2024 09:31:36 +0000 Subject: [PATCH 0831/1722] Allow un-quoted skip index parameters Previously, only this syntax to create a skip index worked: INDEX index_name column_name TYPE vector_similarity('hnsw', 'L2Distance') Now, this syntax will work as well: INDEX index_name column_name TYPE vector_similarity(hnsw, L2Distance) --- .../mergetree-family/annindexes.md | 15 +++++++++++- src/Storages/IndicesDescription.cpp | 12 +++++++--- ...search_unquoted_index_parameters.reference | 0 ...ector_search_unquoted_index_parameters.sql | 23 +++++++++++++++++++ 4 files changed, 46 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 354fac6ea74..e73d6f07a32 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -43,7 +43,7 @@ CREATE TABLE table ( id Int64, vectors Array(Float32), - INDEX index_name vec TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N] + INDEX index_name vectors TYPE vector_similarity(method, distance_function[, quantization, connectivity, expansion_add, expansion_search]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; @@ -59,6 +59,19 @@ Parameters: - `ef_construction`: (optional, default: 128) - `ef_search`: (optional, default: 64) +Example: + +```sql +CREATE TABLE table +( + id Int64, + vectors Array(Float32), + INDEX idx vectors TYPE vector_similarity('hnsw', 'L2Distance') -- Alternative syntax: TYPE vector_similarity(hnsw, L2Distance) +) +ENGINE = MergeTree +ORDER BY id; +``` + Vector similarity indexes are based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW algorithm](https://arxiv.org/abs/1603.09320), i.e., a hierarchical graph where each point represents a vector and the edges represent similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from the diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index cef8fd85f97..753fbf1d635 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -130,10 +131,15 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast { for (size_t i = 0; i < index_type->arguments->children.size(); ++i) { - const auto * argument = index_type->arguments->children[i]->as(); - if (!argument) + const auto & child = index_type->arguments->children[i]; + if (const auto * ast_literal = child->as(); ast_literal != nullptr) + /// E.g. INDEX index_name column_name TYPE vector_similarity('hnsw', 'f32') + result.arguments.emplace_back(ast_literal->value); + else if (const auto * ast_identifier = child->as(); ast_identifier != nullptr) + /// E.g. INDEX index_name column_name TYPE vector_similarity(hnsw, f32) + result.arguments.emplace_back(ast_identifier->name()); + else throw Exception(ErrorCodes::INCORRECT_QUERY, "Only literals can be skip index arguments"); - result.arguments.emplace_back(argument->value); } } diff --git a/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.reference b/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql b/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql new file mode 100644 index 00000000000..da6494bf831 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_unquoted_index_parameters.sql @@ -0,0 +1,23 @@ +-- Tags: no-fasttest, no-ordinary-database + +SET allow_experimental_vector_similarity_index = 1; + +-- Tests that quoted and unquoted parameters can be passed to vector search indexes. + +DROP TABLE IF EXISTS tab1; +DROP TABLE IF EXISTS tab2; + +CREATE TABLE tab1 (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); +CREATE TABLE tab2 (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity(hnsw, L2Distance)); + +DROP TABLE tab1; +DROP TABLE tab2; + +CREATE TABLE tab1 (id Int32, vec Array(Float32), PRIMARY KEY id); +CREATE TABLE tab2 (id Int32, vec Array(Float32), PRIMARY KEY id); + +ALTER TABLE tab1 ADD INDEX idx1(vec) TYPE vector_similarity('hnsw', 'L2Distance'); +ALTER TABLE tab2 ADD INDEX idx2(vec) TYPE vector_similarity(hnsw, L2Distance); + +DROP TABLE tab1; +DROP TABLE tab2; From ea1cd665750f82bbeaf66f67b1d85e014afdec18 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 12 Aug 2024 17:32:43 +0200 Subject: [PATCH 0832/1722] fix tidy --- src/Storages/VirtualColumnUtils.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1abac56d266..3143c7f78f6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -38,7 +38,6 @@ #include #include #include -#include #include #include #include @@ -46,15 +45,7 @@ #include #include #include -#include "Functions/FunctionsLogical.h" -#include "Functions/IFunction.h" -#include "Functions/IFunctionAdaptors.h" -#include "Functions/indexHint.h" #include -#include -#include -#include -#include namespace DB From 44d4784da559cc3b73e71acdee95c39cc3403c42 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 15:52:39 +0000 Subject: [PATCH 0833/1722] Use std::string_view for lookups in hash table --- src/Columns/ColumnObject.cpp | 43 +++++++++---------- src/Columns/ColumnObject.h | 29 +++++++------ src/Common/StringHashForHeterogeneousLookup.h | 25 +++++++++++ src/Functions/JSONPaths.cpp | 28 ++++++------ 4 files changed, 76 insertions(+), 49 deletions(-) create mode 100644 src/Common/StringHashForHeterogeneousLookup.h diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 1f16c12f6ba..4ab4a4e441a 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -243,7 +244,7 @@ void ColumnObject::insertData(const char *, size_t) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertData is not supported for {}", getName()); } -ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(const String & path) +ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(const std::string_view path) { if (dynamic_paths.size() == max_dynamic_paths) return nullptr; @@ -435,7 +436,7 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) /// Second, insert dynamic paths and extend them if needed. /// We can reach the limit of dynamic paths, and in this case /// the rest of dynamic paths will be inserted into shared data. - std::vector src_dynamic_paths_for_shared_data; + std::vector src_dynamic_paths_for_shared_data; for (const auto & [path, column] : src_object_column.dynamic_paths) { /// Check if we already have such dynamic path. @@ -469,7 +470,7 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l /// Second, insert dynamic paths and extend them if needed. /// We can reach the limit of dynamic paths, and in this case /// the rest of dynamic paths will be inserted into shared data. - std::vector src_dynamic_paths_for_shared_data; + std::vector src_dynamic_paths_for_shared_data; for (const auto & [path, column] : src_object_column.dynamic_paths) { /// Check if we already have such dynamic path. @@ -487,7 +488,7 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, std::move(src_dynamic_paths_for_shared_data), start, length); } -void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length) +void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length) { /// Paths in shared data are sorted, so paths from src_dynamic_paths_for_shared_data should be inserted properly /// to keep paths sorted. Let's sort them in advance. @@ -512,8 +513,8 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co for (size_t i = start; i != start + length; ++i) { /// Paths in src_dynamic_paths_for_shared_data are already sorted. - for (const auto & path : src_dynamic_paths_for_shared_data) - serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, path, *src_object_column.dynamic_paths.at(path), i); + for (const auto path : src_dynamic_paths_for_shared_data) + serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, path, *src_object_column.dynamic_paths.find(path)->second, i); shared_data_offsets.push_back(shared_data_paths->size()); } } @@ -541,9 +542,9 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co size_t end = src_shared_data_offsets[row]; for (size_t i = offset; i != end; ++i) { - auto path = src_shared_data_paths->getDataAt(i); + auto path = src_shared_data_paths->getDataAt(i).toView(); /// Check if we have this path in dynamic paths. - if (auto it = dynamic_paths_ptrs.find(path.toString()); it != dynamic_paths_ptrs.end()) + if (auto it = dynamic_paths_ptrs.find(path); it != dynamic_paths_ptrs.end()) { /// Deserialize binary value into dynamic column from shared data. deserializeValueFromSharedData(src_shared_data_values, i, *it->second); @@ -555,8 +556,8 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co while (src_dynamic_paths_for_shared_data_index < src_dynamic_paths_for_shared_data.size() && src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index] < path) { - const auto & dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; - serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.at(dynamic_path), row); + const auto dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; + serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.find(dynamic_path)->second, row); ++src_dynamic_paths_for_shared_data_index; } @@ -569,8 +570,8 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co /// Insert remaining dynamic paths from src_dynamic_paths_for_shared_data. for (; src_dynamic_paths_for_shared_data_index != src_dynamic_paths_for_shared_data.size(); ++src_dynamic_paths_for_shared_data_index) { - const auto & dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; - serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.at(dynamic_path), row); + const auto dynamic_path = src_dynamic_paths_for_shared_data[src_dynamic_paths_for_shared_data_index]; + serializePathAndValueIntoSharedData(shared_data_paths, shared_data_values, dynamic_path, *src_object_column.dynamic_paths.find(dynamic_path)->second, row); } shared_data_offsets.push_back(shared_data_paths->size()); @@ -584,7 +585,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co } } -void ColumnObject::serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const String & path, const IColumn & column, size_t n) +void ColumnObject::serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const std::string_view path, const IColumn & column, size_t n) { /// Don't store Null values in shared data. We consider Null value equivalent to the absence /// of this path in the row because we cannot distinguish these 2 cases for dynamic paths. @@ -700,11 +701,10 @@ const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) auto path_size = unalignedLoad(pos); pos += sizeof(size_t); std::string_view path(pos, path_size); - String path_str(path); pos += path_size; /// Check if it's a typed path. In this case we should use /// deserializeAndInsertFromArena of corresponding column. - if (auto typed_it = typed_paths.find(path_str); typed_it != typed_paths.end()) + if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) { pos = typed_it->second->deserializeAndInsertFromArena(pos); } @@ -712,19 +712,18 @@ const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) /// to dynamic paths or shared data. else { - auto value_size = unalignedLoad(pos); pos += sizeof(size_t); std::string_view value(pos, value_size); pos += value_size; /// Check if we have this path in dynamic paths. - if (auto dynamic_it = dynamic_paths.find(path_str); dynamic_it != dynamic_paths.end()) + if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) { ReadBufferFromMemory buf(value.data(), value.size()); getDynamicSerialization()->deserializeBinary(*dynamic_it->second, buf, getFormatSettings()); } /// Try to add a new dynamic path. - else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path_str)) + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) { ReadBufferFromMemory buf(value.data(), value.size()); getDynamicSerialization()->deserializeBinary(*dynamic_path_column, buf, getFormatSettings()); @@ -773,7 +772,7 @@ const char * ColumnObject::skipSerializedInArena(const char * pos) const { auto path_size = unalignedLoad(pos); pos += sizeof(size_t); - String path(pos, path_size); + std::string_view path(pos, path_size); pos += path_size; if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) { @@ -1167,7 +1166,7 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou if (path_to_total_number_of_non_null_values.size() > max_dynamic_paths) { /// Sort paths by total_number_of_non_null_values. - std::vector> paths_with_sizes; + std::vector> paths_with_sizes; paths_with_sizes.reserve(path_to_total_number_of_non_null_values.size()); for (const auto & [path, size] : path_to_total_number_of_non_null_values) paths_with_sizes.emplace_back(size, path); @@ -1176,8 +1175,8 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou /// Fill dynamic_paths with first max_dynamic_paths paths in sorted list. for (size_t i = 0; i != max_dynamic_paths; ++i) { - dynamic_paths[paths_with_sizes[i].second] = ColumnDynamic::create(max_dynamic_types); - dynamic_paths_ptrs[paths_with_sizes[i].second] = assert_cast(dynamic_paths[paths_with_sizes[i].second].get()); + dynamic_paths.emplace(paths_with_sizes[i].second, ColumnDynamic::create(max_dynamic_types)); + dynamic_paths_ptrs.emplace(paths_with_sizes[i].second, assert_cast(dynamic_paths.find(paths_with_sizes[i].second)->second.get())); } } /// Use all dynamic paths from all source columns. diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index fbb68897e08..ecb6c4e0e15 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -9,7 +9,7 @@ #include #include #include - +#include #include namespace DB @@ -44,6 +44,9 @@ private: size_t max_dynamic_types_, const Statistics & statistics_ = {}); + /// Use StringHashForHeterogeneousLookup hash for hash maps to be able to use std::string_view in find() method. + using PathToColumnMap = std::unordered_map; + using PathToDynamicColumnPtrMap = std::unordered_map; public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use mutate in order to make mutable column and mutate shared nested columns. @@ -158,14 +161,14 @@ public: bool hasDynamicStructure() const override { return true; } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; - const std::unordered_map & getTypedPaths() const { return typed_paths; } - std::unordered_map & getTypedPaths() { return typed_paths; } + const PathToColumnMap & getTypedPaths() const { return typed_paths; } + PathToColumnMap & getTypedPaths() { return typed_paths; } - const std::unordered_map & getDynamicPaths() const { return dynamic_paths; } - std::unordered_map & getDynamicPaths() { return dynamic_paths; } + const PathToColumnMap & getDynamicPaths() const { return dynamic_paths; } + PathToColumnMap & getDynamicPaths() { return dynamic_paths; } - const std::unordered_map & getDynamicPathsPtrs() const { return dynamic_paths_ptrs; } - std::unordered_map & getDynamicPathsPtrs() { return dynamic_paths_ptrs; } + const PathToDynamicColumnPtrMap & getDynamicPathsPtrs() const { return dynamic_paths_ptrs; } + PathToDynamicColumnPtrMap & getDynamicPathsPtrs() { return dynamic_paths_ptrs; } const Statistics & getStatistics() const { return statistics; } @@ -198,12 +201,12 @@ public: /// Try to add new dynamic path. Returns pointer to the new dynamic /// path column or nullptr if limit on dynamic paths is reached. - ColumnDynamic * tryToAddNewDynamicPath(const String & path); + ColumnDynamic * tryToAddNewDynamicPath(const std::string_view path); void setDynamicPaths(const std::vector & paths); void setStatistics(const Statistics & statistics_) { statistics = statistics_; } - void serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const String & path, const IColumn & column, size_t n); + void serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const std::string_view path, const IColumn & column, size_t n); void deserializeValueFromSharedData(const ColumnString * shared_data_values, size_t n, IColumn & column) const; /// Paths in shared data are sorted in each row. Use this method to find the lower bound for specific path in the row. @@ -212,19 +215,19 @@ public: static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); private: - void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length); + void insertFromSharedDataAndFillRemainingDynamicPaths(const ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length); void serializePathAndValueIntoArena(Arena & arena, const char *& begin, StringRef path, StringRef value, StringRef & res) const; /// Map path -> column for paths with explicitly specified types. /// This set of paths is constant and cannot be changed. - std::unordered_map typed_paths; + PathToColumnMap typed_paths; /// Map path -> column for dynamically added paths. All columns /// here are Dynamic columns. This set of paths can be extended /// during inerts into the column. - std::unordered_map dynamic_paths; + PathToColumnMap dynamic_paths; /// Store and use pointers to ColumnDynamic to avoid virtual calls. /// With hundreds of dynamic paths these virtual calls are noticeable. - std::unordered_map dynamic_paths_ptrs; + PathToDynamicColumnPtrMap dynamic_paths_ptrs; /// Shared storage for all other paths and values. It's filled /// when the number of dynamic paths reaches the limit. /// It has type Array(Tuple(String, String)) and stores diff --git a/src/Common/StringHashForHeterogeneousLookup.h b/src/Common/StringHashForHeterogeneousLookup.h new file mode 100644 index 00000000000..0983fd460d6 --- /dev/null +++ b/src/Common/StringHashForHeterogeneousLookup.h @@ -0,0 +1,25 @@ +#pragma once +#include + +namespace DB +{ + +/// See https://www.open-std.org/jtc1/sc22/wg21/docs/papers/2018/p0919r3.html +struct StringHashForHeterogeneousLookup +{ + using hash_type = std::hash; + using transparent_key_equal = std::equal_to<>; + using is_transparent = void; // required to make find() work with different type than key_type + + auto operator()(const std::string_view view) const + { + return hash_type()(view); + } + + auto operator()(const std::string & str) const + { + return hash_type()(str); + } +}; + +} diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index 4a84cec711b..35613e40aac 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -139,7 +139,7 @@ private: { /// Collect all dynamic paths. const auto & dynamic_path_columns = column_object.getDynamicPaths(); - std::vector dynamic_paths; + std::vector dynamic_paths; dynamic_paths.reserve(dynamic_path_columns.size()); for (const auto & [path, _] : dynamic_path_columns) dynamic_paths.push_back(path); @@ -149,11 +149,11 @@ private: size_t size = column_object.size(); for (size_t i = 0; i != size; ++i) { - for (const auto & path : dynamic_paths) + for (const auto path : dynamic_paths) { /// Don't include path if it contains NULL, because we consider /// it to be equivalent to the absence of this path in this row. - if (!dynamic_path_columns.at(path)->isNullAt(i)) + if (!dynamic_path_columns.find(path)->second->isNullAt(i)) data.insertData(path.data(), path.size()); } offsets.push_back(data.size()); @@ -162,7 +162,7 @@ private: } /// Collect all paths: typed, dynamic and paths from shared data. - std::vector sorted_dynamic_and_typed_paths; + std::vector sorted_dynamic_and_typed_paths; const auto & typed_path_columns = column_object.getTypedPaths(); const auto & dynamic_path_columns = column_object.getDynamicPaths(); sorted_dynamic_and_typed_paths.reserve(typed_path_columns.size() + dynamic_path_columns.size()); @@ -184,22 +184,22 @@ private: size_t sorted_paths_index = 0; for (size_t j = start; j != end; ++j) { - auto shared_data_path = shared_data_paths->getDataAt(j); + auto shared_data_path = shared_data_paths->getDataAt(j).toView(); while (sorted_paths_index != sorted_dynamic_and_typed_paths.size() && sorted_dynamic_and_typed_paths[sorted_paths_index] < shared_data_path) { - const auto & path = sorted_dynamic_and_typed_paths[sorted_paths_index]; + const auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; /// If it's dynamic path include it only if it's not NULL. if (auto it = dynamic_path_columns.find(path); it == dynamic_path_columns.end() || !it->second->isNullAt(i)) data.insertData(path.data(), path.size()); ++sorted_paths_index; } - data.insertData(shared_data_path.data, shared_data_path.size); + data.insertData(shared_data_path.data(), shared_data_path.size()); } for (; sorted_paths_index != sorted_dynamic_and_typed_paths.size(); ++sorted_paths_index) { - const auto & path = sorted_dynamic_and_typed_paths[sorted_paths_index]; + const auto path = sorted_dynamic_and_typed_paths[sorted_paths_index]; if (auto it = dynamic_path_columns.find(path); it == dynamic_path_columns.end() || !it->second->isNullAt(i)) data.insertData(path.data(), path.size()); } @@ -220,7 +220,7 @@ private: if constexpr (Impl::paths_mode == PathsMode::DYNAMIC_PATHS) { const auto & dynamic_path_columns = column_object.getDynamicPaths(); - std::vector sorted_dynamic_paths; + std::vector sorted_dynamic_paths; sorted_dynamic_paths.reserve(dynamic_path_columns.size()); for (const auto & [path, _] : dynamic_path_columns) sorted_dynamic_paths.push_back(path); @@ -230,9 +230,9 @@ private: /// Iterate over all rows and extract types from dynamic columns. for (size_t i = 0; i != column_object.size(); ++i) { - for (auto & path : sorted_dynamic_paths) + for (const auto path : sorted_dynamic_paths) { - auto column = dynamic_path_columns.at(path); + const auto & column = dynamic_path_columns.find(path)->second; if (!column->isNullAt(i)) { auto type = getDynamicValueType(column, i); @@ -272,7 +272,7 @@ private: } /// Iterate over all rows and extract types from dynamic columns from dynamic paths and from values in shared data. - std::vector> sorted_typed_and_dynamic_paths_with_types; + std::vector> sorted_typed_and_dynamic_paths_with_types; const auto & typed_path_types = type_object.getTypedPaths(); const auto & dynamic_path_columns = column_object.getDynamicPaths(); sorted_typed_and_dynamic_paths_with_types.reserve(typed_path_types.size() + dynamic_path_columns.size()); @@ -294,7 +294,7 @@ private: size_t sorted_paths_index = 0; for (size_t j = start; j != end; ++j) { - auto shared_data_path = shared_data_paths->getDataAt(j); + auto shared_data_path = shared_data_paths->getDataAt(j).toView(); auto type_name = getDynamicValueTypeFromSharedData(shared_data_values->getDataAt(j)); /// Skip NULL values. if (!type_name) @@ -319,7 +319,7 @@ private: ++sorted_paths_index; } - paths_column->insertData(shared_data_path.data, shared_data_path.size); + paths_column->insertData(shared_data_path.data(), shared_data_path.size()); types_column->insertData(type_name->data(), type_name->size()); } From 44e267ec76bcdb625a8bb6e5ffccc7bdc74d5cef Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 15:58:40 +0000 Subject: [PATCH 0834/1722] Fix conflicts and style check --- src/Columns/ColumnObjectDeprecated.cpp | 4 ++-- src/DataTypes/DataTypeObject.cpp | 2 -- src/DataTypes/DataTypeObjectDeprecated.cpp | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnObjectDeprecated.cpp b/src/Columns/ColumnObjectDeprecated.cpp index d3f23dc6b57..d03b1d0df82 100644 --- a/src/Columns/ColumnObjectDeprecated.cpp +++ b/src/Columns/ColumnObjectDeprecated.cpp @@ -698,7 +698,7 @@ void ColumnObjectDeprecated::forEachSubcolumnRecursively(RecursiveMutableColumnC void ColumnObjectDeprecated::insert(const Field & field) { - const auto & object = field.get(); + const auto & object = field.safeGet(); HashSet inserted_paths; size_t old_size = size(); @@ -754,7 +754,7 @@ void ColumnObjectDeprecated::get(size_t n, Field & res) const { assert(n < size()); res = Object(); - auto & object = res.get(); + auto & object = res.safeGet(); for (const auto & entry : subcolumns) { diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 4f88bed8de8..b5d1f429001 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -26,8 +26,6 @@ #endif #include - - namespace DB { diff --git a/src/DataTypes/DataTypeObjectDeprecated.cpp b/src/DataTypes/DataTypeObjectDeprecated.cpp index d9ec70ca7cc..07f9c116e58 100644 --- a/src/DataTypes/DataTypeObjectDeprecated.cpp +++ b/src/DataTypes/DataTypeObjectDeprecated.cpp @@ -72,7 +72,7 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Object data type family must have a const string as its schema name parameter"); - return std::make_shared(literal->value.get(), is_nullable); + return std::make_shared(literal->value.safeGet(), is_nullable); } void registerDataTypeObjectDeprecated(DataTypeFactory & factory) From 3a6e05eb43cbb9937cded286ac7259b2f7168057 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 12 Aug 2024 18:03:42 +0200 Subject: [PATCH 0835/1722] try to fix includes --- src/Storages/VirtualColumnUtils.cpp | 55 ++++++++++++++++------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 3143c7f78f6..d932f5cc469 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,43 +1,40 @@ #include #include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include + #include -#include -#include -#include #include +#include +#include +#include #include -#include -#include + #include +#include #include +#include #include #include -#include -#include + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include #include #include -#include #include +#include #include + +#include #include #include #include @@ -45,7 +42,15 @@ #include #include #include +#include "Functions/FunctionsLogical.h" +#include "Functions/IFunction.h" +#include "Functions/IFunctionAdaptors.h" +#include "Functions/indexHint.h" #include +#include +#include +#include +#include namespace DB From 858b7e55d0df3db1412d538f701c30584b5783bf Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 12 Aug 2024 16:16:50 +0000 Subject: [PATCH 0836/1722] Improve condition in case the default column consumes slightly more memory It never happened in the few hundreds of tests I ran successfully, but we'd rather be safe than sorry. --- .../01903_correct_block_size_prediction_with_default.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index 075d9a1dacf..1482730af2c 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -28,7 +28,8 @@ function test() SYSTEM FLUSH LOGS; WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) - SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage FROM memory_1, memory_2;" + SELECT memory_1.memory_usage <= 1.2 * memory_2.memory_usage OR + memory_2.memory_usage <= 1.2 * memory_1.memory_usage FROM memory_1, memory_2;" } test "" From 25ce6df8957c36358498aed83ce2e17d034261f3 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 16:22:30 +0000 Subject: [PATCH 0837/1722] Return local table --- .../TableFunctionObjectStorage.cpp | 2 +- .../TableFunctionObjectStorage.h | 22 +++++++++++++++---- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..6235108e2ba 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -222,5 +222,5 @@ template class TableFunctionObjectStorage template class TableFunctionObjectStorage; template class TableFunctionObjectStorage; #endif - +template class TableFunctionObjectStorage; } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 86b8f0d5e14..3468e5c5007 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include "config.h" -#include -#include #include -#include +#include #include +#include +#include +#include "config.h" namespace DB { @@ -14,6 +14,7 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; +class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -90,6 +91,17 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; +struct LocalDefinition +{ + static constexpr auto name = "local"; + static constexpr auto storage_type_name = "Local"; + static constexpr auto signature = " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif + +using TableFunctionLocal = TableFunctionObjectStorage; } From f7af4c5643af2ee87b81a7972c0bb91cf723c8a2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 17:27:43 +0100 Subject: [PATCH 0838/1722] don't report system-wide metrics when cgroup metrics present --- src/Common/AsynchronousMetrics.cpp | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 02c130d3caa..9b6a7428411 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -75,12 +75,8 @@ AsynchronousMetrics::AsynchronousMetrics( , protocol_server_metrics_func(protocol_server_metrics_func_) { #if defined(OS_LINUX) - openFileIfExists("/proc/meminfo", meminfo); - openFileIfExists("/proc/loadavg", loadavg); - openFileIfExists("/proc/stat", proc_stat); openFileIfExists("/proc/cpuinfo", cpuinfo); openFileIfExists("/proc/sys/fs/file-nr", file_nr); - openFileIfExists("/proc/uptime", uptime); openFileIfExists("/proc/net/dev", net_dev); /// CGroups v2 @@ -103,6 +99,19 @@ AsynchronousMetrics::AsynchronousMetrics( if (!cgroupcpu_stat) openFileIfExists("/sys/fs/cgroup/cpuacct/cpuacct.stat", cgroupcpuacct_stat); + if (!cgroupcpu_stat && !cgroupcpuacct_stat) + { + /// The following metrics are not cgroup-aware and we've found cgroup-specific metric files for the similar metrics, + /// so we're better not reporting them at all to avoid confusion + openFileIfExists("/proc/loadavg", loadavg); + openFileIfExists("/proc/stat", proc_stat); + openFileIfExists("/proc/uptime", uptime); + } + + /// The same story for memory metrics + if (!cgroupmem_limit_in_bytes) + openFileIfExists("/proc/meminfo", meminfo); + openFileIfExists("/proc/sys/vm/max_map_count", vm_max_map_count); openFileIfExists("/proc/self/maps", vm_maps); @@ -1193,8 +1202,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) tryLogCurrentException(__PRETTY_FUNCTION__); } } - - if (meminfo) + else if (meminfo) { try { From 0c209242b1331cd714be186239e90a68acc44eae Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 16:28:18 +0000 Subject: [PATCH 0839/1722] Fix build --- src/DataTypes/Serializations/SerializationObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 9091702326a..68d25eaeaff 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -466,7 +466,7 @@ void SerializationObject::deserializeBinaryBulkWithMultipleStreams( void SerializationObject::serializeBinary(const Field & field, WriteBuffer & ostr, const DB::FormatSettings & settings) const { - const auto & object = field.get(); + const auto & object = field.safeGet(); /// Serialize number of paths and then pairs (path, value). writeVarUInt(object.size(), ostr); for (const auto & [path, value] : object) From f0f10bc0099e659bfc0bf31079e89832f9db4b17 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 17:30:12 +0100 Subject: [PATCH 0840/1722] remove cgroupsV2MemoryControllerEnabled() --- base/base/cgroupsv2.cpp | 24 ------------------------ base/base/cgroupsv2.h | 4 ---- base/base/getMemoryAmount.cpp | 3 --- 3 files changed, 31 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index 4372696c2b7..d8f95b23ae7 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -27,27 +27,6 @@ bool cgroupsV2Enabled() #endif } -bool cgroupsV2MemoryControllerEnabled() -{ -#if defined(OS_LINUX) - chassert(cgroupsV2Enabled()); - /// According to https://docs.kernel.org/admin-guide/cgroup-v2.html, file "cgroup.controllers" defines which controllers are available - /// for the current + child cgroups. The set of available controllers can be restricted from level to level using file - /// "cgroups.subtree_control". It is therefore sufficient to check the bottom-most nested "cgroup.controllers" file. - fs::path cgroup_dir = cgroupV2PathOfProcess(); - if (cgroup_dir.empty()) - return false; - std::ifstream controllers_file(cgroup_dir / "cgroup.controllers"); - if (!controllers_file.is_open()) - return false; - std::string controllers; - std::getline(controllers_file, controllers); - return controllers.find("memory") != std::string::npos; -#else - return false; -#endif -} - fs::path cgroupV2PathOfProcess() { #if defined(OS_LINUX) @@ -77,9 +56,6 @@ std::optional getCgroupsV2PathContainingFile(std::string_view file_ if (!cgroupsV2Enabled()) return {}; - if (!cgroupsV2MemoryControllerEnabled()) - return {}; - fs::path current_cgroup = cgroupV2PathOfProcess(); if (current_cgroup.empty()) return {}; diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h index 9d8e178a866..925a399471e 100644 --- a/base/base/cgroupsv2.h +++ b/base/base/cgroupsv2.h @@ -12,10 +12,6 @@ static inline const std::filesystem::path default_cgroups_mount = "/sys/fs/cgrou /// Is cgroups v2 enabled on the system? bool cgroupsV2Enabled(); -/// Is the memory controller of cgroups v2 enabled on the system? -/// Assumes that cgroupsV2Enabled() is enabled. -bool cgroupsV2MemoryControllerEnabled(); - /// Detects which cgroup v2 the process belongs to and returns the filesystem path to the cgroup. /// Returns an empty path the cgroup cannot be determined. /// Assumes that cgroupsV2Enabled() is enabled. diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 03aab1eac72..bbfbecdbffd 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -19,9 +19,6 @@ std::optional getCgroupsV2MemoryLimit() if (!cgroupsV2Enabled()) return {}; - if (!cgroupsV2MemoryControllerEnabled()) - return {}; - std::filesystem::path current_cgroup = cgroupV2PathOfProcess(); if (current_cgroup.empty()) return {}; From 05b595094868dd29e59ea9c766d0829f57ce94f9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 12 Aug 2024 17:31:56 +0100 Subject: [PATCH 0841/1722] small fix --- base/base/cgroupsv2.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index d8f95b23ae7..b4ca8271d64 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -60,8 +60,8 @@ std::optional getCgroupsV2PathContainingFile(std::string_view file_ if (current_cgroup.empty()) return {}; - /// Return the bottom-most nested current memory file. If there is no such file at the current - /// level, try again at the parent level as memory settings are inherited. + /// Return the bottom-most nested file. If there is no such file at the current + /// level, try again at the parent level as settings are inherited. while (current_cgroup != default_cgroups_mount.parent_path()) { const auto path = current_cgroup / file_name; From f8011d53d6ec21a3572c67179d7752f65f032260 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 17:14:43 +0000 Subject: [PATCH 0842/1722] Fix data types parsing --- src/Parsers/ParserCreateQuery.cpp | 34 ------------------- src/Parsers/ParserCreateQuery.h | 9 ----- src/Parsers/ParserDataType.cpp | 10 ++++-- .../queries/0_stateless/03205_json_syntax.sql | 2 +- .../03210_json_type_alter_add_column.sql.j2 | 1 + 5 files changed, 10 insertions(+), 46 deletions(-) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 62c5ff42ddf..85b2c52643b 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -53,40 +53,6 @@ ASTPtr parseComment(IParser::Pos & pos, Expected & expected) } - -bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserToken open(TokenType::OpeningRoundBracket); - ParserToken close(TokenType::ClosingRoundBracket); - ParserIdentifier name_p; - ParserNameTypePairList columns_p; - - ASTPtr name; - ASTPtr columns; - - /// For now `name == 'Nested'` or `name == 'Tuple'`, probably alternative nested data structures will appear - if (!name_p.parse(pos, name, expected)) - return false; - - if (!open.ignore(pos, expected)) - return false; - - if (!columns_p.parse(pos, columns, expected)) - return false; - - if (!close.ignore(pos, expected)) - return false; - - auto func = std::make_shared(); - tryGetIdentifierNameInto(name, func->name); - - func->arguments = columns; - func->children.push_back(columns); - node = func; - - return true; -} - bool ParserSQLSecurity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken s_eq(TokenType::Equals); diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 53a62deb22b..82da2e7ea0b 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -18,15 +18,6 @@ namespace DB { -/** A nested table. For example, Nested(UInt32 CounterID, FixedString(2) UserAgentMajor) - */ -class ParserNestedTable : public IParserBase -{ -protected: - const char * getName() const override { return "nested table"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - /** Parses sql security option. DEFINER = user_name SQL SECURITY DEFINER */ class ParserSQLSecurity : public IParserBase diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index fab9ed728a7..e941c05eba1 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -273,8 +273,14 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (type_name == "Nested") { - ParserNestedTable nested_parser; - nested_parser.parse(pos, arg, expected); + ParserNameTypePair name_and_type_parser; + name_and_type_parser.parse(pos, arg, expected); + } + else if (type_name == "Tuple") + { + ParserNameTypePair name_and_type_parser; + ParserDataType only_type_parser; + name_and_type_parser.parse(pos, arg, expected) || only_type_parser.parse(pos, arg, expected); } else if (type_name == "AggregateFunction" || type_name == "SimpleAggregateFunction") { diff --git a/tests/queries/0_stateless/03205_json_syntax.sql b/tests/queries/0_stateless/03205_json_syntax.sql index 11fa513f265..e3c88c81d0d 100644 --- a/tests/queries/0_stateless/03205_json_syntax.sql +++ b/tests/queries/0_stateless/03205_json_syntax.sql @@ -36,5 +36,5 @@ create table test (json JSON(SKIP `some path`.`path some`)) engine=Memory; drop table test; create table test (json JSON(SKIP REGEXP '.*a.*')) engine=Memory; drop table test; -create table test (json JSON(max_dynamic_paths=10, max_dynamic_types=10, a.b.c UInt32, b.c.d String, SKIP g.d.a, SKIP o.g.a, SKIP REGEXP '.*a.*', SKIP REGEXP 'abc')) engine=Memory; +create table test (json JSON(max_dynamic_paths=10, max_dynamic_types=10, a.b.c UInt32, b.c.d String, SKIP g.d.a, SKIP o.g.a, SKIP REGEXP '.*u.*', SKIP REGEXP 'abc')) engine=Memory; drop table test; diff --git a/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 b/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 index 8ab9f5181e3..add57928804 100644 --- a/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 +++ b/tests/queries/0_stateless/03210_json_type_alter_add_column.sql.j2 @@ -2,6 +2,7 @@ set allow_experimental_dynamic_type = 1; set allow_experimental_variant_type = 1; +set allow_experimental_json_type = 1; set use_variant_as_common_type = 1; drop table if exists test; From c22265b889684b7fa34ba6816ce3910143ef7226 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 17:11:11 +0000 Subject: [PATCH 0843/1722] Some fixups --- docs/en/operations/query-cache.md | 16 +++++----- docs/en/operations/settings/settings.md | 8 ++--- .../operations/system-tables/query_cache.md | 4 +-- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/Cache/QueryCache.cpp | 25 ++++++++-------- src/Interpreters/Cache/QueryCache.h | 11 ++++--- src/Interpreters/executeQuery.cpp | 5 ++-- .../System/StorageSystemQueryCache.cpp | 16 +++++----- .../02494_query_cache_tag.reference | 12 ++++---- .../0_stateless/02494_query_cache_tag.sql | 30 ++++++++++--------- 11 files changed, 66 insertions(+), 65 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index a6c4d74f4ac..384938e28f6 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -143,16 +143,18 @@ value can be specified at session, profile or query level using setting [query_c Entries in the query cache are compressed by default. This reduces the overall memory consumption at the cost of slower writes into / reads from the query cache. To disable compression, use setting [query_cache_compress_entries](settings/settings.md#query-cache-compress-entries). -Entries in the query cache can separate by tag, using setting [query_cache_tag](settings/settings.md#query-cache-tag). Queries with different tags are considered different entries. For example, the result of query +Sometimes it is useful to keep multiple results for the same query cached. This can be achieved using setting +[query_cache_tag](settings/settings.md#query-cache-tag) that acts as as a label (or namespace) for a query cache entries. The query cache +considers results of the same query with different tags different. -``` sql -SELECT 1 SETTINGS use_query_cache = true; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; +Example for creating three different query cache entries for the same query: + +```sql +SELECT 1 SETTINGS use_query_cache = true; -- query_cache_tag is implicitly '' (empty string) +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 1'; +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 2'; ``` -have different entries in the query cache, find the specified tag in system table [system.query_cache](system-tables/query_cache.md) - ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation, etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting [query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7b855665efb..e4a126249ca 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1802,14 +1802,14 @@ Default value: `0`. ## query_cache_tag {#query-cache-tag} -An arbitrary string to separate entries in the [query cache](../query-cache.md). -Queries with different values of this setting are considered different. +A string which acts as a label for [query cache](../query-cache.md) entries. +The same queries with different tags are considered different by the query cache. Possible values: -- string: name of query cache tag +- Any string -Default value: `''`. +Default value: `''` ## query_cache_max_size_in_bytes {#query-cache-max-size-in-bytes} diff --git a/docs/en/operations/system-tables/query_cache.md b/docs/en/operations/system-tables/query_cache.md index 393b37d3616..9c48574a329 100644 --- a/docs/en/operations/system-tables/query_cache.md +++ b/docs/en/operations/system-tables/query_cache.md @@ -9,12 +9,12 @@ Columns: - `query` ([String](../../sql-reference/data-types/string.md)) — Query string. - `result_size` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Size of the query cache entry. +- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Tag of the query cache entry. - `stale` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is stale. - `shared` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is shared between multiple users. - `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed. - `expires_at` ([DateTime](../../sql-reference/data-types/datetime.md)) — When the query cache entry becomes stale. - `key_hash` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — A hash of the query string, used as a key to find query cache entries. -- `tag` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — An arbitrary string to separate entries in the query cache. **Example** @@ -27,12 +27,12 @@ Row 1: ────── query: SELECT 1 SETTINGS use_query_cache = 1 result_size: 128 +tag: stale: 0 shared: 0 compressed: 1 expires_at: 2023-10-13 13:35:45 key_hash: 12188185624808016954 -tag: 1 row in set. Elapsed: 0.004 sec. ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 75579f20187..f9ffab0ea57 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -676,7 +676,7 @@ class IColumn; M(Bool, query_cache_squash_partial_results, true, "Squash partial result blocks to blocks of size 'max_block_size'. Reduces performance of inserts into the query cache but improves the compressability of cache entries.", 0) \ M(Seconds, query_cache_ttl, 60, "After this time in seconds entries in the query cache become stale", 0) \ M(Bool, query_cache_share_between_users, false, "Allow other users to read entry in the query cache", 0) \ - M(String, query_cache_tag, "", "An arbitrary string to separate entries in the query cache. Queries with different values of this setting are considered different.", 0) \ + M(String, query_cache_tag, "", "A string which acts as a label for query cache entries. The same queries with different tags are considered different by the query cache.", 0) \ M(Bool, enable_sharing_sets_for_mutations, true, "Allow sharing set objects build for IN subqueries between different tasks of the same mutation. This reduces memory usage and CPU consumption", 0) \ \ M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8fd16504e95..0528287e83e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -82,9 +82,9 @@ static std::initializer_list user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed_, - const String & tag_) - : ast_hash(calculateAstHash(ast_, current_database, settings, tag_)) + bool is_compressed_) + : ast_hash(calculateAstHash(ast_, current_database, settings)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -247,12 +242,18 @@ QueryCache::Key::Key( , expires_at(expires_at_) , is_compressed(is_compressed_) , query_string(queryStringFromAST(ast_)) - , tag(tag_) + , tag(settings.query_cache_tag) { } -QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_, const String & tag_) - : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false, tag_) /// dummy values for everything != AST, current database, user name/roles +QueryCache::Key::Key( + ASTPtr ast_, + const String & current_database, + const Settings & settings, + std::optional user_id_, + const std::vector & current_user_roles_) + : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) + /// ^^ dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 54de5edb145..c7ebaf4d26a 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -88,8 +88,9 @@ public: /// SYSTEM.QUERY_CACHE. const String query_string; - /// An arbitrary string to separate entries in the query cache. - /// Queries with different values of this setting are considered different. + /// A tag (namespace) for distinguish multiple entries of the same query. + /// This member has currently no use besides that SYSTEM.QUERY_CACHE can populate the 'tag' column conveniently without having to + /// compute the tag from the query AST. const String tag; /// Ctor to construct a Key for writing into query cache. @@ -100,15 +101,13 @@ public: std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, - bool is_compressed, - const String & tag_); + bool is_compressed); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). Key(ASTPtr ast_, const String & current_database, const Settings & settings, - std::optional user_id_, const std::vector & current_user_roles_, - const String & tag_); + std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 6422d3128fa..fe87eed5570 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1129,7 +1129,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles(), settings.query_cache_tag); + QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1258,8 +1258,7 @@ static std::tuple executeQueryImpl( context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), - settings.query_cache_compress_entries, - settings.query_cache_tag); + settings.query_cache_compress_entries); const size_t num_query_runs = settings.query_cache_min_query_runs ? query_cache->recordQueryRun(key) : 1; /// try to avoid locking a mutex in recordQueryRun() if (num_query_runs <= settings.query_cache_min_query_runs) diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index f81d50e8806..b3532ba40a7 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -16,12 +16,12 @@ ColumnsDescription StorageSystemQueryCache::getColumnsDescription() { {"query", std::make_shared(), "Query string."}, {"result_size", std::make_shared(), "Size of the query cache entry."}, + {"tag", std::make_shared(std::make_shared()), "Tag of the query cache entry."}, {"stale", std::make_shared(), "If the query cache entry is stale."}, {"shared", std::make_shared(), "If the query cache entry is shared between multiple users."}, {"compressed", std::make_shared(), "If the query cache entry is compressed."}, {"expires_at", std::make_shared(), "When the query cache entry becomes stale."}, - {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."}, - {"tag", std::make_shared(std::make_shared()), "An arbitrary string to separate entries in the query cache."} + {"key_hash", std::make_shared(), "A hash of the query string, used as a key to find query cache entries."} }; } @@ -53,12 +53,12 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[0]->insert(key.query_string); /// approximates the original query string res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result)); - res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); - res_columns[3]->insert(key.is_shared); - res_columns[4]->insert(key.is_compressed); - res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); - res_columns[6]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258) - res_columns[7]->insert(key.tag); + res_columns[2]->insert(key.tag); + res_columns[3]->insert(key.expires_at < std::chrono::system_clock::now()); + res_columns[4]->insert(key.is_shared); + res_columns[5]->insert(key.is_compressed); + res_columns[6]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); + res_columns[7]->insert(key.ast_hash.low64); /// query cache considers aliases (issue #56258) } } diff --git a/tests/queries/0_stateless/02494_query_cache_tag.reference b/tests/queries/0_stateless/02494_query_cache_tag.reference index 055d3d4c5bb..f7be5c06ecf 100644 --- a/tests/queries/0_stateless/02494_query_cache_tag.reference +++ b/tests/queries/0_stateless/02494_query_cache_tag.reference @@ -1,14 +1,12 @@ 1 -1 +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = \'abc\' abc --- 1 1 -1 -2 +SELECT 1 SETTINGS use_query_cache = true +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = \'abc\' abc --- 1 1 -1 -2 -1 -3 +SELECT 1 SETTINGS use_query_cache = true abc +SELECT 1 SETTINGS use_query_cache = true def diff --git a/tests/queries/0_stateless/02494_query_cache_tag.sql b/tests/queries/0_stateless/02494_query_cache_tag.sql index 054607058e8..62d36f6ebe6 100644 --- a/tests/queries/0_stateless/02494_query_cache_tag.sql +++ b/tests/queries/0_stateless/02494_query_cache_tag.sql @@ -3,30 +3,32 @@ SYSTEM DROP QUERY CACHE; --- Cache the query after the query invocation -SELECT 1 SETTINGS use_query_cache = true; -SELECT COUNT(*) FROM system.query_cache; +-- Store the result a single query with a tag in the query cache and check that the system table knows about the tag +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc'; + +SELECT query, tag FROM system.query_cache; SELECT '---'; SYSTEM DROP QUERY CACHE; --- Queries with tag value of this setting or not are considered different cache entries. -SELECT 1 SETTINGS use_query_cache = true; -SELECT COUNT(*) FROM system.query_cache; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; -SELECT COUNT(*) FROM system.query_cache; +-- Store the result of the same query with two different tags. The cache should store two entries. +SELECT 1 SETTINGS use_query_cache = true; -- default query_cache_tag = '' +SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc'; +SELECT query, tag FROM system.query_cache ORDER BY ALL; SELECT '---'; SYSTEM DROP QUERY CACHE; --- Queries with different tags values of this setting are considered different cache entries. +-- Like before but the tag is set standalone. + +SET query_cache_tag = 'abc'; SELECT 1 SETTINGS use_query_cache = true; -SELECT COUNT(*) FROM system.query_cache; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one'; -SELECT COUNT(*) FROM system.query_cache; -SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'one diff'; -SELECT COUNT(*) FROM system.query_cache; + +SET query_cache_tag = 'def'; +SELECT 1 SETTINGS use_query_cache = true; + +SELECT query, tag FROM system.query_cache ORDER BY ALL; SYSTEM DROP QUERY CACHE; From 38405dd7cdfb7189c1a1184c0eb8b3e23fda55e6 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 12 Aug 2024 18:14:22 +0000 Subject: [PATCH 0844/1722] add projection merge doc --- docs/en/operations/settings/merge-tree-settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 67fa45c20cd..a3bd919d3ce 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1041,3 +1041,14 @@ Compression rates of LZ4 or ZSTD improve on average by 20-40%. This setting works best for tables with no primary key or a low-cardinality primary key, i.e. a table with only few distinct primary key values. High-cardinality primary keys, e.g. involving timestamp columns of type `DateTime64`, are not expected to benefit from this setting. + +### deduplicate_merge_projection_mode + +Whether to allow create projection for the table with non-classic MergeTree, that is not (Replicated, Shared) MergeTree. If allowed, what is the action when merge projections, either drop or rebuild. So classic MergeTree would ignore this setting. +It also controls `OPTIMIZE DEDUPLICATE` as well, but has effect on all MergeTree family members. + +Possible values: + +- throw, drop, rebuild + +Default value: throw \ No newline at end of file From aa7a2bcb02f6c2f48bcc7acca3bcec2f1a16130b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 12 Aug 2024 20:34:02 +0200 Subject: [PATCH 0845/1722] Fix typo --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 21e4a6599ea..4f51dc6b8d3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -4073,7 +4073,7 @@ getSubcolumn(col_name, subcol_name) **Returned value** -- Returns the extracted sub-colum. +- Returns the extracted sub-column. **Example** From eab8594570e703a766f2f91ae3d13b0ed640b554 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 12 Aug 2024 20:35:33 +0200 Subject: [PATCH 0846/1722] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 862f38976ce..51246d990fa 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1697,6 +1697,8 @@ getOSKernelVersion getServerPort getSetting getSizeOfEnumType +getSubcolumn +getTypeSerializationStreams getblockinfo getevents ghcnd From 45a14fa0ce3ae94a374bbf955ba0fb7109b7e678 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 18:54:06 +0000 Subject: [PATCH 0847/1722] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ffd9fae7f45..03ec8e1752c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2115,6 +2115,7 @@ namenode namepassword nameprofile namequota +namespace namespaces natively nats From 469c1698b0dbf8a91a6e94a2bab0669f33bf7be2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 19:31:57 +0000 Subject: [PATCH 0848/1722] Fix 'Refresh set entry already exists' --- src/Interpreters/InterpreterSystemQuery.cpp | 17 +++-- src/Interpreters/InterpreterSystemQuery.h | 2 +- src/Storages/MaterializedView/RefreshSet.cpp | 75 +++++++++++-------- src/Storages/MaterializedView/RefreshSet.h | 23 +++--- src/Storages/MaterializedView/RefreshTask.cpp | 10 +-- src/Storages/MaterializedView/RefreshTask.h | 3 +- .../MaterializedView/RefreshTask_fwd.h | 1 + src/Storages/StorageMaterializedView.cpp | 1 - .../System/StorageSystemViewRefreshes.cpp | 3 + 9 files changed, 77 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index ef6d1040c5e..1cd55a0020c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -663,13 +663,16 @@ BlockIO InterpreterSystemQuery::execute() startStopAction(ActionLocks::ViewRefresh, false); break; case Type::REFRESH_VIEW: - getRefreshTask()->run(); + for (const auto & task : getRefreshTasks()) + task->run(); break; case Type::CANCEL_VIEW: - getRefreshTask()->cancel(); + for (const auto & task : getRefreshTasks()) + task->cancel(); break; case Type::TEST_VIEW: - getRefreshTask()->setFakeTime(query.fake_time_for_view); + for (const auto & task : getRefreshTasks()) + task->setFakeTime(query.fake_time_for_view); break; case Type::DROP_REPLICA: dropReplica(query); @@ -1242,15 +1245,15 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery & query) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYSTEM RESTART DISK is not supported"); } -RefreshTaskHolder InterpreterSystemQuery::getRefreshTask() +RefreshTaskList InterpreterSystemQuery::getRefreshTasks() { auto ctx = getContext(); ctx->checkAccess(AccessType::SYSTEM_VIEWS); - auto task = ctx->getRefreshSet().getTask(table_id); - if (!task) + auto tasks = ctx->getRefreshSet().findTasks(table_id); + if (tasks.empty()) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Refreshable view {} doesn't exist", table_id.getNameForLogs()); - return task; + return tasks; } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 776dd7915f0..f44fe930b04 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -74,7 +74,7 @@ private: void flushDistributed(ASTSystemQuery & query); [[noreturn]] void restartDisk(String & name); - RefreshTaskHolder getRefreshTask(); + RefreshTaskList getRefreshTasks(); AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index a3ef327dc24..43aa0ada99b 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -27,6 +27,7 @@ RefreshSet::Handle & RefreshSet::Handle::operator=(Handle && other) noexcept parent_set = std::exchange(other.parent_set, nullptr); id = std::move(other.id); dependencies = std::move(other.dependencies); + iter = std::move(other.iter); metric_increment = std::move(other.metric_increment); return *this; } @@ -39,21 +40,21 @@ RefreshSet::Handle::~Handle() void RefreshSet::Handle::rename(StorageID new_id) { std::lock_guard lock(parent_set->mutex); - parent_set->removeDependenciesLocked(id, dependencies); - auto it = parent_set->tasks.find(id); - auto task = it->second; - parent_set->tasks.erase(it); + RefreshTaskHolder task = *iter; + parent_set->removeDependenciesLocked(task, dependencies); + parent_set->removeTaskLocked(id, iter); id = new_id; - parent_set->tasks.emplace(id, task); - parent_set->addDependenciesLocked(id, dependencies); + iter = parent_set->addTaskLocked(id, task); + parent_set->addDependenciesLocked(task, dependencies); } void RefreshSet::Handle::changeDependencies(std::vector deps) { std::lock_guard lock(parent_set->mutex); - parent_set->removeDependenciesLocked(id, dependencies); + RefreshTaskHolder task = *iter; + parent_set->removeDependenciesLocked(task, dependencies); dependencies = std::move(deps); - parent_set->addDependenciesLocked(id, dependencies); + parent_set->addDependenciesLocked(task, dependencies); } void RefreshSet::Handle::reset() @@ -63,8 +64,8 @@ void RefreshSet::Handle::reset() { std::lock_guard lock(parent_set->mutex); - parent_set->removeDependenciesLocked(id, dependencies); - parent_set->tasks.erase(id); + parent_set->removeDependenciesLocked(*iter, dependencies); + parent_set->removeTaskLocked(id, iter); } parent_set = nullptr; @@ -76,37 +77,50 @@ RefreshSet::RefreshSet() = default; void RefreshSet::emplace(StorageID id, const std::vector & dependencies, RefreshTaskHolder task) { std::lock_guard guard(mutex); - auto [it, is_inserted] = tasks.emplace(id, task); - if (!is_inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Refresh set entry already exists for table {}", id.getFullTableName()); - addDependenciesLocked(id, dependencies); + const auto iter = addTaskLocked(id, task); + addDependenciesLocked(task, dependencies); - task->setRefreshSetHandleUnlock(Handle(this, id, dependencies)); + task->setRefreshSetHandleUnlock(Handle(this, id, iter, dependencies)); } -void RefreshSet::addDependenciesLocked(const StorageID & id, const std::vector & dependencies) +RefreshTaskList::iterator RefreshSet::addTaskLocked(StorageID id, RefreshTaskHolder task) +{ + RefreshTaskList & list = tasks[id]; + list.push_back(task); + return std::prev(list.end()); +} + +void RefreshSet::removeTaskLocked(StorageID id, RefreshTaskList::iterator iter) +{ + const auto it = tasks.find(id); + it->second.erase(iter); + if (it->second.empty()) + tasks.erase(it); +} + +void RefreshSet::addDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies) { for (const StorageID & dep : dependencies) - dependents[dep].insert(id); + dependents[dep].insert(task); } -void RefreshSet::removeDependenciesLocked(const StorageID & id, const std::vector & dependencies) +void RefreshSet::removeDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies) { for (const StorageID & dep : dependencies) { auto & set = dependents[dep]; - set.erase(id); + set.erase(task); if (set.empty()) dependents.erase(dep); } } -RefreshTaskHolder RefreshSet::getTask(const StorageID & id) const +RefreshTaskList RefreshSet::findTasks(const StorageID & id) const { std::lock_guard lock(mutex); - if (auto task = tasks.find(id); task != tasks.end()) - return task->second; - return nullptr; + if (auto it = tasks.find(id); it != tasks.end()) + return it->second; + return {}; } RefreshSet::InfoContainer RefreshSet::getInfo() const @@ -116,26 +130,23 @@ RefreshSet::InfoContainer RefreshSet::getInfo() const lock.unlock(); InfoContainer res; - for (const auto & [id, task] : tasks_copy) - res.push_back(task->getInfo()); + for (const auto & [id, list] : tasks_copy) + for (const auto & task : list) + res.push_back(task->getInfo()); return res; } std::vector RefreshSet::getDependents(const StorageID & id) const { std::lock_guard lock(mutex); - std::vector res; auto it = dependents.find(id); if (it == dependents.end()) return {}; - for (const StorageID & dep_id : it->second) - if (auto task = tasks.find(dep_id); task != tasks.end()) - res.push_back(task->second); - return res; + return std::vector(it->second.begin(), it->second.end()); } -RefreshSet::Handle::Handle(RefreshSet * parent_set_, StorageID id_, std::vector dependencies_) +RefreshSet::Handle::Handle(RefreshSet * parent_set_, StorageID id_, RefreshTaskList::iterator iter_, std::vector dependencies_) : parent_set(parent_set_), id(std::move(id_)), dependencies(std::move(dependencies_)) - , metric_increment(CurrentMetrics::Increment(CurrentMetrics::RefreshableViews)) {} + , iter(iter_), metric_increment(CurrentMetrics::Increment(CurrentMetrics::RefreshableViews)) {} } diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index eff445023a6..7fb583fd316 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -5,12 +5,11 @@ #include #include #include +#include namespace DB { -using DatabaseAndTableNameSet = std::unordered_set; - enum class RefreshState : RefreshTaskStateUnderlying { Disabled = 0, @@ -46,8 +45,7 @@ struct RefreshInfo class RefreshSet { public: - /// RAII thing that unregisters a task and its dependencies in destructor. - /// Storage IDs must be unique. Not thread safe. + /// RAII thing that unregisters a task and its dependencies in destructor. Not thread safe. class Handle { friend class RefreshSet; @@ -73,9 +71,10 @@ public: RefreshSet * parent_set = nullptr; StorageID id = StorageID::createEmpty(); std::vector dependencies; + RefreshTaskList::iterator iter; // in parent_set->tasks[id] std::optional metric_increment; - Handle(RefreshSet * parent_set_, StorageID id_, std::vector dependencies_); + Handle(RefreshSet * parent_set_, StorageID id_, RefreshTaskList::iterator iter_, std::vector dependencies_); }; using InfoContainer = std::vector; @@ -84,7 +83,9 @@ public: void emplace(StorageID id, const std::vector & dependencies, RefreshTaskHolder task); - RefreshTaskHolder getTask(const StorageID & id) const; + /// Finds active refreshable view(s) by database and table name. + /// Normally there's at most one, but we allow name collisions here, just in case. + RefreshTaskList findTasks(const StorageID & id) const; InfoContainer getInfo() const; @@ -92,8 +93,8 @@ public: std::vector getDependents(const StorageID & id) const; private: - using TaskMap = std::unordered_map; - using DependentsMap = std::unordered_map; + using TaskMap = std::unordered_map; + using DependentsMap = std::unordered_map, StorageID::DatabaseAndTableNameHash, StorageID::DatabaseAndTableNameEqual>; /// Protects the two maps below, not locked for any nontrivial operations (e.g. operations that /// block or lock other mutexes). @@ -102,8 +103,10 @@ private: TaskMap tasks; DependentsMap dependents; - void addDependenciesLocked(const StorageID & id, const std::vector & dependencies); - void removeDependenciesLocked(const StorageID & id, const std::vector & dependencies); + RefreshTaskList::iterator addTaskLocked(StorageID id, RefreshTaskHolder task); + void removeTaskLocked(StorageID id, RefreshTaskList::iterator iter); + void addDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies); + void removeDependenciesLocked(RefreshTaskHolder task, const std::vector & dependencies); }; } diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index aa8f51d5295..0837eaf97fd 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -33,7 +33,6 @@ RefreshTask::RefreshTask( {} RefreshTaskHolder RefreshTask::create( - const StorageMaterializedView & view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) { @@ -46,12 +45,9 @@ RefreshTaskHolder RefreshTask::create( t->refreshTask(); }); - std::vector deps; if (strategy.dependencies) for (auto && dependency : strategy.dependencies->children) - deps.emplace_back(dependency->as()); - - context->getRefreshSet().emplace(view.getStorageID(), deps, task); + task->initial_dependencies.emplace_back(dependency->as()); return task; } @@ -61,6 +57,7 @@ void RefreshTask::initializeAndStart(std::shared_ptr vi view_to_refresh = view; if (view->getContext()->getSettingsRef().stop_refreshable_materialized_views_on_startup) stop_requested = true; + view->getContext()->getRefreshSet().emplace(view->getStorageID(), initial_dependencies, shared_from_this()); populateDependencies(); advanceNextRefreshTime(currentTime()); refresh_task->schedule(); @@ -69,7 +66,8 @@ void RefreshTask::initializeAndStart(std::shared_ptr vi void RefreshTask::rename(StorageID new_id) { std::lock_guard guard(mutex); - set_handle.rename(new_id); + if (set_handle) + set_handle.rename(new_id); } void RefreshTask::alterRefreshParams(const DB::ASTRefreshStrategy & new_strategy) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 1f050a97cd9..623493f6aec 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -26,7 +26,6 @@ public: /// The only proper way to construct task static RefreshTaskHolder create( - const StorageMaterializedView & view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); @@ -84,9 +83,11 @@ private: RefreshSchedule refresh_schedule; RefreshSettings refresh_settings; // TODO: populate, use, update on alter + std::vector initial_dependencies; RefreshSet::Handle set_handle; /// StorageIDs of our dependencies that we're waiting for. + using DatabaseAndTableNameSet = std::unordered_set; DatabaseAndTableNameSet remaining_dependencies; bool time_arrived = false; diff --git a/src/Storages/MaterializedView/RefreshTask_fwd.h b/src/Storages/MaterializedView/RefreshTask_fwd.h index 1f366962eb6..9a0a122381e 100644 --- a/src/Storages/MaterializedView/RefreshTask_fwd.h +++ b/src/Storages/MaterializedView/RefreshTask_fwd.h @@ -11,5 +11,6 @@ class RefreshTask; using RefreshTaskStateUnderlying = UInt8; using RefreshTaskHolder = std::shared_ptr; using RefreshTaskObserver = std::weak_ptr; +using RefreshTaskList = std::list; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 696136834d4..4c6c2fff209 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -203,7 +203,6 @@ StorageMaterializedView::StorageMaterializedView( { fixed_uuid = false; refresher = RefreshTask::create( - *this, getContext(), *query.refresh_strategy); refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 30539ed6b6a..061201017a7 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,7 @@ ColumnsDescription StorageSystemViewRefreshes::getColumnsDescription() { {"database", std::make_shared(), "The name of the database the table is in."}, {"view", std::make_shared(), "Table name."}, + {"uuid", std::make_shared(), "Table uuid (Atomic database)."}, {"status", std::make_shared(), "Current state of the refresh."}, {"last_refresh_result", std::make_shared(), "Outcome of the latest refresh attempt."}, {"last_refresh_time", std::make_shared(std::make_shared()), @@ -63,6 +65,7 @@ void StorageSystemViewRefreshes::fillData( std::size_t i = 0; res_columns[i++]->insert(refresh.view_id.getDatabaseName()); res_columns[i++]->insert(refresh.view_id.getTableName()); + res_columns[i++]->insert(refresh.view_id.uuid); res_columns[i++]->insert(toString(refresh.state)); res_columns[i++]->insert(toString(refresh.last_refresh_result)); From 5a683796a0dc8408ed2694af672675929352bf8f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 12 Aug 2024 22:34:14 +0200 Subject: [PATCH 0849/1722] Update DatabaseReplicated.cpp --- src/Databases/DatabaseReplicated.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f4aa925d6dd..6011b8e65e3 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1584,6 +1584,8 @@ void DatabaseReplicated::dropTable(ContextPtr local_context, const String & tabl } auto table = tryGetTable(table_name, getContext()); + if (!table) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} doesn't exist", table_name); if (table->getName() == "MaterializedView" || table->getName() == "WindowView") { /// Avoid recursive locking of metadata_mutex From c2185606398526716bf4505b4e359be0beb605cc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 21:14:42 +0000 Subject: [PATCH 0850/1722] Don't enable allow_materialized_view_with_bad_select yet, someone has to add support for fixture reuse in test_replicated_database first --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fee60ec7981..cb61f829da3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,7 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ + M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index dce51b30382..cae37e7ddde 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,7 +78,7 @@ static std::initializer_list Date: Mon, 12 Aug 2024 21:20:57 +0000 Subject: [PATCH 0851/1722] fix drift of profile event time --- src/Storages/MergeTree/MergeTask.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index cb1921ede2b..3aa4d764685 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -944,8 +944,13 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const MergeTask::StageRuntimeContextPtr MergeTask::MergeProjectionsStage::getContextForNextStage() { - ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); - ProfileEvents::increment(ProfileEvents::MergeProjectionStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + /// Do not increment for projection stage because time is already accounted in main task. + /// The projection stage has its own empty projection stage which may add a drift of severals milliseconds. + if (global_ctx->parent_part == nullptr) + { + ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + ProfileEvents::increment(ProfileEvents::MergeProjectionStageExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); + } return nullptr; } @@ -1034,6 +1039,8 @@ bool MergeTask::execute() UInt64 stage_elapsed_ms = current_elapsed_ms - global_ctx->prev_elapsed_ms; global_ctx->prev_elapsed_ms = current_elapsed_ms; + auto next_stage_context = current_stage->getContextForNextStage(); + /// Do not increment for projection stage because time is already accounted in main task. if (global_ctx->parent_part == nullptr) { @@ -1041,8 +1048,6 @@ bool MergeTask::execute() ProfileEvents::increment(ProfileEvents::MergeTotalMilliseconds, stage_elapsed_ms); } - auto next_stage_context = current_stage->getContextForNextStage(); - /// Move to the next stage in an array of stages ++stages_iterator; if (stages_iterator == stages.end()) From 8136e6a45275b958a663ac0ee4682984e1536b07 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 21:29:26 +0000 Subject: [PATCH 0852/1722] Update new prepareForSquashing method for ColumnDynamic --- src/Columns/ColumnDynamic.cpp | 23 ++++++++++--------- src/DataTypes/DataTypeVariant.cpp | 2 +- .../03210_dynamic_squashing.reference | 20 +++++++++------- .../0_stateless/03210_dynamic_squashing.sql | 17 ++++++++------ 4 files changed, 35 insertions(+), 27 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index ecc2c738366..69b4c5dfc4e 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -987,7 +987,8 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) /// Internal variants of source dynamic columns may differ. /// We want to preallocate memory for all variants we will have after squashing. /// It may happen that the total number of variants in source columns will - /// exceed the limit, in this case we will choose the most frequent variants. + /// exceed the limit, in this case we will choose the most frequent variants + /// and insert the rest types into the shared variant. /// First, preallocate memory for variant discriminators and offsets. size_t new_size = size(); @@ -1030,17 +1031,14 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) DataTypePtr result_variant_type; /// Check if the number of all variants exceeds the limit. - if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_variant_sizes.contains("String"))) + if (!canAddNewVariants(0, all_variants.size())) { /// We want to keep the most frequent variants in the resulting dynamic column. DataTypes result_variants; - result_variants.reserve(max_dynamic_types); + result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant. /// Add variants from current variant column as we will not rewrite it. for (const auto & variant : assert_cast(*variant_info.variant_type).getVariants()) result_variants.push_back(variant); - /// Add String variant in advance (if we didn't add it yet) as we must have it across variants when we reach the limit. - if (!variant_info.variant_name_to_discriminator.contains("String")) - result_variants.push_back(std::make_shared()); /// Create list of remaining variants with their sizes and sort it. std::vector> variants_with_sizes; @@ -1049,15 +1047,18 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { /// Add variant to the list only of we didn't add it yet. auto variant_name = variant->getName(); - if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name)) - variants_with_sizes.emplace_back(total_variant_sizes[variant->getName()], variant); + if (!variant_info.variant_name_to_discriminator.contains(variant_name)) + variants_with_sizes.emplace_back(total_variant_sizes[variant_name], variant); } std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); /// Add the most frequent variants until we reach max_dynamic_types. - size_t num_new_variants = max_dynamic_types - result_variants.size(); - for (size_t i = 0; i != num_new_variants; ++i) - result_variants.push_back(variants_with_sizes[i].second); + for (const auto & [_, new_variant] : variants_with_sizes) + { + if (!canAddNewVariant(result_variants.size())) + break; + result_variants.push_back(new_variant); + } result_variant_type = std::make_shared(result_variants); } diff --git a/src/DataTypes/DataTypeVariant.cpp b/src/DataTypes/DataTypeVariant.cpp index 8a10ca7d06d..cc8d04e94da 100644 --- a/src/DataTypes/DataTypeVariant.cpp +++ b/src/DataTypes/DataTypeVariant.cpp @@ -117,7 +117,7 @@ bool DataTypeVariant::equals(const IDataType & rhs) const /// The same data types with different custom names considered different. /// For example, UInt8 and Bool. - if ((variants[i]->hasCustomName() || rhs_variant.variants[i]) && variants[i]->getName() != rhs_variant.variants[i]->getName()) + if ((variants[i]->hasCustomName() || rhs_variant.variants[i]->hasCustomName()) && variants[i]->getName() != rhs_variant.variants[i]->getName()) return false; } diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.reference b/tests/queries/0_stateless/03210_dynamic_squashing.reference index 4f5b5ba098c..1c23c22f550 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.reference +++ b/tests/queries/0_stateless/03210_dynamic_squashing.reference @@ -1,8 +1,12 @@ -Array(UInt8) -None -UInt64 -None -String -UInt64 -String -UInt64 +1 +Array(UInt8) true +None false +UInt64 false +2 +Array(UInt8) true +None false +UInt64 false +3 +Array(UInt8) true +String false +UInt64 true diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql index 23b47184e33..da3b911e796 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.sql +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -4,17 +4,20 @@ set max_block_size = 1000; drop table if exists test; create table test (d Dynamic) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); -select distinct dynamicType(d) as type from test order by type; +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +select '1'; +select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; drop table test; -create table test (d Dynamic(max_types=2)) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); -select distinct dynamicType(d) as type from test order by type; +create table test (d Dynamic(max_types=1)) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +select '2'; +select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; truncate table test; -insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=2), number < 3000, range(number % 5)::Dynamic(max_types=2), number::Dynamic(max_types=2)) from numbers(1000000); -select distinct dynamicType(d) as type from test order by type; +insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +select '3'; +select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; drop table test; From 83cb991f75f242b11beb48134d6ebfb26c73bcd7 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Aug 2024 21:30:30 +0000 Subject: [PATCH 0853/1722] Fix special build --- src/Columns/ColumnDynamic.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index d80055c1716..e6e720765f6 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -368,10 +368,10 @@ public: /// Check if we can add new variant types. /// Shared variant doesn't count in the limit but always presents, /// so we should subtract 1 from the total types count. - bool canAddNewVariants(size_t current_variants_count, size_t new_variants_count) { return current_variants_count + new_variants_count - 1 <= max_dynamic_types; } - bool canAddNewVariant(size_t current_variants_count) { return canAddNewVariants(current_variants_count, 1); } - bool canAddNewVariants(size_t new_variants_count) { return canAddNewVariants(variant_info.variant_names.size(), new_variants_count); } - bool canAddNewVariant() { return canAddNewVariants(variant_info.variant_names.size(), 1); } + bool canAddNewVariants(size_t current_variants_count, size_t new_variants_count) const { return current_variants_count + new_variants_count - 1 <= max_dynamic_types; } + bool canAddNewVariant(size_t current_variants_count) const { return canAddNewVariants(current_variants_count, 1); } + bool canAddNewVariants(size_t new_variants_count) const { return canAddNewVariants(variant_info.variant_names.size(), new_variants_count); } + bool canAddNewVariant() const { return canAddNewVariants(variant_info.variant_names.size(), 1); } void setVariantType(const DataTypePtr & variant_type); void setMaxDynamicPaths(size_t max_dynamic_type_); From 3b1d6f30bec5a8a568ab477e639d97c9c95a3f2c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 23:55:01 +0200 Subject: [PATCH 0854/1722] Debug test --- .../0_stateless/02490_benchmark_max_consecutive_errors.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh index f747b3156a5..df7e9386662 100755 --- a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh +++ b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh @@ -11,5 +11,6 @@ if [ "$RES" -eq 10 ] then echo "$RES" else + echo "$RES" cat "${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}.log" fi From 9c7d9a6a8d96b88c56aaa95b691f2b9bf79cf8d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 23:57:03 +0200 Subject: [PATCH 0855/1722] Annotations --- tests/queries/0_stateless/02293_ttest_large_samples.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02293_ttest_large_samples.sql b/tests/queries/0_stateless/02293_ttest_large_samples.sql index 826bd483fe9..b4687541360 100644 --- a/tests/queries/0_stateless/02293_ttest_large_samples.sql +++ b/tests/queries/0_stateless/02293_ttest_large_samples.sql @@ -1,3 +1,5 @@ +-- Tags: long + SELECT roundBankers(result.1, 5), roundBankers(result.2, 5) FROM ( SELECT studentTTest(sample, variant) as result From 1767ec6b4ca0fc0e8546e705e0d0dff3ffa797cb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Aug 2024 23:55:01 +0200 Subject: [PATCH 0856/1722] Debug test --- .../0_stateless/02490_benchmark_max_consecutive_errors.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh index f747b3156a5..df7e9386662 100755 --- a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh +++ b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh @@ -11,5 +11,6 @@ if [ "$RES" -eq 10 ] then echo "$RES" else + echo "$RES" cat "${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}.log" fi From 11d5531f829630b64566536526bd1c6369f2005e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 23:06:26 +0000 Subject: [PATCH 0857/1722] allow_materialized_view_with_bad_select=0 in test --- .../0_stateless/02932_refreshable_materialized_views_2.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh index 2a803114842..50a905576d5 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh @@ -10,7 +10,7 @@ CLICKHOUSE_LOG_COMMENT= # Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh # scheduling is done in UTC. CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" -CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --allow_materialized_view_with_bad_select=0 --session_timezone Etc/UTC"`" $CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" From f12609440f081f19b0b21fdd15229cfdbb7cbb3d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 23:09:57 +0000 Subject: [PATCH 0858/1722] fashion --- src/Storages/MaterializedView/RefreshSet.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index 43aa0ada99b..7536f59c1e4 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -9,11 +9,6 @@ namespace CurrentMetrics namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - RefreshSet::Handle::Handle(Handle && other) noexcept { *this = std::move(other); From ddd5a96950528513eec78d56b103435964576b12 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 23:56:02 +0000 Subject: [PATCH 0859/1722] Fix --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 111 ++++++++++-------- 1 file changed, 64 insertions(+), 47 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 1268f1df5f6..38d5094b267 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -63,7 +63,7 @@ namespace ErrorCodes /// * We dispatch based on the parquet logical+converted+physical type instead of the ClickHouse type. /// The idea is that this is similar to what we'll have to do when reimplementing Parquet parsing in /// ClickHouse instead of using Arrow (for speed). So, this is an exercise in parsing Parquet manually. -static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type::type physical_type, const parquet::ColumnDescriptor & descr) +static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type::type physical_type, const parquet::ColumnDescriptor & descr, TypeIndex type_hint) { using namespace parquet; @@ -127,20 +127,19 @@ static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type if (size < 32 && (val >> (size * 8 - 1)) != 0) val |= ~((Int256(1) << (size * 8)) - 1); - auto narrow = [&]() -> Field { - using T = typename D::NativeType; - T x = 0; - memcpy(&x, &val, sizeof(T)); - return Field(DecimalField(D(x), static_cast(scale))); + auto narrow = [&](auto x) -> Field + { + memcpy(&x, &val, sizeof(x)); + return Field(DecimalField(x, static_cast(scale))); }; if (size <= 4) - return narrow.template operator()(); + return narrow(Decimal32(0)); else if (size <= 8) - return narrow.template operator()(); + return narrow(Decimal64(0)); else if (size <= 16) - return narrow.template operator()(); + return narrow(Decimal128(0)); else - return narrow.template operator()(); + return narrow(Decimal256(0)); } while (false); @@ -197,8 +196,6 @@ static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type return Field(val); } - /// Strings. - if (physical_type == Type::type::BYTE_ARRAY || physical_type == Type::type::FIXED_LEN_BYTE_ARRAY) { /// Arrow's parquet decoder handles missing min/max values slightly incorrectly. @@ -227,6 +224,25 @@ static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type if (data.empty()) return Field(); + /// Long integers. + auto reinterpret_fixed_string = [&](auto x) + { + if (data.size() != sizeof(x)) + throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected {} size: {}", fieldTypeToString(Field::TypeToEnum::value), data.size()); + memcpy(&x, data.data(), data.size()); + return Field(x); + }; + switch (type_hint) + { + case TypeIndex::UInt128: return reinterpret_fixed_string(UInt128(0)); + case TypeIndex::UInt256: return reinterpret_fixed_string(UInt256(0)); + case TypeIndex::Int128: return reinterpret_fixed_string(Int128(0)); + case TypeIndex::Int256: return reinterpret_fixed_string(Int256(0)); + case TypeIndex::IPv6: return reinterpret_fixed_string(IPv6(0)); + default: break; + } + + /// Strings. return Field(data); } @@ -302,6 +318,7 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa if (type->isNullable()) type = assert_cast(*type).getNestedType(); Field default_value = type->getDefault(); + TypeIndex type_index = type->getTypeId(); /// Only primitive fields are supported, not arrays, maps, tuples, or Nested. /// Arrays, maps, and Nested can't be meaningfully supported because Parquet only has min/max @@ -315,8 +332,41 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa { try { - min = decodePlainParquetValueSlow(stats->EncodeMin(), stats->physical_type(), *stats->descr()); - max = decodePlainParquetValueSlow(stats->EncodeMax(), stats->physical_type(), *stats->descr()); + min = decodePlainParquetValueSlow(stats->EncodeMin(), stats->physical_type(), *stats->descr(), type_index); + max = decodePlainParquetValueSlow(stats->EncodeMax(), stats->physical_type(), *stats->descr(), type_index); + + /// If the data type in parquet file substantially differs from the requested data type, + /// it's sometimes correct to just typecast the min/max values. + /// Other times it's incorrect, e.g.: + /// INSERT INTO FUNCTION file('t.parquet', Parquet, 'x String') VALUES ('1'), ('100'), ('2'); + /// SELECT * FROM file('t.parquet', Parquet, 'x Int64') WHERE x >= 3; + /// If we just typecast min/max from string to integer, this query will incorrectly return empty result. + /// Allow conversion in some simple cases, otherwise ignore the min/max values. + auto min_type = min.getType(); + auto max_type = max.getType(); + min = convertFieldToType(min, *type); + max = convertFieldToType(max, *type); + auto ok_cast = [&](Field::Types::Which from, Field::Types::Which to) -> bool + { + if (from == to) + return true; + /// Decimal -> wider decimal. + if (Field::isDecimal(from) || Field::isDecimal(to)) + return Field::isDecimal(from) && Field::isDecimal(to) && to >= from; + /// Integer -> IP. + if (to == Field::Types::IPv4) + return from == Field::Types::UInt64; + /// Disable index for everything else, especially string <-> number. + return false; + }; + if (!(ok_cast(min_type, min.getType()) && ok_cast(max_type, max.getType())) && + !(min == max) && + !(min_type == Field::Types::Int64 && min.getType() == Field::Types::UInt64 && min.get() >= 0) && + !(max_type == Field::Types::UInt64 && max.getType() == Field::Types::Int64 && max.get() <= UInt64(INT64_MAX))) + { + min = Field(); + max = Field(); + } } catch (Exception & e) { @@ -325,39 +375,6 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa } } - /// If the data type in parquet file substantially differs from the requested data type, - /// it's sometimes correct to just typecast the min/max values. - /// Other times it's incorrect, e.g.: - /// INSERT INTO FUNCTION file('t.parquet', Parquet, 'x String') VALUES ('1'), ('100'), ('2'); - /// SELECT * FROM file('t.parquet', Parquet, 'x Int64') WHERE x >= 3; - /// If we just typecast min/max from string to integer, this query will incorrectly return empty result. - /// Allow conversion in some simple cases and ignore the min/max values otherwise. - auto min_type = min.getType(); - auto max_type = max.getType(); - min = convertFieldToType(min, *type); - max = convertFieldToType(max, *type); - auto ok_cast = [&](Field::Types::Which from, Field::Types::Which to) -> bool - { - if (from == to) - return true; - /// Decimal -> wider decimal. - if (Field::isDecimal(from) || Field::isDecimal(to)) - return Field::isDecimal(from) && Field::isDecimal(to) && to >= from; - /// Integer -> IP. - if (to == Field::Types::IPv4 || to == Field::Types::IPv6) - return from == Field::Types::UInt64 || from == Field::Types::Int64; - /// Disable index for everything else, especially string <-> number. - return false; - }; - if (!(ok_cast(min_type, min.getType()) && ok_cast(max_type, max.getType())) && - !(min == max) && - !(min_type == Field::Types::Int64 && min.getType() == Field::Types::UInt64 && min.get() >= 0) && - !(max_type == Field::Types::UInt64 && max.getType() == Field::Types::Int64 && max.get() <= UInt64(INT64_MAX))) - { - min = Field(); - max = Field(); - } - /// In Range, NULL is represented as positive or negative infinity (represented by a special /// kind of Field, different from floating-point infinities). From f16c969352d2938cda1b1953af0f7b895d732873 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Aug 2024 02:02:15 +0200 Subject: [PATCH 0860/1722] Fix flaky check --- tests/queries/0_stateless/00600_replace_running_query.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index ad7c49a9ad3..4e7ed7c8cfa 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none From c3e32bbecf0497a5eae6ab0cf23ba641591cfa0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Aug 2024 02:07:51 +0200 Subject: [PATCH 0861/1722] Fix flaky check --- .../00600_replace_running_query.sh | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index 4e7ed7c8cfa..80e2ecf5d5b 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none @@ -7,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -TEST_PREFIX=$RANDOM +TEST_PREFIX="${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600${TEST_PREFIX}" ${CLICKHOUSE_CLIENT} -q "create user u_00600${TEST_PREFIX} settings max_execution_time=60, readonly=1" ${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600${TEST_PREFIX}" @@ -29,34 +28,34 @@ function wait_for_queries_to_finish() } -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d 'SELECT 1, count() FROM system.numbers' > /dev/null 2>&1 & -wait_for_query_to_start 'hello' +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=${CLICKHOUSE_DATABASE}hello&replace_running_query=1" -d 'SELECT 1, count() FROM system.numbers' > /dev/null 2>&1 & +wait_for_query_to_start "${CLICKHOUSE_DATABASE}hello" # Replace it -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d 'SELECT 0' +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=${CLICKHOUSE_DATABASE}hello&replace_running_query=1" -d 'SELECT 0' # Wait for it to be replaced wait wait_for_queries_to_finish -${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' & -wait_for_query_to_start '42' +${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id="${CLICKHOUSE_DATABASE}42" --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' & +wait_for_query_to_start "${CLICKHOUSE_DATABASE}42" # Trying to run another query with the same query_id -${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 43' 2>&1 | grep -cF 'is already running by user' +${CLICKHOUSE_CLIENT} --query_id="${CLICKHOUSE_DATABASE}42" --query='SELECT 43' 2>&1 | grep -cF 'is already running by user' # Trying to replace query of a different user -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=42&replace_running_query=1" -d 'SELECT 1' | grep -cF 'is already running by user' +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=${CLICKHOUSE_DATABASE}42&replace_running_query=1" -d 'SELECT 1' | grep -cF 'is already running by user' -$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" > /dev/null +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}42' SYNC" > /dev/null wait wait_for_queries_to_finish -${CLICKHOUSE_CLIENT} --query_id=42 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' & -wait_for_query_to_start '42' -${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null +${CLICKHOUSE_CLIENT} --query_id="${CLICKHOUSE_DATABASE}42" --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' & +wait_for_query_to_start "${CLICKHOUSE_DATABASE}42" +${CLICKHOUSE_CLIENT} --query_id="${CLICKHOUSE_DATABASE}42" --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null wait wait_for_queries_to_finish -${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --query='SELECT 44' +${CLICKHOUSE_CLIENT} --query_id="${CLICKHOUSE_DATABASE}42" --replace_running_query=1 --query='SELECT 44' ${CLICKHOUSE_CLIENT} -q "drop user u_00600${TEST_PREFIX}" From 5e6f728248396fefd1b003555558977cb79a592e Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 13 Aug 2024 00:35:40 +0000 Subject: [PATCH 0862/1722] safeGet --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 38d5094b267..3f2e701afc2 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -361,8 +361,8 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa }; if (!(ok_cast(min_type, min.getType()) && ok_cast(max_type, max.getType())) && !(min == max) && - !(min_type == Field::Types::Int64 && min.getType() == Field::Types::UInt64 && min.get() >= 0) && - !(max_type == Field::Types::UInt64 && max.getType() == Field::Types::Int64 && max.get() <= UInt64(INT64_MAX))) + !(min_type == Field::Types::Int64 && min.getType() == Field::Types::UInt64 && min.safeGet() >= 0) && + !(max_type == Field::Types::UInt64 && max.getType() == Field::Types::Int64 && max.safeGet() <= UInt64(INT64_MAX))) { min = Field(); max = Field(); From b80dd738b03c1618e0872b14472457a2462c334e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Aug 2024 02:41:00 +0200 Subject: [PATCH 0863/1722] Fix error --- src/IO/ReadHelpers.h | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 580aa51b238..d63ba16e080 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -259,12 +259,18 @@ inline void readBoolText(bool & x, ReadBuffer & buf) readChar(tmp, buf); x = tmp != '0'; - if (!buf.eof() && isAlphaASCII(*buf.position())) + if (!buf.eof() && isAlphaASCII(tmp)) { - if (tmp == 't') - assertString("rue", buf); - else if (tmp == 'T') - assertString("RUE", buf); + if (tmp == 't' || tmp == 'T') + { + assertStringCaseInsensitive("rue", buf); + x = true; + } + else if (tmp == 'f' || tmp == 'F') + { + assertStringCaseInsensitive("alse", buf); + x = false; + } } } From a517bc90cd9e369a4385f367e9f5e9688520c8bb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 12 Aug 2024 21:42:47 -0400 Subject: [PATCH 0864/1722] Update PULL_REQUEST_TEMPLATE.md --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 8b6e957e1d8..3dcce68ab46 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -60,7 +60,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with aarch64, release, debug --- - [ ] Run only fuzzers related jobs (libFuzzer fuzzers, AST fuzzers, etc.) -- [ ] Exclude AST fuzzers +- [ ] Exclude: AST fuzzers --- - [ ] Do not test - [ ] Woolen Wolfdog From cb7a67c6dd9df40d728108c37bfd423435b45141 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 13 Aug 2024 11:39:22 +0800 Subject: [PATCH 0865/1722] fix failed uts --- .../02210_toColumnTypeName_toLowCardinality_const.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference index 1e3d3a50562..e3978020431 100644 --- a/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference +++ b/tests/queries/0_stateless/02210_toColumnTypeName_toLowCardinality_const.reference @@ -1 +1 @@ -Const(ColumnLowCardinality) +Const(LowCardinality(UInt8)) From 761a28502ef79787244ad1da7b15b3fedfb24221 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 13 Aug 2024 05:17:18 +0000 Subject: [PATCH 0866/1722] Unchange integration test --- tests/integration/test_replicated_database/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index a9b178302f9..60a6e099b22 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -712,7 +712,6 @@ def create_some_tables(db): "distributed_ddl_task_timeout": 0, "allow_experimental_object_type": 1, "allow_suspicious_codecs": 1, - "allow_materialized_view_with_bad_select": 1, } main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( From 2a51b6c403c8f86fb9e68f358ca490630c40fec6 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 13 Aug 2024 10:20:05 +0800 Subject: [PATCH 0867/1722] fix crash in lag/lead --- src/Processors/Transforms/WindowTransform.cpp | 6 ++++++ .../03210_lag_lead_inframe_types.reference | 16 +++++++++++++++ .../03210_lag_lead_inframe_types.sql | 20 +++++++++++++++++++ 3 files changed, 42 insertions(+) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 85e6b2ec55e..cae817380e0 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1159,6 +1159,12 @@ void WindowTransform::appendChunk(Chunk & chunk) { if (ws.window_function_impl) block.casted_columns.push_back(ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices)); + else + { + /// `castColumn` returns nullptr at default, so it's OK to put nullptr as a placeholder here + /// it should not be used in fact. + block.casted_columns.push_back(nullptr); + } block.output_columns.push_back(ws.aggregate_function->getResultType() ->createColumn()); diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference index d4734a85e72..4ecf7f56b07 100644 --- a/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.reference @@ -38,3 +38,19 @@ 7 8 9 +15 \N 3 15 15 15 15 +14 \N 2 10 10 10 154 +13 \N 2 10 10 10 143 +12 \N 2 10 10 10 14 +11 \N 2 10 10 10 12 +10 \N 2 10 10 10 10 +9 \N 1 5 5 5 99 +8 \N 1 5 5 5 88 +7 \N 1 5 5 5 9 +6 \N 1 5 5 5 7 +5 \N 1 5 5 5 5 +4 \N 0 0 0 0 44 +3 \N 0 0 0 0 33 +2 \N 0 0 0 0 4 +1 \N 0 0 0 0 2 +0 \N 0 0 0 0 0 diff --git a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql index f6017ee6690..cc6746e428f 100644 --- a/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql +++ b/tests/queries/0_stateless/03210_lag_lead_inframe_types.sql @@ -2,3 +2,23 @@ SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (OR SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); SELECT lagInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); SELECT leadInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number); + +SELECT + number, + YYYYMMDDToDate(1, toLowCardinality(11), max(YYYYMMDDToDate(YYYYMMDDToDate(toLowCardinality(1), 11, materialize(NULL), 19700101.1, 1, 27, 7, materialize(toUInt256(37)), 9, 19, 9), 1, toUInt128(11), NULL, 19700101.1, 1, 27, 7, 37, 9, 19, 9), toUInt256(30)) IGNORE NULLS OVER w, NULL, 19700101.1, toNullable(1), 27, materialize(7), 37, 9, 19, 9), + p, + pp, + lagInFrame(number, number - pp) OVER w AS lag2, + lagInFrame(number, number - pp, number * 11) OVER w AS lag, + leadInFrame(number, number - pp, number * 11) OVER w AS lead +FROM +( + SELECT + number, + intDiv(number, 5) AS p, + p * 5 AS pp + FROM numbers(16) +) +WHERE toLowCardinality(1) +WINDOW w AS (PARTITION BY p ORDER BY number ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY number DESC NULLS LAST; From a0f617c6cc06cd80ecbb485965f6b7e7763be18c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Aug 2024 10:14:37 +0200 Subject: [PATCH 0868/1722] tests: make 01600_parts_states_metrics_long better - better bash - HTTP protocol cannot handle multiple queries fix this - decrease number of retries (this should be ok after no-parallel) to print final debug info Signed-off-by: Azat Khuzhin --- .../01600_parts_states_metrics_long.sh | 43 +++++++++++-------- 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index a07dd306b3e..0a9f94cc451 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -11,33 +11,40 @@ function query() ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&database_atomic_wait_for_drop_and_detach_synchronously=1" -d "$*" } -# NOTE: database = $CLICKHOUSE_DATABASE is unwanted -verify_sql="SELECT - (SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) - = (SELECT sum(active), sum(NOT active) FROM - (SELECT active FROM system.parts UNION ALL SELECT active FROM system.projection_parts UNION ALL SELECT 1 FROM system.dropped_tables_parts))" # The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. # So, there is inherent race condition. But it should get expected result eventually. # In case of test failure, this code will do infinite loop and timeout. verify() { - for i in {1..5000} - do - result=$( query "$verify_sql" ) - [ "$result" = "1" ] && echo "$result" && break - sleep 0.1 + local result - if [[ $i -eq 5000 ]] - then - query " - SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; - SELECT sum(active), sum(NOT active) FROM system.parts; - SELECT sum(active), sum(NOT active) FROM system.projection_parts; - SELECT count() FROM system.dropped_tables_parts; - " + for _ in {1..100}; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + result=$( query "SELECT + (SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) + = + (SELECT sum(active), sum(NOT active) FROM ( + SELECT active FROM system.parts + UNION ALL SELECT active FROM system.projection_parts + UNION ALL SELECT 1 FROM system.dropped_tables_parts + ))" + ) + + if [ "$result" = "1" ]; then + echo "$result" + return fi + + sleep 0.5 done + + $CLICKHOUSE_CLIENT -q " + SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics; + SELECT sum(active), sum(NOT active) FROM system.parts; + SELECT sum(active), sum(NOT active) FROM system.projection_parts; + SELECT count() FROM system.dropped_tables_parts; + " } query "DROP TABLE IF EXISTS test_table" From 4c043301e6dde6b0c83394d6721e112c9c7bf4ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Aug 2024 10:30:31 +0200 Subject: [PATCH 0869/1722] Avoid ignoring errors of execute_process() (set COMMAND_ERROR_IS_FATAL=ANY) This will fix with issues like this [1]: Aug 12 09:58:44 '/usr/bin/cmake' '--build' '/build/build_docker/native' '--target' 'pre_compressor' Aug 12 09:58:44 sccache: error: Server startup failed: cache storage failed to read: Unexpected (temporary) at stat Aug 12 09:58:45 ninja: build stopped: subcommand failed. Aug 12 09:58:45 -- Configuring done (77.7s) Aug 12 09:58:47 -- Generating done (1.8s) Aug 12 09:58:47 -- Build files have been written to: /build/build_docker So as you can see even if ninja fails it still wrote build files, while it should fail. [1]: https://s3.amazonaws.com/clickhouse-test-reports/64955/0af41e32a5822d25ac3760f1ebb2313557474701/builds/report.html [2]: https://s3.amazonaws.com/clickhouse-builds/PRs/64955/0af41e32a5822d25ac3760f1ebb2313557474701/binary_darwin_aarch64/build_log.log Note, COMMAND_ERROR_IS_FATAL is 3.19+, and the requirement for now is 3.20 Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 12 ++++++-- PreLoad.cmake | 10 +++++-- cmake/freebsd/default_libs.cmake | 12 ++++++-- cmake/linux/default_libs.cmake | 6 +++- cmake/tools.cmake | 6 +++- cmake/utils.cmake | 5 +++- contrib/cctz-cmake/CMakeLists.txt | 4 ++- contrib/google-protobuf-cmake/CMakeLists.txt | 12 ++++++-- contrib/grpc-cmake/CMakeLists.txt | 30 +++++++++++++------ .../completions/CMakeLists.txt | 1 + 10 files changed, 75 insertions(+), 23 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7b4e0484ab1..8e2302e6c52 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -609,7 +609,9 @@ if (NATIVE_BUILD_TARGETS execute_process( COMMAND ${CMAKE_COMMAND} -E make_directory "${NATIVE_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} @@ -621,9 +623,13 @@ if (NATIVE_BUILD_TARGETS "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=${ENABLE_CLICKHOUSE_SELF_EXTRACTING}" ${PROJECT_SOURCE_DIR} WORKING_DIRECTORY "${NATIVE_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${NATIVE_BUILD_DIR}" --target ${NATIVE_BUILD_TARGETS} - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) endif () diff --git a/PreLoad.cmake b/PreLoad.cmake index e0fd37b2fd6..92b221c9f63 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -51,8 +51,14 @@ if (NOT "$ENV{CFLAGS}" STREQUAL "" endif() # Default toolchain - this is needed to avoid dependency on OS files. -execute_process(COMMAND uname -s OUTPUT_VARIABLE OS) -execute_process(COMMAND uname -m OUTPUT_VARIABLE ARCH) +execute_process(COMMAND uname -s + OUTPUT_VARIABLE OS + COMMAND_ERROR_IS_FATAL ANY +) +execute_process(COMMAND uname -m + OUTPUT_VARIABLE ARCH + COMMAND_ERROR_IS_FATAL ANY +) # By default, prefer clang on Linux # But note, that you still may change the compiler with -DCMAKE_C_COMPILER/-DCMAKE_CXX_COMPILER. diff --git a/cmake/freebsd/default_libs.cmake b/cmake/freebsd/default_libs.cmake index 6bde75f8c9a..3f5b3829877 100644 --- a/cmake/freebsd/default_libs.cmake +++ b/cmake/freebsd/default_libs.cmake @@ -9,10 +9,18 @@ endif () file(GLOB bprefix "/usr/local/llvm${COMPILER_VERSION_MAJOR}/lib/clang/${COMPILER_VERSION_MAJOR}/lib/${system_processor}-portbld-freebsd*/") message(STATUS "-Bprefix: ${bprefix}") -execute_process(COMMAND ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins-${system_processor}.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process(COMMAND + ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins-${system_processor}.a + OUTPUT_VARIABLE BUILTINS_LIBRARY + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE) # --print-file-name simply prints what you passed in case of nothing was resolved, so let's try one other possible option if (BUILTINS_LIBRARY STREQUAL "libclang_rt.builtins-${system_processor}.a") - execute_process(COMMAND ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) + execute_process(COMMAND + ${CMAKE_CXX_COMPILER} -Bprefix=${bprefix} --print-file-name=libclang_rt.builtins.a + OUTPUT_VARIABLE BUILTINS_LIBRARY + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE) endif() if (BUILTINS_LIBRARY STREQUAL "libclang_rt.builtins.a") message(FATAL_ERROR "libclang_rt.builtins had not been found") diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index 4a06243243e..51620bc9f33 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -5,7 +5,11 @@ set (DEFAULT_LIBS "-nodefaultlibs") # We need builtins from Clang's RT even without libcxx - for ubsan+int128. # See https://bugs.llvm.org/show_bug.cgi?id=16404 -execute_process (COMMAND ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process (COMMAND + ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt + OUTPUT_VARIABLE BUILTINS_LIBRARY + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE) # Apparently, in clang-19, the UBSan support library for C++ was moved out into ubsan_standalone_cxx.a, so we have to include both. if (SANITIZE STREQUAL undefined) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 7aa5d4c51ce..5c7da54b779 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -5,7 +5,11 @@ if (NOT CMAKE_CXX_COMPILER_ID MATCHES "Clang") endif () # Print details to output -execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version OUTPUT_VARIABLE COMPILER_SELF_IDENTIFICATION OUTPUT_STRIP_TRAILING_WHITESPACE) +execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version + OUTPUT_VARIABLE COMPILER_SELF_IDENTIFICATION + COMMAND_ERROR_IS_FATAL ANY + OUTPUT_STRIP_TRAILING_WHITESPACE +) message (STATUS "Using compiler:\n${COMPILER_SELF_IDENTIFICATION}") # Require minimum compiler versions diff --git a/cmake/utils.cmake b/cmake/utils.cmake index a318408098a..a99d8e050a8 100644 --- a/cmake/utils.cmake +++ b/cmake/utils.cmake @@ -90,7 +90,10 @@ endfunction() # Function get_cmake_properties returns list of all propreties that cmake supports function(get_cmake_properties outvar) - execute_process(COMMAND cmake --help-property-list OUTPUT_VARIABLE cmake_properties) + execute_process(COMMAND cmake --help-property-list + OUTPUT_VARIABLE cmake_properties + COMMAND_ERROR_IS_FATAL ANY + ) # Convert command output into a CMake list string(REGEX REPLACE ";" "\\\\;" cmake_properties "${cmake_properties}") string(REGEX REPLACE "\n" ";" cmake_properties "${cmake_properties}") diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7161f743de1..fadf948b053 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -37,7 +37,9 @@ message(STATUS "Packaging with tzdata version: ${TZDATA_VERSION}") execute_process(COMMAND bash -c "cd ${TZDIR} && find * -type f -and ! -name '*.tab' -and ! -name 'localtime' | LC_ALL=C sort | paste -sd ';' -" OUTPUT_STRIP_TRAILING_WHITESPACE - OUTPUT_VARIABLE TIMEZONES) + OUTPUT_VARIABLE TIMEZONES + COMMAND_ERROR_IS_FATAL ANY +) file(APPEND ${TIMEZONES_FILE} "// autogenerated by ClickHouse/contrib/cctz-cmake/CMakeLists.txt\n") file(APPEND ${TIMEZONES_FILE} "#include \n") diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index e44f737cfc3..f1a744f851f 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -359,7 +359,9 @@ else () execute_process( COMMAND mkdir -p ${PROTOC_BUILD_DIR} - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} @@ -375,11 +377,15 @@ else () "-DABSL_ENABLE_INSTALL=0" "${protobuf_source_dir}" WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) endif () add_executable(protoc IMPORTED GLOBAL) diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 1c0bf41ff78..975774d1990 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -51,8 +51,9 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME set(OPENSSL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/openssl-cmake") execute_process( - COMMAND mkdir -p ${OPENSSL_BUILD_DIR} - COMMAND_ECHO STDOUT + COMMAND mkdir -p ${OPENSSL_BUILD_DIR} + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY ) if (CMAKE_HOST_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") @@ -89,15 +90,21 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME "-DClickHouse_SOURCE_DIR=${ClickHouse_SOURCE_DIR}" "${OPENSSL_SOURCE_DIR}" WORKING_DIRECTORY "${OPENSSL_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${OPENSSL_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --install "${OPENSSL_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) # It's not important on which file we depend, we just want to specify right order add_library(openssl_for_grpc STATIC IMPORTED GLOBAL) @@ -108,8 +115,9 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME set (GRPC_CPP_PLUGIN_BUILD_DIR "${_gRPC_BINARY_DIR}/build") execute_process( - COMMAND mkdir -p ${GRPC_CPP_PLUGIN_BUILD_DIR} - COMMAND_ECHO STDOUT + COMMAND mkdir -p ${GRPC_CPP_PLUGIN_BUILD_DIR} + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY ) set(abseil_source_dir "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") @@ -140,11 +148,15 @@ if (NOT CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME "-DgRPC_SSL_PROVIDER=package" "${_gRPC_SOURCE_DIR}" WORKING_DIRECTORY "${GRPC_CPP_PLUGIN_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) execute_process( COMMAND ${CMAKE_COMMAND} --build "${GRPC_CPP_PLUGIN_BUILD_DIR}" - COMMAND_ECHO STDOUT) + COMMAND_ECHO STDOUT + COMMAND_ERROR_IS_FATAL ANY + ) add_executable(grpc_cpp_plugin IMPORTED GLOBAL) set_target_properties (grpc_cpp_plugin PROPERTIES IMPORTED_LOCATION "${GRPC_CPP_PLUGIN_BUILD_DIR}/grpc_cpp_plugin") diff --git a/programs/bash-completion/completions/CMakeLists.txt b/programs/bash-completion/completions/CMakeLists.txt index d364e07ef6e..2e911e81981 100644 --- a/programs/bash-completion/completions/CMakeLists.txt +++ b/programs/bash-completion/completions/CMakeLists.txt @@ -6,6 +6,7 @@ macro(configure_bash_completion) COMMAND ${PKG_CONFIG_BIN} --variable=completionsdir bash-completion OUTPUT_VARIABLE ${out} OUTPUT_STRIP_TRAILING_WHITESPACE + COMMAND_ERROR_IS_FATAL ANY ) endif() string(REPLACE /usr "${CMAKE_INSTALL_PREFIX}" out "${out}") From 7f005a6ca48d4f193470d3a71bc1d97ff55f4a2f Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Tue, 13 Aug 2024 08:38:30 +0000 Subject: [PATCH 0870/1722] Fix small value DateTime64 constant folding in nested query --- src/Analyzer/ConstantNode.cpp | 10 ++++- ...222_datetime64_small_value_const.reference | 18 +++++++++ .../03222_datetime64_small_value_const.sql | 39 +++++++++++++++++++ 3 files changed, 65 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03222_datetime64_small_value_const.reference create mode 100644 tests/queries/0_stateless/03222_datetime64_small_value_const.sql diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index c65090f5b55..3d0f448da4b 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -177,9 +177,15 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const * It could also lead to ambiguous parsing because we don't know if the string literal represents a date or a Decimal64 literal. * For this reason, we use a string literal representing a date instead of a Decimal64 literal. */ - if (WhichDataType(constant_value_type->getTypeId()).isDateTime64()) + if ((WhichDataType(constant_value_type->getTypeId()).isDateTime64()) || + (WhichDataType(constant_value_type->getTypeId()).isNullable() && WhichDataType((typeid_cast(constant_value_type.get()))->getNestedType()->getTypeId()).isDateTime64())) { - const auto * date_time_type = typeid_cast(constant_value_type.get()); + const DataTypeDateTime64 * date_time_type = nullptr; + if (WhichDataType(constant_value_type->getTypeId()).isNullable()) + date_time_type = typeid_cast((typeid_cast(constant_value_type.get()))->getNestedType().get()); + else + date_time_type = typeid_cast(constant_value_type.get()); + DecimalField decimal_value; if (constant_value_literal.tryGet>(decimal_value)) { diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.reference b/tests/queries/0_stateless/03222_datetime64_small_value_const.reference new file mode 100644 index 00000000000..ae36c08acc5 --- /dev/null +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.reference @@ -0,0 +1,18 @@ +0 1970-01-01 00:00:00.000 +0 1970-01-01 00:00:05.000 +0 1970-01-01 00:45:25.456789 +0 1970-01-01 00:53:25.456789123 +0 \N +1 1970-01-01 00:00:00.000 +5 1970-01-01 00:00:00.000 +2 1970-01-01 00:00:02.456 +3 1970-01-01 00:00:04.811 +4 1970-01-01 00:10:05.000 +4 1970-01-01 00:10:05.000 +1 1970-01-01 00:00:00.000 +2 1970-01-01 00:00:02.456 +3 1970-01-01 00:00:04.811 +5 1970-01-01 00:00:00.000 +0 +0 +5 diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql new file mode 100644 index 00000000000..6999ba9662a --- /dev/null +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -0,0 +1,39 @@ +-- Tags: shard + +select *, (select toDateTime64(0, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64(5, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64('1970-01-01 00:45:25.456789', 6)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64('1970-01-01 00:53:25.456789123', 9)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; +select *, (select toDateTime64(null,3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; + +create database if not exists shard_0; +create database if not exists shard_1; + +drop table if exists shard_0.dt64_03222; +drop table if exists shard_1.dt64_03222; +drop table if exists distr_03222_dt64; + +create table shard_0.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree order by id; +create table shard_1.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree order by id; +create table distr_03222_dt64 (id UInt64, dt DateTime64(3)) engine = Distributed(test_cluster_two_shards_different_databases, '', dt64_03222); + +insert into shard_0.dt64_03222 values(1, toDateTime64('1970-01-01 00:00:00.000',3)) +insert into shard_0.dt64_03222 values(2, toDateTime64('1970-01-01 00:00:02.456',3)); +insert into shard_1.dt64_03222 values(3, toDateTime64('1970-01-01 00:00:04.811',3)); +insert into shard_1.dt64_03222 values(4, toDateTime64('1970-01-01 00:10:05',3)); +insert into shard_1.dt64_03222 values(5, toDateTime64(0,3)); + +--Output : 1,5 2,3,4 4 1,2,3,5 0 0 5 +select id, dt from distr_03222_dt64 where dt = (select toDateTime64(0,3)) order by id; +select id, dt from distr_03222_dt64 where dt > (select toDateTime64(0,3)) order by id; +select id, dt from distr_03222_dt64 where dt > (select toDateTime64('1970-01-01 00:10:00.000',3)) order by id; +select id, dt from distr_03222_dt64 where dt < (select toDateTime64(5,3)) order by id; + +select count(*) from distr_03222_dt64 where dt > (select toDateTime64('2024-07-20 00:00:00',3)); +select count(*) from distr_03222_dt64 where dt > (select now()); +select count(*) from distr_03222_dt64 where dt < (select toDateTime64('2004-07-20 00:00:00',3)); + + +drop table if exists shard_0.dt64_03222; +drop table if exists shard_1.dt64_03222; +drop table if exists distr_03222_dt64; From 2896dd51d816e28ee73fd3f1acf20912e2689f35 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 09:52:43 +0000 Subject: [PATCH 0871/1722] Implemenet prepareForSquashing for ColumnObject --- src/Columns/ColumnDynamic.cpp | 19 +- src/Columns/ColumnDynamic.h | 2 + src/Columns/ColumnObject.cpp | 123 +++++++++- src/Columns/ColumnObject.h | 3 + src/Core/SettingsChangesHistory.cpp | 265 +--------------------- src/DataTypes/DataTypesBinaryEncoding.cpp | 10 + src/Interpreters/Squashing.cpp | 1 + 7 files changed, 151 insertions(+), 272 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index ee06a1b10fe..29813abb99e 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -660,11 +660,6 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) if (source_columns.empty()) return; - /// Internal variants of source dynamic columns may differ. - /// We want to preallocate memory for all variants we will have after squashing. - /// It may happen that the total number of variants in source columns will - /// exceed the limit, in this case we will choose the most frequent variants. - /// First, preallocate memory for variant discriminators and offsets. size_t new_size = size(); for (const auto & source_column : source_columns) @@ -673,7 +668,18 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) variant_col.getLocalDiscriminators().reserve_exact(new_size); variant_col.getOffsets().reserve_exact(new_size); - /// Second, collect all variants and their total sizes. + /// Second, preallocate memory for variants. + prepareVariantsForSquashing(source_columns); +} + +void ColumnDynamic::prepareVariantsForSquashing(const Columns & source_columns) +{ + /// Internal variants of source dynamic columns may differ. + /// We want to preallocate memory for all variants we will have after squashing. + /// It may happen that the total number of variants in source columns will + /// exceed the limit, in this case we will choose the most frequent variants. + + /// Collect all variants and their total sizes. std::unordered_map total_variant_sizes; DataTypes all_variants; @@ -747,6 +753,7 @@ void ColumnDynamic::prepareForSquashing(const Columns & source_columns) /// Now current dynamic column has all resulting variants and we can call /// prepareForSquashing on them to preallocate the memory. + auto & variant_col = getVariantColumn(); for (size_t i = 0; i != variant_info.variant_names.size(); ++i) { Columns source_variant_columns; diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index fe9815b1a81..2d695e1948b 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -259,6 +259,8 @@ public: } void prepareForSquashing(const Columns & source_columns) override; + /// Prepare only variants but not discriminators and offsets. + void prepareVariantsForSquashing(const Columns & source_columns); void ensureOwnership() override { diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index ea497451648..852327cfc17 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -257,6 +257,12 @@ ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(const std::string_view path return it_ptr->second; } +void ColumnObject::addNewDynamicPath(const std::string_view path) +{ + if (!tryToAddNewDynamicPath(path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add new dynamic path as the limit ({}) on dynamic paths is reached", max_dynamic_paths); +} + void ColumnObject::setDynamicPaths(const std::vector & paths) { if (paths.size() > max_dynamic_paths) @@ -1123,6 +1129,115 @@ void ColumnObject::getExtremes(DB::Field & min, DB::Field & max) const } } +void ColumnObject::prepareForSquashing(const std::vector & source_columns) +{ + if (source_columns.empty()) + return; + + /// Dynamic paths of source Object columns may differ. + /// We want to preallocate memory for all dynamic paths we will have after squashing. + /// It may happen that the total number of dynamic paths in source columns will + /// exceed the limit, in this case we will choose the most frequent paths. + std::unordered_map path_to_total_number_of_non_null_values; + + auto add_dynamic_paths = [&](const ColumnObject & source_object) + { + for (const auto & [path, dynamic_column_ptr] : source_object.dynamic_paths_ptrs) + { + auto it = path_to_total_number_of_non_null_values.find(path); + if (it == path_to_total_number_of_non_null_values.end()) + it = path_to_total_number_of_non_null_values.emplace(path, 0).first; + it->second += (dynamic_column_ptr->size() - dynamic_column_ptr->getNumberOfDefaultRows()); + } + }; + + for (const auto & source_column : source_columns) + add_dynamic_paths(assert_cast(*source_column)); + + /// Add dynamic paths from this object column. + add_dynamic_paths(*this); + + /// Check if the number of all dynamic paths exceeds the limit. + if (path_to_total_number_of_non_null_values.size() > max_dynamic_paths) + { + /// We want to keep the most frequent paths in the resulting object column. + /// Sort paths by total number of non null values. + /// Don't include paths from current column as we cannot change them. + std::vector> paths_with_sizes; + paths_with_sizes.reserve(path_to_total_number_of_non_null_values.size() - dynamic_paths.size()); + for (const auto & [path, size] : path_to_total_number_of_non_null_values) + { + if (!dynamic_paths.contains(path)) + paths_with_sizes.emplace_back(size, path); + } + std::sort(paths_with_sizes.begin(), paths_with_sizes.end(), std::greater()); + + /// Fill dynamic_paths with first paths in sorted list until we reach the limit. + size_t paths_to_add = max_dynamic_paths - dynamic_paths.size(); + for (size_t i = 0; i != paths_to_add; ++i) + addNewDynamicPath(paths_with_sizes[i].second); + } + /// Otherwise keep all paths. + else + { + /// Create columns for new dynamic paths. + for (const auto & [path, _] : path_to_total_number_of_non_null_values) + { + if (!dynamic_paths.contains(path)) + addNewDynamicPath(path); + } + } + + /// Now current object column has all resulting dynamic paths and we can call + /// prepareForSquashing on them to preallocate the memory. + /// Also we can preallocate memory for dynamic paths and shared data. + Columns shared_data_source_columns; + shared_data_source_columns.reserve(source_columns.size()); + std::unordered_map typed_paths_source_columns; + typed_paths_source_columns.reserve(typed_paths.size()); + std::unordered_map dynamic_paths_source_columns; + dynamic_paths_source_columns.reserve(dynamic_paths.size()); + + for (const auto & [path, column] : typed_paths) + typed_paths_source_columns[path].reserve(source_columns.size()); + + for (const auto & [path, column] : dynamic_paths) + dynamic_paths_source_columns[path].reserve(source_columns.size()); + + size_t total_size = 0; + for (const auto & source_column : source_columns) + { + const auto & source_object_column = assert_cast(*source_column); + total_size += source_object_column.size(); + shared_data_source_columns.push_back(source_object_column.shared_data); + + for (const auto & [path, column] : source_object_column.typed_paths) + typed_paths_source_columns.at(path).push_back(column); + + for (const auto & [path, column] : source_object_column.dynamic_paths) + { + if (dynamic_paths.contains(path)) + dynamic_paths_source_columns.at(path).push_back(column); + } + } + + shared_data->prepareForSquashing(shared_data_source_columns); + + for (const auto & [path, source_typed_columns] : typed_paths_source_columns) + typed_paths[path]->prepareForSquashing(source_typed_columns); + + for (const auto & [path, source_dynamic_columns] : dynamic_paths_source_columns) + { + /// ColumnDynamic::prepareForSquashing may not preallocate enough memory for discriminators and offsets + /// because source columns may not have this dynamic path (and so dynamic columns filled with nulls). + /// For this reason we first call ColumnDynamic::reserve with resulting size to preallocate memory for + /// discriminators and offsets and ColumnDynamic::prepareVariantsForSquashing to preallocate memory + /// for all variants inside Dynamic. + dynamic_paths_ptrs[path]->reserve(total_size); + dynamic_paths_ptrs[path]->prepareVariantsForSquashing(source_dynamic_columns); + } +} + void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & source_columns) { if (!empty()) @@ -1147,14 +1262,14 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou const auto & source_object = assert_cast(*source_column); /// During deserialization from MergeTree we will have statistics from the whole /// data part with number of non null values for each dynamic path. - const auto & source_statistics = source_object.getStatistics(); - for (const auto & [path, column] : source_object.dynamic_paths) + const auto & source_statistics = source_object.getStatistics(); + for (const auto & [path, column_ptr] : source_object.dynamic_paths_ptrs) { auto it = path_to_total_number_of_non_null_values.find(path); if (it == path_to_total_number_of_non_null_values.end()) it = path_to_total_number_of_non_null_values.emplace(path, 0).first; auto statistics_it = source_statistics.data.find(path); - size_t size = statistics_it == source_statistics.data.end() ? (column->size() - column->getNumberOfDefaultRows()) : statistics_it->second; + size_t size = statistics_it == source_statistics.data.end() ? (column_ptr->size() - column_ptr->getNumberOfDefaultRows()) : statistics_it->second; it->second += size; } } @@ -1165,7 +1280,7 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou /// Check if the number of all dynamic paths exceeds the limit. if (path_to_total_number_of_non_null_values.size() > max_dynamic_paths) { - /// Sort paths by total_number_of_non_null_values. + /// Sort paths by total number of non null values. std::vector> paths_with_sizes; paths_with_sizes.reserve(path_to_total_number_of_non_null_values.size()); for (const auto & [path, size] : path_to_total_number_of_non_null_values) diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index ecb6c4e0e15..38c71e94e12 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -141,6 +141,7 @@ public: void reserve(size_t n) override; size_t capacity() const override; + void prepareForSquashing(const std::vector & source_columns) override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; @@ -202,6 +203,8 @@ public: /// Try to add new dynamic path. Returns pointer to the new dynamic /// path column or nullptr if limit on dynamic paths is reached. ColumnDynamic * tryToAddNewDynamicPath(const std::string_view path); + /// Throws an exception if cannot add. + void addNewDynamicPath(const std::string_view path); void setDynamicPaths(const std::vector & paths); void setStatistics(const Statistics & statistics_) { statistics = statistics_; } diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 50d46429803..bcf9b3b835c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,268 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"allow_experimental_json_type", false, false, "Add new experimental JSON type"}, - {"use_json_alias_for_old_object_type", true, false, "Use JSON type alias to create new JSON type"}, - {"type_json_skip_duplicated_paths", false, false, "Allow to skip duplicated paths during JSON parsing"} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -347,6 +85,9 @@ static std::initializer_list(*type); + /// Write version of the serialization because we can add new arguments in the JSON type. + writeBinary(TYPE_JSON_SERIALIZATION_VERSION, buf); writeVarUInt(object_type.getMaxDynamicPaths(), buf); writeBinary(UInt8(object_type.getMaxDynamicTypes()), buf); const auto & typed_paths = object_type.getTypedPaths(); @@ -726,6 +732,10 @@ DataTypePtr decodeDataType(ReadBuffer & buf) } case BinaryTypeIndex::JSON: { + UInt8 serialization_version; + readBinary(serialization_version, buf); + if (serialization_version > TYPE_JSON_SERIALIZATION_VERSION) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected version of JSON type binary encoding"); size_t max_dynamic_paths; readVarUInt(max_dynamic_paths, buf); UInt8 max_dynamic_types; diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 95b76c60063..ad7a5a4c863 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -146,6 +146,7 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl { /// We know all the data we will insert in advance and can make all necessary pre-allocations. mutable_columns[i]->prepareForSquashing(source_columns_list[i]); +// mutable_columns[i]->reserve(rows); for (auto & source_column : source_columns_list[i]) { auto column = std::move(source_column); From 5812dbcf2e4a9eada33a611ea7b63172f6ed0905 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 13 Aug 2024 11:53:33 +0200 Subject: [PATCH 0872/1722] Update 03210_dynamic_squashing.sql --- tests/queries/0_stateless/03210_dynamic_squashing.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql index da3b911e796..d9ebc28fc43 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.sql +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -1,3 +1,5 @@ +-- Tags: long + set allow_experimental_dynamic_type = 1; set max_block_size = 1000; From e46c5a75ef6b5488834add56bea4cab327515bfb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 13 Aug 2024 17:59:05 +0800 Subject: [PATCH 0873/1722] fix building issue --- src/Functions/{FunctionOverlay.cpp => overlay.cpp} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename src/Functions/{FunctionOverlay.cpp => overlay.cpp} (99%) diff --git a/src/Functions/FunctionOverlay.cpp b/src/Functions/overlay.cpp similarity index 99% rename from src/Functions/FunctionOverlay.cpp rename to src/Functions/overlay.cpp index 61d2df88ab1..094da27a71d 100644 --- a/src/Functions/FunctionOverlay.cpp +++ b/src/Functions/overlay.cpp @@ -728,8 +728,8 @@ private: REGISTER_FUNCTION(Overlay) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseSensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Sensitive); } } From 534ec1fa01b9de4206a8d6e3f91d73d7d1b2f743 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 10:12:45 +0000 Subject: [PATCH 0874/1722] Correct compile error --- src/TableFunctions/TableFunctionObjectStorage.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 6235108e2ba..0ddf901ee21 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -14,10 +14,11 @@ #include #include -#include -#include -#include #include +#include +#include +#include +#include namespace DB From 85bd63a2ac54c8665e99c1b07c4a5e0189212635 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 10 May 2024 12:18:06 +0800 Subject: [PATCH 0875/1722] rebase and resolve conflict --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/HashJoin/AddedColumns.cpp | 157 ++++++++++++------ src/Interpreters/HashJoin/AddedColumns.h | 28 +++- src/Interpreters/HashJoin/HashJoin.cpp | 4 +- src/Interpreters/HashJoin/HashJoin.h | 12 ++ src/Interpreters/HashJoin/HashJoinMethods.h | 3 +- .../HashJoin/HashJoinMethodsImpl.h | 49 +++--- src/Interpreters/HashJoin/KnowRowsHolder.h | 9 +- src/Interpreters/RowRefs.cpp | 6 +- src/Interpreters/RowRefs.h | 7 +- src/Interpreters/TableJoin.cpp | 1 + src/Interpreters/TableJoin.h | 2 + tests/performance/all_join_opt.xml | 15 ++ 14 files changed, 206 insertions(+), 89 deletions(-) create mode 100644 tests/performance/all_join_opt.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6f24db57026..e2740026e58 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -325,6 +325,7 @@ class IColumn; \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ \ + M(Int32, join_output_by_rowlist_perkey_rows_threshold, 5, "The lower limit of per-key average rows in the right table to determine whether to output by row list in hash join.", 0) \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ M(Bool, single_join_prefer_left_table, true, "For single JOIN in case of identifier ambiguity prefer left table", IMPORTANT) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 511723f1873..cc9524bea2e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,6 +85,7 @@ static std::initializer_list void AddedColumns::buildOutput() -{ -} +template<> +void AddedColumns::buildOutput() {} + +template<> +void AddedColumns::buildJoinGetOutput() {} + +template<> +template +void AddedColumns::buildOutputFromBlocks() {} template<> void AddedColumns::buildOutput() { - for (size_t i = 0; i < this->size(); ++i) + if (!output_by_row_list) + buildOutputFromBlocks(); + else { - auto& col = columns[i]; - size_t default_count = 0; - auto apply_default = [&]() + if (join_data_avg_perkey_rows < output_by_row_list_threshold) + buildOutputFromBlocks(); + else { - if (default_count > 0) + for (size_t i = 0; i < this->size(); ++i) { - JoinCommon::addDefaultValues(*col, type_name[i].type, default_count); - default_count = 0; - } - }; - - for (size_t j = 0; j < lazy_output.blocks.size(); ++j) - { - if (!lazy_output.blocks[j]) - { - default_count++; - continue; - } - apply_default(); - const auto & column_from_block = reinterpret_cast(lazy_output.blocks[j])->getByPosition(right_indexes[i]); - /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. - if (is_join_get) - { - if (auto * nullable_col = typeid_cast(col.get()); - nullable_col && !column_from_block.column->isNullable()) + auto & col = columns[i]; + for (auto row_ref_i : lazy_output.row_refs) { - nullable_col->insertFromNotNullable(*column_from_block.column, lazy_output.row_nums[j]); - continue; + if (row_ref_i) + { + const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); + for (auto it = row_ref_list->begin(); it.ok(); ++it) + col->insertFrom(*it->block->getByPosition(right_indexes[i]).column, it->row_num); + } + else + type_name[i].type->insertDefaultInto(*col); } } - col->insertFrom(*column_from_block.column, lazy_output.row_nums[j]); } - apply_default(); + } +} + +template<> +void AddedColumns::buildJoinGetOutput() +{ + for (size_t i = 0; i < this->size(); ++i) + { + auto & col = columns[i]; + for (auto row_ref_i : lazy_output.row_refs) + { + if (!row_ref_i) + { + type_name[i].type->insertDefaultInto(*col); + continue; + } + const auto * row_ref = reinterpret_cast(row_ref_i); + const auto & column_from_block = row_ref->block->getByPosition(right_indexes[i]); + if (auto * nullable_col = typeid_cast(col.get()); nullable_col && !column_from_block.column->isNullable()) + nullable_col->insertFromNotNullable(*column_from_block.column, row_ref->row_num); + else + col->insertFrom(*column_from_block.column, row_ref->row_num); + } + } +} + +template<> +template +void AddedColumns::buildOutputFromBlocks() +{ + if (this->size() == 0) + return; + std::vector blocks; + std::vector row_nums; + blocks.reserve(lazy_output.row_refs.size()); + row_nums.reserve(lazy_output.row_refs.size()); + for (auto row_ref_i : lazy_output.row_refs) + { + if (row_ref_i) + { + if constexpr (from_row_list) + { + const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); + for (auto it = row_ref_list->begin(); it.ok(); ++it) + { + blocks.emplace_back(it->block); + row_nums.emplace_back(it->row_num); + } + } + else + { + const RowRef * row_ref = reinterpret_cast(row_ref_i); + blocks.emplace_back(row_ref->block); + row_nums.emplace_back(row_ref->row_num); + } + } + else + { + blocks.emplace_back(nullptr); + row_nums.emplace_back(0); + } + } + for (size_t i = 0; i < this->size(); ++i) + { + auto & col = columns[i]; + for (size_t j = 0; j < blocks.size(); ++j) + { + if (blocks[j]) + col->insertFrom(*blocks[j]->getByPosition(right_indexes[i]).column, row_nums[j]); + else + type_name[i].type->insertDefaultInto(*col); + } } } @@ -72,29 +139,27 @@ void AddedColumns::applyLazyDefaults() } template<> -void AddedColumns::applyLazyDefaults() -{ -} +void AddedColumns::applyLazyDefaults() {} template <> -void AddedColumns::appendFromBlock(const Block & block, size_t row_num,const bool has_defaults) +void AddedColumns::appendFromBlock(const RowRef * row_ref, const bool has_defaults) { if (has_defaults) applyLazyDefaults(); #ifndef NDEBUG - checkBlock(block); + checkBlock(*row_ref->block); #endif if (is_join_get) { size_t right_indexes_size = right_indexes.size(); for (size_t j = 0; j < right_indexes_size; ++j) { - const auto & column_from_block = block.getByPosition(right_indexes[j]); + const auto & column_from_block = row_ref->block->getByPosition(right_indexes[j]); if (auto * nullable_col = nullable_column_ptrs[j]) - nullable_col->insertFromNotNullable(*column_from_block.column, row_num); + nullable_col->insertFromNotNullable(*column_from_block.column, row_ref->row_num); else - columns[j]->insertFrom(*column_from_block.column, row_num); + columns[j]->insertFrom(*column_from_block.column, row_ref->row_num); } } else @@ -102,22 +167,21 @@ void AddedColumns::appendFromBlock(const Block & block, size_t row_num,co size_t right_indexes_size = right_indexes.size(); for (size_t j = 0; j < right_indexes_size; ++j) { - const auto & column_from_block = block.getByPosition(right_indexes[j]); - columns[j]->insertFrom(*column_from_block.column, row_num); + const auto & column_from_block = row_ref->block->getByPosition(right_indexes[j]); + columns[j]->insertFrom(*column_from_block.column, row_ref->row_num); } } } template <> -void AddedColumns::appendFromBlock(const Block & block, size_t row_num, bool) +void AddedColumns::appendFromBlock(const RowRef * row_ref, bool) { #ifndef NDEBUG - checkBlock(block); + checkBlock(*row_ref->block); #endif if (has_columns_to_add) { - lazy_output.blocks.emplace_back(reinterpret_cast(&block)); - lazy_output.row_nums.emplace_back(static_cast(row_num)); + lazy_output.row_refs.emplace_back(reinterpret_cast(row_ref)); } } template<> @@ -131,8 +195,7 @@ void AddedColumns::appendDefaultRow() { if (has_columns_to_add) { - lazy_output.blocks.emplace_back(0); - lazy_output.row_nums.emplace_back(0); + lazy_output.row_refs.emplace_back(0); } } } diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 13a7df6f498..f1b95a63be6 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -50,8 +50,7 @@ public: struct LazyOutput { - PaddedPODArray blocks; - PaddedPODArray row_nums; + PaddedPODArray row_refs; }; AddedColumns( @@ -76,8 +75,7 @@ public: if constexpr (lazy) { has_columns_to_add = num_columns_to_add > 0; - lazy_output.blocks.reserve(rows_to_add); - lazy_output.row_nums.reserve(rows_to_add); + lazy_output.row_refs.reserve(rows_to_add); } columns.reserve(num_columns_to_add); @@ -115,18 +113,22 @@ public: if (columns[j]->isNullable() && !saved_column->isNullable()) nullable_column_ptrs[j] = typeid_cast(columns[j].get()); } + join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows(); + output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold(); } size_t size() const { return columns.size(); } void buildOutput(); + void buildJoinGetOutput(); + ColumnWithTypeAndName moveColumn(size_t i) { return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); } - void appendFromBlock(const Block & block, size_t row_num, bool has_default); + void appendFromBlock(const RowRef * row_ref, bool has_default); void appendDefaultRow(); @@ -134,6 +136,8 @@ public: const IColumn & leftAsofKey() const { return *left_asof_key; } + static constexpr bool isLazy() { return lazy; } + Block left_block; std::vector join_on_keys; ExpressionActionsPtr additional_filter_expression; @@ -142,6 +146,9 @@ public: size_t rows_to_add; std::unique_ptr offsets_to_replicate; bool need_filter = false; + bool output_by_row_list = false; + size_t join_data_avg_perkey_rows = 0; + size_t output_by_row_list_threshold = 0; IColumn::Filter filter; void reserve(bool need_replicate) @@ -212,15 +219,22 @@ private: columns.back()->reserve(src_column.column->size()); type_name.emplace_back(src_column.type, src_column.name, qualified_name); } + + /** Build output from the blocks that extract from `RowRef` or `RowRefList`, to avoid block cache miss which may cause performance slow down. + * And This problem would happen it we directly build output from `RowRef` or `RowRefList`. + */ + template + void buildOutputFromBlocks(); }; /// Adapter class to pass into addFoundRowAll /// In joinRightColumnsWithAdditionalFilter we don't want to add rows directly into AddedColumns, /// because they need to be filtered by additional_filter_expression. -class PreSelectedRows : public std::vector +class PreSelectedRows : public std::vector { public: - void appendFromBlock(const Block & block, size_t row_num, bool /* has_default */) { this->emplace_back(&block, row_num); } + void appendFromBlock(const RowRef * row_ref, bool /* has_default */) { this->emplace_back(row_ref); } + static constexpr bool isLazy() { return false; } }; } diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index dd7d42de63e..9c07a71e614 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -495,7 +495,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) } size_t rows = source_block.rows(); - + data->rows_to_join += rows; const auto & right_key_names = table_join->getAllNames(JoinTableSide::Right); ColumnPtrMap all_key_columns(right_key_names.size()); for (const auto & column_name : right_key_names) @@ -647,7 +647,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) total_bytes = getTotalByteCount(); } } - + data->keys_to_join = total_rows; shrinkStoredBlocksToFit(total_bytes); return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 00f5ef6d214..d645b8e9273 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -345,6 +345,18 @@ public: size_t blocks_allocated_size = 0; size_t blocks_nullmaps_allocated_size = 0; + + /// Number of rows of right table to join + size_t rows_to_join = 0; + /// Number of keys of right table to join + size_t keys_to_join = 0; + + size_t avgPerKeyRows() const + { + if (keys_to_join == 0) + return 0; + return rows_to_join / keys_to_join; + } }; using RightTableDataPtr = std::shared_ptr; diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 3b7a67467e3..97ad57d26ea 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -83,6 +83,7 @@ public: const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get = false); + private: template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); @@ -128,7 +129,7 @@ private: template static ColumnPtr buildAdditionalFilter( size_t left_start_row, - const std::vector & selected_rows, + const std::vector & selected_rows, const std::vector & row_replicate_offset, AddedColumns & added_columns); diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index aedd24630d1..0d90bad2d8a 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -95,7 +95,10 @@ Block HashJoinMethods::joinBlockImpl( added_columns.join_on_keys.clear(); Block remaining_block = sliceBlock(block, num_joined); - added_columns.buildOutput(); + if (is_join_get) + added_columns.buildJoinGetOutput(); + else + added_columns.buildOutput(); for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); @@ -339,6 +342,8 @@ size_t HashJoinMethods::joinRightColumns( size_t rows = added_columns.rows_to_add; if constexpr (need_filter) added_columns.filter = IColumn::Filter(rows, 0); + if constexpr (!flag_per_row && (STRICTNESS == JoinStrictness::All || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Right))) + added_columns.output_by_row_list = true; Arena pool; @@ -381,15 +386,15 @@ size_t HashJoinMethods::joinRightColumns( const IColumn & left_asof_key = added_columns.leftAsofKey(); auto row_ref = mapped->findAsof(left_asof_key, i); - if (row_ref.block) + if (row_ref && row_ref->block) { setUsed(added_columns.filter, i); if constexpr (flag_per_row) - used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); + used_flags.template setUsed(row_ref->block, row_ref->row_num, 0); else used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); + added_columns.appendFromBlock(row_ref, join_features.add_missing); } else addNotFoundRow(added_columns, current_offset); @@ -420,7 +425,7 @@ size_t HashJoinMethods::joinRightColumns( if (used_once) { setUsed(added_columns.filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); + added_columns.appendFromBlock(&mapped, join_features.add_missing); } break; @@ -438,7 +443,7 @@ size_t HashJoinMethods::joinRightColumns( { setUsed(added_columns.filter, i); used_flags.template setUsed(find_result); - added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); + added_columns.appendFromBlock(&mapped, join_features.add_missing); if (join_features.is_any_or_semi_join) { @@ -477,7 +482,7 @@ template template ColumnPtr HashJoinMethods::buildAdditionalFilter( size_t left_start_row, - const std::vector & selected_rows, + const std::vector & selected_rows, const std::vector & row_replicate_offset, AddedColumns & added_columns) { @@ -489,7 +494,7 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter result_column = ColumnUInt8::create(); break; } - const Block & sample_right_block = *selected_rows.begin()->block; + const Block & sample_right_block = *((*selected_rows.begin())->block); if (!sample_right_block || !added_columns.additional_filter_expression) { auto filter = ColumnUInt8::create(); @@ -519,8 +524,8 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter auto new_col = col.column->cloneEmpty(); for (const auto & selected_row : selected_rows) { - const auto & src_col = selected_row.block->getByPosition(right_col_pos); - new_col->insertFrom(*src_col.column, selected_row.row_num); + const auto & src_col = selected_row->block->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row->row_num); } executed_block.insert({std::move(new_col), col.type, col.name}); } @@ -700,26 +705,24 @@ size_t HashJoinMethods::joinRightColumnsWithAddt { // For inner join, we need mark each right row'flag, because we only use each right row once. auto used_once = used_flags.template setUsedOnce( - selected_right_row_it->block, selected_right_row_it->row_num, 0); + (*selected_right_row_it)->block, (*selected_right_row_it)->row_num, 0); if (used_once) { any_matched = true; total_added_rows += 1; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + added_columns.appendFromBlock(*selected_right_row_it, join_features.add_missing); } } } else { auto used_once = used_flags.template setUsedOnce( - selected_right_row_it->block, selected_right_row_it->row_num, 0); + (*selected_right_row_it)->block, (*selected_right_row_it)->row_num, 0); if (used_once) { any_matched = true; total_added_rows += 1; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + added_columns.appendFromBlock(*selected_right_row_it, join_features.add_missing); } } } @@ -727,16 +730,14 @@ size_t HashJoinMethods::joinRightColumnsWithAddt { any_matched = true; if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + used_flags.template setUsed((*selected_right_row_it)->block, (*selected_right_row_it)->row_num, 0); } else { any_matched = true; total_added_rows += 1; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); - used_flags.template setUsed( - selected_right_row_it->block, selected_right_row_it->row_num, 0); + added_columns.appendFromBlock(*selected_right_row_it, join_features.add_missing); + used_flags.template setUsed((*selected_right_row_it)->block, (*selected_right_row_it)->row_num, 0); } } @@ -756,8 +757,7 @@ size_t HashJoinMethods::joinRightColumnsWithAddt if (filter_flags[replicated_row]) { any_matched = true; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + added_columns.appendFromBlock(*selected_right_row_it, join_features.add_missing); total_added_rows += 1; } ++selected_right_row_it; @@ -767,8 +767,7 @@ size_t HashJoinMethods::joinRightColumnsWithAddt if (filter_flags[replicated_row]) { any_matched = true; - added_columns.appendFromBlock( - *selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing); + added_columns.appendFromBlock(*selected_right_row_it, join_features.add_missing); total_added_rows += 1; selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row; break; diff --git a/src/Interpreters/HashJoin/KnowRowsHolder.h b/src/Interpreters/HashJoin/KnowRowsHolder.h index d51c96893c5..9223e98d13c 100644 --- a/src/Interpreters/HashJoin/KnowRowsHolder.h +++ b/src/Interpreters/HashJoin/KnowRowsHolder.h @@ -104,7 +104,7 @@ void addFoundRowAll( { if (!known_rows.isKnown(std::make_pair(it->block, it->row_num))) { - added.appendFromBlock(*it->block, it->row_num, false); + added.appendFromBlock(*it, false); ++current_offset; if (!new_known_rows_ptr) { @@ -124,11 +124,16 @@ void addFoundRowAll( known_rows.add(std::cbegin(*new_known_rows_ptr), std::cend(*new_known_rows_ptr)); } } + else if constexpr (AddedColumns::isLazy()) + { + added.appendFromBlock(&mapped, false); + current_offset += mapped.rows; + } else { for (auto it = mapped.begin(); it.ok(); ++it) { - added.appendFromBlock(*it->block, it->row_num, false); + added.appendFromBlock(*it, false); ++current_offset; } } diff --git a/src/Interpreters/RowRefs.cpp b/src/Interpreters/RowRefs.cpp index 9785ba46dab..1b397ab56ef 100644 --- a/src/Interpreters/RowRefs.cpp +++ b/src/Interpreters/RowRefs.cpp @@ -144,7 +144,7 @@ public: return low; } - RowRef findAsof(const IColumn & asof_column, size_t row_num) override + RowRef * findAsof(const IColumn & asof_column, size_t row_num) override { sort(); @@ -156,10 +156,10 @@ public: if (pos != entries.size()) { size_t row_ref_index = entries[pos].row_ref_index; - return row_refs[row_ref_index]; + return &row_refs[row_ref_index]; } - return {nullptr, 0}; + return nullptr; } private: diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 650b2311ba7..7c98c47dd11 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -122,7 +122,7 @@ struct RowRefList : RowRef }; RowRefList() {} /// NOLINT - RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {} + RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {} ForwardIterator begin() const { return ForwardIterator(this); } @@ -135,8 +135,11 @@ struct RowRefList : RowRef *next = Batch(nullptr); } next = next->insert(std::move(row_ref), pool); + ++rows; } +public: + SizeT rows = 0; private: Batch * next = nullptr; }; @@ -158,7 +161,7 @@ struct SortedLookupVectorBase virtual void insert(const IColumn &, const Block *, size_t) = 0; // This needs to be synchronized internally - virtual RowRef findAsof(const IColumn &, size_t) = 0; + virtual RowRef * findAsof(const IColumn &, size_t) = 0; }; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index c8c926db13c..138085f0710 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -115,6 +115,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) + , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 3f2bebb5816..4d626084d81 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -148,6 +148,7 @@ private: const size_t partial_merge_join_left_table_buffer_bytes = 0; const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; + const size_t output_by_rowlist_perkey_rows_threshold = 0; /// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified. size_t max_memory_usage = 0; @@ -295,6 +296,7 @@ public: return join_use_nulls && isRightOrFull(kind()); } + size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml new file mode 100644 index 00000000000..0ab9c39f67c --- /dev/null +++ b/tests/performance/all_join_opt.xml @@ -0,0 +1,15 @@ + + CREATE TABLE test (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a + CREATE TABLE test1 (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a + + INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) + INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) + + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b + + DROP TABLE IF EXISTS test + DROP TABLE IF EXISTS test1 + \ No newline at end of file From d2be1bf693045bebec341a850685b377ee3d88a9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 13 Aug 2024 12:33:44 +0000 Subject: [PATCH 0876/1722] Fix FullSortingJoinTest.AsofGreaterGeneratedTestData with empty data --- src/Processors/tests/gtest_full_sorting_join.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Processors/tests/gtest_full_sorting_join.cpp b/src/Processors/tests/gtest_full_sorting_join.cpp index f678d7984e8..befe5e28b5d 100644 --- a/src/Processors/tests/gtest_full_sorting_join.cpp +++ b/src/Processors/tests/gtest_full_sorting_join.cpp @@ -208,6 +208,12 @@ Block executePipeline(QueryPipeline && pipeline) template void assertColumnVectorEq(const typename ColumnVector::Container & expected, const Block & block, const std::string & name) { + if (expected.empty()) + { + ASSERT_TRUE(block.columns() == 0); + return; + } + const auto * actual = typeid_cast *>(block.getByName(name).column.get()); ASSERT_TRUE(actual) << "unexpected column type: " << block.getByName(name).column->dumpStructure() << "expected: " << typeid(ColumnVector).name(); @@ -230,6 +236,12 @@ void assertColumnVectorEq(const typename ColumnVector::Container & expected, template void assertColumnEq(const IColumn & expected, const Block & block, const std::string & name) { + if (expected.empty()) + { + ASSERT_TRUE(block.columns() == 0); + return; + } + const ColumnPtr & actual = block.getByName(name).column; ASSERT_TRUE(checkColumn(*actual)); ASSERT_TRUE(checkColumn(expected)); From 52dea79a906ecc3d9a19599612b1c2c7708876b6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 13 Aug 2024 15:20:37 +0200 Subject: [PATCH 0877/1722] Update 03210_dynamic_squashing.sql --- tests/queries/0_stateless/03210_dynamic_squashing.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03210_dynamic_squashing.sql b/tests/queries/0_stateless/03210_dynamic_squashing.sql index d9ebc28fc43..71d09263fda 100644 --- a/tests/queries/0_stateless/03210_dynamic_squashing.sql +++ b/tests/queries/0_stateless/03210_dynamic_squashing.sql @@ -6,18 +6,18 @@ set max_block_size = 1000; drop table if exists test; create table test (d Dynamic) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(100000); select '1'; select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; drop table test; create table test (d Dynamic(max_types=1)) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +insert into test select multiIf(number < 1000, NULL::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(100000); select '2'; select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; truncate table test; -insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(1000000); +insert into test select multiIf(number < 1000, 'Str'::Dynamic(max_types=1), number < 3000, range(number % 5)::Dynamic(max_types=1), number::Dynamic(max_types=1)) from numbers(100000); select '3'; select distinct dynamicType(d) as type, isDynamicElementInSharedData(d) as flag from test order by type; From fbe08cc24ca3f1b4472eba0960b14227917c0329 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 13 Aug 2024 07:24:41 -0600 Subject: [PATCH 0878/1722] Add no-parallel flag to 03221_create_if_not_exists_setting.sql --- tests/queries/0_stateless/03221_create_if_not_exists_setting.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sql b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sql index 59535981e7a..18b3ed7bcec 100644 --- a/tests/queries/0_stateless/03221_create_if_not_exists_setting.sql +++ b/tests/queries/0_stateless/03221_create_if_not_exists_setting.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel SET create_if_not_exists=0; -- Default From 973b2405794cebeabf9497e3b10ed6180130b891 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Aug 2024 15:35:14 +0200 Subject: [PATCH 0879/1722] Fix min/max time columns --- src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp | 7 +++++-- .../0_stateless/03222_create_timeseries_table.reference | 0 .../queries/0_stateless/03222_create_timeseries_table.sql | 7 +++++++ 3 files changed, 12 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03222_create_timeseries_table.reference create mode 100644 tests/queries/0_stateless/03222_create_timeseries_table.sql diff --git a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp index f9e7290e514..746a6a28274 100644 --- a/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp +++ b/src/Storages/TimeSeries/TimeSeriesDefinitionNormalizer.cpp @@ -227,8 +227,11 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create) /// We use Nullable(DateTime64(3)) as the default type of the `min_time` and `max_time` columns. /// It's nullable because it allows the aggregation (see aggregate_min_time_and_max_time) work correctly even /// for rows in the "tags" table which doesn't have `min_time` and `max_time` (because they have no matching rows in the "data" table). - make_new_column(TimeSeriesColumnNames::MinTime, make_nullable(timestamp_type)); - make_new_column(TimeSeriesColumnNames::MaxTime, make_nullable(timestamp_type)); + + if (!is_next_column_named(TimeSeriesColumnNames::MinTime)) + make_new_column(TimeSeriesColumnNames::MinTime, make_nullable(timestamp_type)); + if (!is_next_column_named(TimeSeriesColumnNames::MaxTime)) + make_new_column(TimeSeriesColumnNames::MaxTime, make_nullable(timestamp_type)); } /// Add missing columns for the "metrics" table. diff --git a/tests/queries/0_stateless/03222_create_timeseries_table.reference b/tests/queries/0_stateless/03222_create_timeseries_table.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03222_create_timeseries_table.sql b/tests/queries/0_stateless/03222_create_timeseries_table.sql new file mode 100644 index 00000000000..bdb29e7d366 --- /dev/null +++ b/tests/queries/0_stateless/03222_create_timeseries_table.sql @@ -0,0 +1,7 @@ +SET allow_experimental_time_series_table = 1; + +CREATE TABLE 03222_timeseries_table1 ENGINE = TimeSeries FORMAT Null; +CREATE TABLE 03222_timeseries_table2 ENGINE = TimeSeries SETTINGS store_min_time_and_max_time = 1, aggregate_min_time_and_max_time = 1 FORMAT Null; +--- This doesn't work because allow_nullable_key cannot be set in query for the internal MergeTree tables +--- CREATE TABLE 03222_timeseries_table3 ENGINE = TimeSeries SETTINGS store_min_time_and_max_time = 1, aggregate_min_time_and_max_time = 0; +CREATE TABLE 03222_timeseries_table4 ENGINE = TimeSeries SETTINGS store_min_time_and_max_time = 0 FORMAT Null; From 5da5bea8dfb768d3f6fd42f081a3d82e1c782e64 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 13:47:20 +0000 Subject: [PATCH 0880/1722] Reduce flakiness of a test --- .../0_stateless/00652_mergetree_mutations.sh | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index a9d7908a1af..3b0966dd2c3 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -70,7 +70,23 @@ sleep 1 ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner(x) VALUES (4)" sleep 0.1 +for i in {1..10} +do + + if [ $(${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner'") -eq 2 ]; then + break + fi + + if [[ $i -eq 100 ]]; then + echo "Timed out while waiting for outdated mutation record to be deleted!" + fi + + sleep 1 + ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner(x) VALUES (4)" +done + # Check that the first mutation is cleaned ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner" + From 16fd24fb1f82f109e3ac34665941c2acea7bf697 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 13 Aug 2024 13:51:24 +0000 Subject: [PATCH 0881/1722] Update fuzzer dictionary as per commit 99282e526a8aeb175e9f3e69fc9385070d03798a Also update README so that we have consistent behavior of sort across macOS and Linux. --- tests/fuzz/README.md | 4 +- tests/fuzz/all.dict | 926 ++-- tests/fuzz/dictionaries/datatypes.dict | 4416 ++++++++++++++++- tests/fuzz/dictionaries/functions.dict | 110 + .../{key_words.dict => keywords.dict} | 25 +- 5 files changed, 4895 insertions(+), 586 deletions(-) rename tests/fuzz/dictionaries/{key_words.dict => keywords.dict} (95%) diff --git a/tests/fuzz/README.md b/tests/fuzz/README.md index 6b5b161b2d5..576ad66ed93 100644 --- a/tests/fuzz/README.md +++ b/tests/fuzz/README.md @@ -13,11 +13,11 @@ The list of datatypes generated via the following query: The list of keywords generated via the following query: ``` - clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > key_words.dict + clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > keywords.dict ``` Then merge all dictionaries into one (all.dict) ``` - cat ./dictionaries/* | sort | uniq > all.dict + cat ./dictionaries/* | LC_ALL=C sort | uniq > all.dict ``` \ No newline at end of file diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index f08e319f0d4..1c3c657d6b0 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -4,26 +4,26 @@ "accurateCastOrNull" "acos" "acosh" -"ADD" "ADD COLUMN" "ADD CONSTRAINT" +"ADD INDEX" +"ADD PROJECTION" +"ADD STATISTICS" +"ADD" "addDate" "addDays" "addHours" -"ADD INDEX" "addInterval" "addMicroseconds" "addMilliseconds" "addMinutes" "addMonths" "addNanoseconds" -"ADD PROJECTION" "addQuarters" "addressToLine" "addressToLineWithInlines" "addressToSymbol" "addSeconds" -"ADD STATISTIC" "addTupleOfIntervals" "addWeeks" "addYears" @@ -53,7 +53,6 @@ "ALL" "ALLOWED_LATENESS" "alphaTokens" -"ALTER" "ALTER COLUMN" "ALTER DATABASE" "ALTER LIVE VIEW" @@ -66,6 +65,7 @@ "ALTER TABLE" "ALTER TEMPORARY TABLE" "ALTER USER" +"ALTER" "analysisOfVariance" "analysisOfVarianceArgMax" "analysisOfVarianceArgMin" @@ -81,9 +81,9 @@ "analysisOfVarianceResample" "analysisOfVarianceSimpleState" "analysisOfVarianceState" -"and" -"AND" "AND STDOUT" +"AND" +"and" "anova" "anovaArgMax" "anovaArgMin" @@ -100,8 +100,8 @@ "anovaSimpleState" "anovaState" "ANTI" -"any" "ANY" +"any" "anyArgMax" "anyArgMin" "anyArray" @@ -136,6 +136,8 @@ "anyLastOrDefault" "anyLastOrNull" "anyLastResample" +"anyLastSimpleState" +"anyLastState" "anyLast_respect_nulls" "anyLast_respect_nullsArgMax" "anyLast_respect_nullsArgMin" @@ -151,14 +153,14 @@ "anyLast_respect_nullsResample" "anyLast_respect_nullsSimpleState" "anyLast_respect_nullsState" -"anyLastSimpleState" -"anyLastState" "anyMap" "anyMerge" "anyNull" "anyOrDefault" "anyOrNull" "anyResample" +"anySimpleState" +"anyState" "any_respect_nulls" "any_respect_nullsArgMax" "any_respect_nullsArgMin" @@ -174,8 +176,6 @@ "any_respect_nullsResample" "any_respect_nullsSimpleState" "any_respect_nullsState" -"anySimpleState" -"anyState" "any_value" "any_valueArgMax" "any_valueArgMin" @@ -189,6 +189,8 @@ "any_valueOrDefault" "any_valueOrNull" "any_valueResample" +"any_valueSimpleState" +"any_valueState" "any_value_respect_nulls" "any_value_respect_nullsArgMax" "any_value_respect_nullsArgMin" @@ -204,12 +206,10 @@ "any_value_respect_nullsResample" "any_value_respect_nullsSimpleState" "any_value_respect_nullsState" -"any_valueSimpleState" -"any_valueState" "APPEND" "appendTrailingCharIfAbsent" -"APPLY" "APPLY DELETED MASK" +"APPLY" "approx_top_count" "approx_top_countArgMax" "approx_top_countArgMin" @@ -285,43 +285,14 @@ "argMinResample" "argMinSimpleState" "argMinState" -"array" +"ARRAY JOIN" "Array" -"array_agg" -"array_aggArgMax" -"array_aggArgMin" -"array_aggArray" -"array_aggDistinct" -"array_aggForEach" -"array_aggIf" -"array_aggMap" -"array_aggMerge" -"array_aggNull" -"array_aggOrDefault" -"array_aggOrNull" -"array_aggResample" -"array_aggSimpleState" -"array_aggState" +"array" "arrayAll" "arrayAUC" "arrayAvg" "arrayCompact" "arrayConcat" -"array_concat_agg" -"array_concat_aggArgMax" -"array_concat_aggArgMin" -"array_concat_aggArray" -"array_concat_aggDistinct" -"array_concat_aggForEach" -"array_concat_aggIf" -"array_concat_aggMap" -"array_concat_aggMerge" -"array_concat_aggNull" -"array_concat_aggOrDefault" -"array_concat_aggOrNull" -"array_concat_aggResample" -"array_concat_aggSimpleState" -"array_concat_aggState" "arrayCount" "arrayCumSum" "arrayCumSumNonNegative" @@ -345,7 +316,6 @@ "arrayIntersect" "arrayJaccardIndex" "arrayJoin" -"ARRAY JOIN" "arrayLast" "arrayLastIndex" "arrayLastOrNull" @@ -382,6 +352,36 @@ "arrayUniq" "arrayWithConstant" "arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" "AS" "ASC" "ASCENDING" @@ -396,7 +396,6 @@ "atan" "atan2" "atanh" -"ATTACH" "ATTACH PART" "ATTACH PARTITION" "ATTACH POLICY" @@ -406,6 +405,7 @@ "ATTACH ROW POLICY" "ATTACH SETTINGS PROFILE" "ATTACH USER" +"ATTACH" "AUTO_INCREMENT" "avg" "avgArgMax" @@ -445,39 +445,17 @@ "base58Encode" "base64Decode" "base64Encode" -"base_backup" +"base64URLDecode" +"base64URLEncode" "basename" +"base_backup" "BCRYPT_HASH" "BCRYPT_PASSWORD" "BEGIN TRANSACTION" "BETWEEN" "BIDIRECTIONAL" -"BIGINT" -"BIGINT SIGNED" -"BIGINT UNSIGNED" "bin" -"BINARY" -"BINARY LARGE OBJECT" -"BINARY VARYING" -"BIT" "bitAnd" -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" "bitCount" "bitHammingDistance" "bitmapAnd" @@ -503,6 +481,31 @@ "bitmaskToList" "bitNot" "bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" "BIT_OR" "BIT_ORArgMax" "BIT_ORArgMin" @@ -518,18 +521,6 @@ "BIT_ORResample" "BIT_ORSimpleState" "BIT_ORState" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"__bitSwapLastTwo" -"bitTest" -"bitTestAll" -"bitTestAny" -"__bitWrapperFunc" -"bitXor" "BIT_XOR" "BIT_XORArgMax" "BIT_XORArgMin" @@ -546,13 +537,9 @@ "BIT_XORSimpleState" "BIT_XORState" "BLAKE3" -"BLOB" "blockNumber" "blockSerializedSize" "blockSize" -"bool" -"Bool" -"boolean" "BOTH" "boundingRatio" "boundingRatioArgMax" @@ -571,8 +558,6 @@ "boundingRatioState" "buildId" "BY" -"BYTE" -"BYTEA" "byteHammingDistance" "byteSize" "byteSlice" @@ -583,7 +568,6 @@ "caseWithExpression" "caseWithoutExpr" "caseWithoutExpression" -"_CAST" "CAST" "catboostEvaluate" "categoricalInformationValue" @@ -607,36 +591,41 @@ "CHANGE" "CHANGEABLE_IN_READONLY" "CHANGED" -"char" -"CHAR" -"CHARACTER" -"CHARACTER LARGE OBJECT" -"CHARACTER_LENGTH" -"CHARACTER VARYING" -"CHAR LARGE OBJECT" -"CHAR_LENGTH" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" "CHAR VARYING" -"CHECK" +"CHAR" +"char" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CHARACTER" +"CHARACTER_LENGTH" +"CHAR_LENGTH" "CHECK ALL TABLES" "CHECK TABLE" +"CHECK" "cityHash64" +"clamp" "CLEANUP" "CLEAR COLUMN" "CLEAR INDEX" "CLEAR PROJECTION" -"CLEAR STATISTIC" -"CLOB" +"CLEAR STATISTICS" "CLUSTER" -"cluster_host_ids" "CLUSTERS" +"cluster_host_ids" "CN" "coalesce" "CODEC" "COLLATE" "COLUMN" "COLUMNS" -"COMMENT" "COMMENT COLUMN" +"COMMENT" "COMMIT" "COMPRESSION" "concat" @@ -644,8 +633,8 @@ "concatWithSeparator" "concatWithSeparatorAssumeInjective" "concat_ws" -"connection_id" "connectionId" +"connection_id" "CONST" "CONSTRAINT" "contingency" @@ -735,21 +724,13 @@ "countSubstringsCaseInsensitive" "countSubstringsCaseInsensitiveUTF8" "covarPop" -"COVAR_POP" "covarPopArgMax" -"COVAR_POPArgMax" "covarPopArgMin" -"COVAR_POPArgMin" "covarPopArray" -"COVAR_POPArray" "covarPopDistinct" -"COVAR_POPDistinct" "covarPopForEach" -"COVAR_POPForEach" "covarPopIf" -"COVAR_POPIf" "covarPopMap" -"COVAR_POPMap" "covarPopMatrix" "covarPopMatrixArgMax" "covarPopMatrixArgMin" @@ -766,17 +747,11 @@ "covarPopMatrixSimpleState" "covarPopMatrixState" "covarPopMerge" -"COVAR_POPMerge" "covarPopNull" -"COVAR_POPNull" "covarPopOrDefault" -"COVAR_POPOrDefault" "covarPopOrNull" -"COVAR_POPOrNull" "covarPopResample" -"COVAR_POPResample" "covarPopSimpleState" -"COVAR_POPSimpleState" "covarPopStable" "covarPopStableArgMax" "covarPopStableArgMin" @@ -793,23 +768,14 @@ "covarPopStableSimpleState" "covarPopStableState" "covarPopState" -"COVAR_POPState" "covarSamp" -"COVAR_SAMP" "covarSampArgMax" -"COVAR_SAMPArgMax" "covarSampArgMin" -"COVAR_SAMPArgMin" "covarSampArray" -"COVAR_SAMPArray" "covarSampDistinct" -"COVAR_SAMPDistinct" "covarSampForEach" -"COVAR_SAMPForEach" "covarSampIf" -"COVAR_SAMPIf" "covarSampMap" -"COVAR_SAMPMap" "covarSampMatrix" "covarSampMatrixArgMax" "covarSampMatrixArgMin" @@ -826,17 +792,11 @@ "covarSampMatrixSimpleState" "covarSampMatrixState" "covarSampMerge" -"COVAR_SAMPMerge" "covarSampNull" -"COVAR_SAMPNull" "covarSampOrDefault" -"COVAR_SAMPOrDefault" "covarSampOrNull" -"COVAR_SAMPOrNull" "covarSampResample" -"COVAR_SAMPResample" "covarSampSimpleState" -"COVAR_SAMPSimpleState" "covarSampStable" "covarSampStableArgMax" "covarSampStableArgMin" @@ -853,6 +813,35 @@ "covarSampStableSimpleState" "covarSampStableState" "covarSampState" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" "COVAR_SAMPState" "cramersV" "cramersVArgMax" @@ -887,7 +876,6 @@ "CRC32" "CRC32IEEE" "CRC64" -"CREATE" "CREATE POLICY" "CREATE PROFILE" "CREATE QUOTA" @@ -897,25 +885,27 @@ "CREATE TABLE" "CREATE TEMPORARY TABLE" "CREATE USER" +"CREATE" "CROSS" "CUBE" "curdate" -"current_database" -"currentDatabase" -"current_date" "CURRENT GRANTS" -"currentProfiles" "CURRENT QUOTA" -"currentRoles" "CURRENT ROLES" "CURRENT ROW" -"current_schemas" -"currentSchemas" -"current_timestamp" "CURRENT TRANSACTION" -"currentUser" -"CURRENT_USER" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" "CURRENTUSER" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"CURRENT_USER" +"current_user" "cutFragment" "cutIPv6" "cutQueryString" @@ -932,27 +922,29 @@ "cutWWW" "D" "damerauLevenshteinDistance" +"DATA INNER UUID" +"DATA" "DATABASE" "DATABASES" -"Date" "DATE" -"Date32" -"DATE_ADD" +"Date" "DATEADD" -"date_diff" -"dateDiff" -"DATE_DIFF" "DATEDIFF" -"DATE_FORMAT" +"dateDiff" "dateName" -"DATE_SUB" "DATESUB" "DateTime" -"DateTime32" "DateTime64" "dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" "dateTimeToSnowflake" +"dateTimeToSnowflakeID" "dateTrunc" +"DATE_ADD" +"DATE_DIFF" +"date_diff" +"DATE_FORMAT" +"DATE_SUB" "DATE_TRUNC" "DAY" "DAYOFMONTH" @@ -960,10 +952,8 @@ "DAYOFYEAR" "DAYS" "DD" -"DEC" "Decimal" "Decimal128" -"Decimal256" "Decimal32" "Decimal64" "decodeHTMLComponent" @@ -972,17 +962,17 @@ "decodeXMLComponent" "decrypt" "DEDUPLICATE" -"DEFAULT" "DEFAULT DATABASE" -"defaultProfiles" "DEFAULT ROLE" +"DEFAULT" +"defaultProfiles" "defaultRoles" "defaultValueOfArgumentType" "defaultValueOfTypeName" "DEFINER" "degrees" -"DELETE" "DELETE WHERE" +"DELETE" "deltaSum" "deltaSumArgMax" "deltaSumArgMin" @@ -1014,6 +1004,21 @@ "deltaSumTimestampSimpleState" "deltaSumTimestampState" "demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" "dense_rank" "dense_rankArgMax" "dense_rankArgMin" @@ -1033,9 +1038,9 @@ "DESC" "DESCENDING" "DESCRIBE" -"DETACH" "DETACH PART" "DETACH PARTITION" +"DETACH" "detectCharset" "detectLanguage" "detectLanguageMixed" @@ -1092,8 +1097,8 @@ "distanceL2Squared" "distanceLinf" "distanceLp" -"DISTINCT" "DISTINCT ON" +"DISTINCT" "DIV" "divide" "divideDecimal" @@ -1102,11 +1107,8 @@ "domainWithoutWWW" "domainWithoutWWWRFC" "dotProduct" -"DOUBLE" -"DOUBLE PRECISION" "DOUBLE_SHA1_HASH" "DOUBLE_SHA1_PASSWORD" -"DROP" "DROP COLUMN" "DROP CONSTRAINT" "DROP DEFAULT" @@ -1116,15 +1118,20 @@ "DROP PART" "DROP PARTITION" "DROP PROJECTION" -"DROP STATISTIC" +"DROP STATISTICS" "DROP TABLE" "DROP TEMPORARY TABLE" +"DROP" "dumpColumnStructure" +"dynamicElement" +"dynamicType" "e" "editDistance" +"editDistanceUTF8" "ELSE" -"empty" +"EMPTY AS" "EMPTY" +"empty" "emptyArrayDate" "emptyArrayDateTime" "emptyArrayFloat32" @@ -1139,10 +1146,9 @@ "emptyArrayUInt32" "emptyArrayUInt64" "emptyArrayUInt8" -"EMPTY AS" +"ENABLED ROLES" "enabledProfiles" "enabledRoles" -"ENABLED ROLES" "encodeURLComponent" "encodeURLFormComponent" "encodeXMLComponent" @@ -1168,11 +1174,10 @@ "entropySimpleState" "entropyState" "Enum" -"ENUM" "Enum16" "Enum8" -"EPHEMERAL" "EPHEMERAL SEQUENTIAL" +"EPHEMERAL" "equals" "erf" "erfc" @@ -1182,11 +1187,11 @@ "EVENT" "EVENTS" "EVERY" -"EXCEPT" "EXCEPT DATABASE" "EXCEPT DATABASES" "EXCEPT TABLE" "EXCEPT TABLES" +"EXCEPT" "EXCHANGE DICTIONARIES" "EXCHANGE TABLES" "EXISTS" @@ -1272,8 +1277,8 @@ "EXPRESSION" "EXTENDED" "EXTERNAL DDL FROM" -"extract" "EXTRACT" +"extract" "extractAll" "extractAllGroups" "extractAllGroupsHorizontal" @@ -1289,15 +1294,15 @@ "FALSE" "farmFingerprint64" "farmHash64" -"FETCH" "FETCH PART" "FETCH PARTITION" +"FETCH" "FIELDS" -"file" "FILE" -"filesystemAvailable" +"file" "FILESYSTEM CACHE" "FILESYSTEM CACHES" +"filesystemAvailable" "filesystemCapacity" "filesystemUnreserved" "FILTER" @@ -1322,6 +1327,8 @@ "first_valueOrDefault" "first_valueOrNull" "first_valueResample" +"first_valueSimpleState" +"first_valueState" "first_value_respect_nulls" "first_value_respect_nullsArgMax" "first_value_respect_nullsArgMin" @@ -1337,9 +1344,6 @@ "first_value_respect_nullsResample" "first_value_respect_nullsSimpleState" "first_value_respect_nullsState" -"first_valueSimpleState" -"first_valueState" -"FIXED" "FixedString" "flameGraph" "flameGraphArgMax" @@ -1358,19 +1362,17 @@ "flameGraphState" "flatten" "flattenTuple" -"FLOAT" "Float32" "Float64" "floor" "FOLLOWING" "FOR" "ForEach" -"FOREIGN" "FOREIGN KEY" +"FOREIGN" "FORGET PARTITION" -"format" "FORMAT" -"FORMAT_BYTES" +"format" "formatDateTime" "formatDateTimeInJodaSyntax" "formatQuery" @@ -1383,26 +1385,27 @@ "formatReadableTimeDelta" "formatRow" "formatRowNoNewline" +"FORMAT_BYTES" "FQDN" "fragment" "FREEZE" +"FROM INFILE" +"FROM SHARD" "FROM" -"FROM_BASE64" -"FROM_DAYS" "fromDaysSinceYearZero" "fromDaysSinceYearZero32" -"FROM INFILE" "fromModifiedJulianDay" "fromModifiedJulianDayOrNull" -"FROM SHARD" -"FROM_UNIXTIME" "fromUnixTimestamp" "fromUnixTimestamp64Micro" "fromUnixTimestamp64Milli" "fromUnixTimestamp64Nano" "fromUnixTimestampInJodaSyntax" -"from_utc_timestamp" "fromUTCTimestamp" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"from_utc_timestamp" "FULL" "fullHostName" "FULLTEXT" @@ -1411,44 +1414,46 @@ "gccMurmurHash" "gcd" "generateRandomStructure" +"generateSnowflakeID" "generateULID" "generateUUIDv4" +"generateUUIDv7" "geoDistance" "geohashDecode" "geohashEncode" "geohashesInBox" -"GEOMETRY" "geoToH3" "geoToS2" +"getClientHTTPHeader" "getMacro" "getOSKernelVersion" -"__getScalar" "getServerPort" "getSetting" "getSizeOfEnumType" "getSubcolumn" "getTypeSerializationStreams" +"GLOBAL IN" +"GLOBAL NOT IN" "GLOBAL" "globalIn" -"GLOBAL IN" "globalInIgnoreSet" "globalNotIn" -"GLOBAL NOT IN" "globalNotInIgnoreSet" "globalNotNullIn" "globalNotNullInIgnoreSet" "globalNullIn" "globalNullInIgnoreSet" "globalVariable" +"GRANT OPTION FOR" "GRANT" "GRANTEES" -"GRANT OPTION FOR" "GRANULARITY" "greatCircleAngle" "greatCircleDistance" "greater" "greaterOrEquals" "greatest" +"GROUP BY" "groupArray" "groupArrayArgMax" "groupArrayArgMin" @@ -1673,7 +1678,21 @@ "groupBitXorResample" "groupBitXorSimpleState" "groupBitXorState" -"GROUP BY" +"groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" "GROUPING SETS" "GROUPS" "groupUniqArray" @@ -1691,6 +1710,21 @@ "groupUniqArrayResample" "groupUniqArraySimpleState" "groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" "H" "h3CellAreaM2" "h3CellAreaRads2" @@ -1753,6 +1787,8 @@ "hex" "HH" "HIERARCHICAL" +"hilbertDecode" +"hilbertEncode" "histogram" "histogramArgMax" "histogramArgMin" @@ -1773,8 +1809,8 @@ "hopEnd" "hopStart" "HOST" -"hostname" "hostName" +"hostname" "HOUR" "HOURS" "HTTP" @@ -1784,25 +1820,24 @@ "identity" "idnaDecode" "idnaEncode" -"if" "IF EMPTY" "IF EXISTS" "IF NOT EXISTS" +"if" "ifNotFinite" "ifNull" -"ignore" "IGNORE NULLS" -"ilike" +"ignore" "ILIKE" -"in" +"ilike" +"IN PARTITION" "IN" +"in" "INDEX" "INDEXES" "indexHint" "indexOf" "INDICES" -"INET4" -"INET6" "INET6_ATON" "INET6_NTOA" "INET_ATON" @@ -1812,28 +1847,18 @@ "initcap" "initcapUTF8" "initializeAggregation" -"initial_query_id" "initialQueryID" +"initial_query_id" "INJECTIVE" "INNER" -"IN PARTITION" "INSERT INTO" "instr" -"INT" -"INT1" -"Int128" "Int16" -"INT1 SIGNED" -"INT1 UNSIGNED" -"Int256" "Int32" "Int64" "Int8" "intDiv" "intDivOrZero" -"INTEGER" -"INTEGER SIGNED" -"INTEGER UNSIGNED" "INTERPOLATE" "INTERSECT" "INTERVAL" @@ -1854,11 +1879,8 @@ "intervalLengthSumResample" "intervalLengthSumSimpleState" "intervalLengthSumState" -"IntervalMicrosecond" -"IntervalMillisecond" "IntervalMinute" "IntervalMonth" -"IntervalNanosecond" "IntervalQuarter" "IntervalSecond" "IntervalWeek" @@ -1868,12 +1890,9 @@ "intHash32" "intHash64" "INTO OUTFILE" -"INT SIGNED" -"INT UNSIGNED" "INVISIBLE" "INVOKER" "IP" -"IPv4" "IPv4CIDRToRange" "IPv4NumToString" "IPv4NumToStringClassC" @@ -1881,12 +1900,14 @@ "IPv4StringToNumOrDefault" "IPv4StringToNumOrNull" "IPv4ToIPv6" -"IPv6" "IPv6CIDRToRange" "IPv6NumToString" "IPv6StringToNum" "IPv6StringToNumOrDefault" "IPv6StringToNumOrNull" +"IS NOT DISTINCT FROM" +"IS NOT NULL" +"IS NULL" "isConstant" "isDecimalOverflow" "isFinite" @@ -1896,16 +1917,13 @@ "isIPv6String" "isNaN" "isNotDistinctFrom" -"IS NOT DISTINCT FROM" "isNotNull" -"IS NOT NULL" "isNull" -"IS NULL" "isNullable" -"IS_OBJECT_ID" "isValidJSON" "isValidUTF8" "isZeroOrNull" +"IS_OBJECT_ID" "jaroSimilarity" "jaroWinklerSimilarity" "javaHash" @@ -1913,10 +1931,7 @@ "JOIN" "joinGet" "joinGetOrNull" -"JSON" "JSONArrayLength" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" "JSONExtract" "JSONExtractArrayRaw" "JSONExtractBool" @@ -1931,15 +1946,19 @@ "JSONHas" "JSONKey" "JSONLength" +"JSONMergePatch" "jsonMergePatch" -"JSON_QUERY" "JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" "JSON_VALUE" "jumpConsistentHash" +"JWT" "kafkaMurmurHash" "KERBEROS" -"KEY" "KEY BY" +"KEY" "KEYED BY" "KEYS" "KILL" @@ -2046,6 +2065,8 @@ "last_valueOrDefault" "last_valueOrNull" "last_valueResample" +"last_valueSimpleState" +"last_valueState" "last_value_respect_nulls" "last_value_respect_nullsArgMax" "last_value_respect_nullsArgMin" @@ -2061,8 +2082,6 @@ "last_value_respect_nullsResample" "last_value_respect_nullsSimpleState" "last_value_respect_nullsState" -"last_valueSimpleState" -"last_valueState" "LAYOUT" "lcase" "lcm" @@ -2084,25 +2103,26 @@ "leadInFrameState" "LEADING" "least" -"left" -"LEFT" "LEFT ARRAY JOIN" +"LEFT" +"left" "leftPad" "leftPadUTF8" "leftUTF8" "lemmatize" "length" "lengthUTF8" +"LESS THAN" "less" "lessOrEquals" -"LESS THAN" "LEVEL" "levenshteinDistance" +"levenshteinDistanceUTF8" "lgamma" "LIFETIME" "LIGHTWEIGHT" -"like" "LIKE" +"like" "LIMIT" "LINEAR" "LinfDistance" @@ -2118,8 +2138,6 @@ "log1p" "log2" "logTrace" -"LONGBLOB" -"LONGTEXT" "LowCardinality" "lowCardinalityIndices" "lowCardinalityKeys" @@ -2129,8 +2147,8 @@ "LpDistance" "LpNorm" "LpNormalize" -"ltrim" "LTRIM" +"ltrim" "lttb" "lttbArgMax" "lttbArgMin" @@ -2170,7 +2188,6 @@ "mannWhitneyUTestSimpleState" "mannWhitneyUTestState" "map" -"Map" "mapAdd" "mapAll" "mapApply" @@ -2181,7 +2198,6 @@ "mapExtractKeyLike" "mapFilter" "mapFromArrays" -"MAP_FROM_ARRAYS" "mapFromString" "mapKeys" "mapPartialReverseSort" @@ -2192,18 +2208,19 @@ "mapSubtract" "mapUpdate" "mapValues" -"match" +"MAP_FROM_ARRAYS" "MATCH" -"materialize" -"MATERIALIZE" +"match" "MATERIALIZE COLUMN" -"MATERIALIZED" "MATERIALIZE INDEX" "MATERIALIZE PROJECTION" -"MATERIALIZE STATISTIC" +"MATERIALIZE STATISTICS" "MATERIALIZE TTL" -"max" +"MATERIALIZE" +"materialize" +"MATERIALIZED" "MAX" +"max" "max2" "maxArgMax" "maxArgMin" @@ -2507,14 +2524,11 @@ "medianTimingWeightedResample" "medianTimingWeightedSimpleState" "medianTimingWeightedState" -"MEDIUMBLOB" -"MEDIUMINT" -"MEDIUMINT SIGNED" -"MEDIUMINT UNSIGNED" -"MEDIUMTEXT" "MEMORY" "Merge" "MERGES" +"METRICS INNER UUID" +"METRICS" "metroHash64" "MI" "MICROSECOND" @@ -2522,8 +2536,8 @@ "mid" "MILLISECOND" "MILLISECONDS" -"min" "MIN" +"min" "min2" "minArgMax" "minArgMin" @@ -2562,18 +2576,20 @@ "MINUTES" "mismatches" "MM" -"mod" "MOD" -"MODIFY" +"mod" "MODIFY COLUMN" "MODIFY COMMENT" +"MODIFY DEFINER" "MODIFY ORDER BY" "MODIFY QUERY" "MODIFY REFRESH" "MODIFY SAMPLE BY" "MODIFY SETTING" "MODIFY SQL SECURITY" +"MODIFY STATISTICS" "MODIFY TTL" +"MODIFY" "modulo" "moduloLegacy" "moduloOrZero" @@ -2582,9 +2598,9 @@ "MONTHS" "mortonDecode" "mortonEncode" -"MOVE" "MOVE PART" "MOVE PARTITION" +"MOVE" "movingXXX" "MS" "multiFuzzyMatchAllIndices" @@ -2596,7 +2612,6 @@ "multiMatchAnyIndex" "multiply" "multiplyDecimal" -"MultiPolygon" "multiSearchAllPositions" "multiSearchAllPositionsCaseInsensitive" "multiSearchAllPositionsCaseInsensitiveUTF8" @@ -2624,18 +2639,10 @@ "NAMED COLLECTION" "NANOSECOND" "NANOSECONDS" -"NATIONAL CHAR" -"NATIONAL CHARACTER" -"NATIONAL CHARACTER LARGE OBJECT" -"NATIONAL CHARACTER VARYING" -"NATIONAL CHAR VARYING" -"NCHAR" -"NCHAR LARGE OBJECT" -"NCHAR VARYING" "negate" "neighbor" -"nested" "Nested" +"nested" "netloc" "NEXT" "ngramDistance" @@ -2678,7 +2685,6 @@ "nonNegativeDerivativeResample" "nonNegativeDerivativeSimpleState" "nonNegativeDerivativeState" -"NO_PASSWORD" "normalizedQueryHash" "normalizedQueryHashKeepNames" "normalizeL1" @@ -2696,13 +2702,19 @@ "normL2Squared" "normLinf" "normLp" -"not" -"NOT" "NOT BETWEEN" +"NOT IDENTIFIED" +"NOT ILIKE" +"NOT IN" +"NOT KEYED" +"NOT LIKE" +"NOT OVERRIDABLE" +"NOT" +"not" "notEmpty" "notEquals" -"nothing" "Nothing" +"nothing" "nothingArgMax" "nothingArgMin" "nothingArray" @@ -2746,21 +2758,16 @@ "nothingUInt64Resample" "nothingUInt64SimpleState" "nothingUInt64State" -"NOT IDENTIFIED" "notILike" -"NOT ILIKE" "notIn" -"NOT IN" "notInIgnoreSet" -"NOT KEYED" "notLike" -"NOT LIKE" "notNullIn" "notNullInIgnoreSet" -"NOT OVERRIDABLE" "now" "now64" "nowInBlock" +"NO_PASSWORD" "NS" "nth_value" "nth_valueArgMax" @@ -2792,28 +2799,25 @@ "ntileResample" "ntileSimpleState" "ntileState" -"Null" "NULL" +"Null" "Nullable" "nullIf" "nullIn" "nullInIgnoreSet" "NULLS" -"NUMERIC" -"NVARCHAR" -"Object" "OCTET_LENGTH" "OFFSET" -"ON" "ON DELETE" -"ONLY" "ON UPDATE" "ON VOLUME" +"ON" +"ONLY" "OPTIMIZE TABLE" -"or" -"OR" -"ORDER BY" "OR REPLACE" +"OR" +"or" +"ORDER BY" "OUTER" "OVER" "OVERRIDABLE" @@ -2838,32 +2842,64 @@ "parseDateTimeInJodaSyntaxOrZero" "parseDateTimeOrNull" "parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" "parseTimeDelta" "PART" "PARTIAL" -"PARTITION" "PARTITION BY" +"PARTITION" +"partitionID" "partitionId" "PARTITIONS" "PART_MOVE_TO_SHARD" "PASTE" "path" "pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" "PERIODIC REFRESH" "PERMANENTLY" "PERMISSIVE" -"PERSISTENT" "PERSISTENT SEQUENTIAL" +"PERSISTENT" "pi" "PIPELINE" "PLAINTEXT_PASSWORD" "PLAN" "plus" "pmod" -"Point" "pointInEllipses" "pointInPolygon" -"Polygon" "polygonAreaCartesian" "polygonAreaSpherical" "polygonConvexHullCartesian" @@ -2887,15 +2923,16 @@ "positionCaseInsensitive" "positionCaseInsensitiveUTF8" "positionUTF8" -"positive_modulo" "positiveModulo" +"positive_modulo" "pow" "power" "PRECEDING" "PRECISION" "PREWHERE" -"PRIMARY" "PRIMARY KEY" +"PRIMARY" +"printf" "PROFILE" "PROJECTION" "proportionsZTest" @@ -2906,6 +2943,7 @@ "punycodeEncode" "Q" "QQ" +"QUALIFY" "quantile" "quantileArgMax" "quantileArgMin" @@ -3418,12 +3456,12 @@ "quantileTimingWeightedState" "QUARTER" "QUARTERS" +"QUERY TREE" "QUERY" -"query_id" "queryID" "queryString" "queryStringAndFragment" -"QUERY TREE" +"query_id" "QUOTA" "radians" "rand" @@ -3440,16 +3478,16 @@ "randNegativeBinomial" "randNormal" "randomFixedString" -"RANDOMIZED" "RANDOMIZE FOR" +"RANDOMIZED" "randomPrintableASCII" "randomString" "randomStringUTF8" "randPoisson" "randStudentT" "randUniform" -"range" "RANGE" +"range" "rank" "rankArgMax" "rankArgMin" @@ -3481,20 +3519,22 @@ "rankSimpleState" "rankState" "READONLY" +"readWKTLineString" +"readWKTMultiLineString" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" "readWKTRing" -"REAL" "REALM" "RECOMPRESS" +"RECURSIVE" "REFERENCES" "REFRESH" "REGEXP" "regexpExtract" +"regexpQuoteMeta" "REGEXP_EXTRACT" "REGEXP_MATCHES" -"regexpQuoteMeta" "REGEXP_REPLACE" "regionHierarchy" "regionIn" @@ -3526,21 +3566,21 @@ "reinterpretAsUInt64" "reinterpretAsUInt8" "reinterpretAsUUID" -"REMOVE" "REMOVE SAMPLE BY" "REMOVE TTL" -"RENAME" +"REMOVE" "RENAME COLUMN" "RENAME DATABASE" "RENAME DICTIONARY" "RENAME TABLE" "RENAME TO" +"RENAME" "repeat" -"replace" +"REPLACE PARTITION" "REPLACE" +"replace" "replaceAll" "replaceOne" -"REPLACE PARTITION" "replaceRegexpAll" "replaceRegexpOne" "replicate" @@ -3570,12 +3610,11 @@ "reverseUTF8" "revision" "REVOKE" -"right" "RIGHT" +"right" "rightPad" "rightPadUTF8" "rightUTF8" -"Ring" "ROLLBACK" "ROLLUP" "round" @@ -3585,6 +3624,9 @@ "roundDuration" "roundToExp2" "ROW" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"ROWS" "row_number" "row_numberArgMax" "row_numberArgMin" @@ -3592,8 +3634,6 @@ "row_numberDistinct" "row_numberForEach" "row_numberIf" -"rowNumberInAllBlocks" -"rowNumberInBlock" "row_numberMap" "row_numberMerge" "row_numberNull" @@ -3602,10 +3642,9 @@ "row_numberResample" "row_numberSimpleState" "row_numberState" -"ROWS" "rpad" -"rtrim" "RTRIM" +"rtrim" "runningAccumulate" "runningConcurrency" "runningDifference" @@ -3622,10 +3661,10 @@ "s2ToGeo" "S3" "SALT" -"SAMPLE" "SAMPLE BY" +"SAMPLE" +"SAN" "scalarProduct" -"__scalarSubqueryResult" "SCHEMA" "SCHEME" "SECOND" @@ -3681,18 +3720,18 @@ "seriesOutliersDetectTukey" "seriesPeriodDetectFFT" "SERVER" -"serverTimezone" "serverTimeZone" +"serverTimezone" "serverUUID" -"SET" -"SET DEFAULT" "SET DEFAULT ROLE" +"SET DEFAULT" "SET FAKE TIME" "SET NULL" -"SET ROLE" "SET ROLE DEFAULT" -"SETTINGS" +"SET ROLE" "SET TRANSACTION SNAPSHOT" +"SET" +"SETTINGS" "SHA1" "SHA224" "SHA256" @@ -3703,9 +3742,7 @@ "SHA512_256" "shardCount" "shardNum" -"SHOW" "SHOW ACCESS" -"showCertificate" "SHOW CREATE" "SHOW ENGINES" "SHOW FUNCTIONS" @@ -3713,11 +3750,12 @@ "SHOW PRIVILEGES" "SHOW PROCESSLIST" "SHOW SETTING" +"SHOW" +"showCertificate" "sigmoid" "sign" "SIGNED" "SIMPLE" -"SimpleAggregateFunction" "simpleJSONExtractBool" "simpleJSONExtractFloat" "simpleJSONExtractInt" @@ -3741,7 +3779,6 @@ "simpleLinearRegressionSimpleState" "simpleLinearRegressionState" "sin" -"SINGLE" "singleValueOrNull" "singleValueOrNullArgMax" "singleValueOrNullArgMin" @@ -3796,44 +3833,43 @@ "skewSampState" "sleep" "sleepEachRow" -"SMALLINT" -"SMALLINT SIGNED" -"SMALLINT UNSIGNED" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" "snowflakeToDateTime" "snowflakeToDateTime64" "soundex" "SOURCE" "space" -"sparkbar" "sparkBar" -"sparkbarArgMax" +"sparkbar" "sparkBarArgMax" -"sparkbarArgMin" +"sparkbarArgMax" "sparkBarArgMin" -"sparkbarArray" +"sparkbarArgMin" "sparkBarArray" -"sparkbarDistinct" +"sparkbarArray" "sparkBarDistinct" -"sparkbarForEach" +"sparkbarDistinct" "sparkBarForEach" -"sparkbarIf" +"sparkbarForEach" "sparkBarIf" -"sparkbarMap" +"sparkbarIf" "sparkBarMap" -"sparkbarMerge" +"sparkbarMap" "sparkBarMerge" -"sparkbarNull" +"sparkbarMerge" "sparkBarNull" -"sparkbarOrDefault" +"sparkbarNull" "sparkBarOrDefault" -"sparkbarOrNull" +"sparkbarOrDefault" "sparkBarOrNull" -"sparkbarResample" +"sparkbarOrNull" "sparkBarResample" -"sparkbarSimpleState" +"sparkbarResample" "sparkBarSimpleState" -"sparkbarState" +"sparkbarSimpleState" "sparkBarState" +"sparkbarState" "SPATIAL" "splitByAlpha" "splitByChar" @@ -3860,42 +3896,29 @@ "SS" "SSH_KEY" "SSL_CERTIFICATE" +"START TRANSACTION" "startsWith" "startsWithUTF8" "State" -"STATISTIC" +"STATISTICS" "STD" "STDArgMax" "STDArgMin" "STDArray" "stddevPop" -"STDDEV_POP" "stddevPopArgMax" -"STDDEV_POPArgMax" "stddevPopArgMin" -"STDDEV_POPArgMin" "stddevPopArray" -"STDDEV_POPArray" "stddevPopDistinct" -"STDDEV_POPDistinct" "stddevPopForEach" -"STDDEV_POPForEach" "stddevPopIf" -"STDDEV_POPIf" "stddevPopMap" -"STDDEV_POPMap" "stddevPopMerge" -"STDDEV_POPMerge" "stddevPopNull" -"STDDEV_POPNull" "stddevPopOrDefault" -"STDDEV_POPOrDefault" "stddevPopOrNull" -"STDDEV_POPOrNull" "stddevPopResample" -"STDDEV_POPResample" "stddevPopSimpleState" -"STDDEV_POPSimpleState" "stddevPopStable" "stddevPopStableArgMax" "stddevPopStableArgMin" @@ -3912,35 +3935,20 @@ "stddevPopStableSimpleState" "stddevPopStableState" "stddevPopState" -"STDDEV_POPState" "stddevSamp" -"STDDEV_SAMP" "stddevSampArgMax" -"STDDEV_SAMPArgMax" "stddevSampArgMin" -"STDDEV_SAMPArgMin" "stddevSampArray" -"STDDEV_SAMPArray" "stddevSampDistinct" -"STDDEV_SAMPDistinct" "stddevSampForEach" -"STDDEV_SAMPForEach" "stddevSampIf" -"STDDEV_SAMPIf" "stddevSampMap" -"STDDEV_SAMPMap" "stddevSampMerge" -"STDDEV_SAMPMerge" "stddevSampNull" -"STDDEV_SAMPNull" "stddevSampOrDefault" -"STDDEV_SAMPOrDefault" "stddevSampOrNull" -"STDDEV_SAMPOrNull" "stddevSampResample" -"STDDEV_SAMPResample" "stddevSampSimpleState" -"STDDEV_SAMPSimpleState" "stddevSampStable" "stddevSampStableArgMax" "stddevSampStableArgMin" @@ -3957,6 +3965,35 @@ "stddevSampStableSimpleState" "stddevSampStableState" "stddevSampState" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" "STDDEV_SAMPState" "STDDistinct" "STDForEach" @@ -4008,10 +4045,10 @@ "stringJaccardIndex" "stringJaccardIndexUTF8" "stringToH3" -"str_to_date" -"str_to_map" "structureToCapnProtoSchema" "structureToProtobufSchema" +"str_to_date" +"str_to_map" "studentTTest" "studentTTestArgMax" "studentTTestArgMin" @@ -4029,16 +4066,16 @@ "studentTTestState" "subBitmap" "subDate" -"SUBPARTITION" "SUBPARTITION BY" +"SUBPARTITION" "SUBPARTITIONS" "substr" -"substring" "SUBSTRING" +"substring" "substringIndex" -"SUBSTRING_INDEX" "substringIndexUTF8" "substringUTF8" +"SUBSTRING_INDEX" "subtractDays" "subtractHours" "subtractInterval" @@ -4173,22 +4210,23 @@ "sumWithOverflowSimpleState" "sumWithOverflowState" "SUSPEND" -"svg" "SVG" +"svg" "SYNC" "synonyms" "SYNTAX" "SYSTEM" -"TABLE" "TABLE OVERRIDE" +"TABLE" "TABLES" +"TAGS INNER UUID" +"TAGS" "tan" "tanh" "tcpPort" -"TEMPORARY" "TEMPORARY TABLE" +"TEMPORARY" "TEST" -"TEXT" "tgamma" "theilsU" "theilsUArgMax" @@ -4208,33 +4246,31 @@ "THEN" "throwIf" "tid" -"TIME" "timeDiff" "timeSlot" "timeSlots" -"timestamp" "TIMESTAMP" -"TIMESTAMP_ADD" +"timestamp" "TIMESTAMPADD" -"timestamp_diff" -"timestampDiff" -"TIMESTAMP_DIFF" "TIMESTAMPDIFF" -"TIMESTAMP_SUB" +"timestampDiff" "TIMESTAMPSUB" -"timezone" +"TIMESTAMP_ADD" +"TIMESTAMP_DIFF" +"timestamp_diff" +"TIMESTAMP_SUB" "timeZone" -"timezoneOf" +"timezone" "timeZoneOf" -"timezoneOffset" +"timezoneOf" "timeZoneOffset" -"TINYBLOB" -"TINYINT" -"TINYINT SIGNED" -"TINYINT UNSIGNED" -"TINYTEXT" +"timezoneOffset" +"TO DISK" +"TO INNER UUID" +"TO SHARD" +"TO TABLE" +"TO VOLUME" "TO" -"TO_BASE64" "toBool" "toColumnTypeName" "toDate" @@ -4258,7 +4294,6 @@ "toDayOfMonth" "toDayOfWeek" "toDayOfYear" -"TO_DAYS" "toDaysSinceYearZero" "toDecimal128" "toDecimal128OrDefault" @@ -4277,7 +4312,6 @@ "toDecimal64OrNull" "toDecimal64OrZero" "toDecimalString" -"TO DISK" "toFixedString" "toFloat32" "toFloat32OrDefault" @@ -4288,7 +4322,6 @@ "toFloat64OrNull" "toFloat64OrZero" "toHour" -"TO INNER UUID" "toInt128" "toInt128OrDefault" "toInt128OrNull" @@ -4389,7 +4422,6 @@ "toRelativeWeekNum" "toRelativeYearNum" "toSecond" -"TO SHARD" "toStartOfDay" "toStartOfFifteenMinutes" "toStartOfFiveMinute" @@ -4409,11 +4441,10 @@ "toStartOfYear" "toString" "toStringCutToZero" -"TO TABLE" "TOTALS" "toTime" -"toTimezone" "toTimeZone" +"toTimezone" "toTypeName" "toUInt128" "toUInt128OrDefault" @@ -4439,25 +4470,26 @@ "toUInt8OrDefault" "toUInt8OrNull" "toUInt8OrZero" -"TO_UNIXTIME" "toUnixTimestamp" "toUnixTimestamp64Micro" "toUnixTimestamp64Milli" "toUnixTimestamp64Nano" -"to_utc_timestamp" "toUTCTimestamp" "toUUID" "toUUIDOrDefault" "toUUIDOrNull" "toUUIDOrZero" "toValidUTF8" -"TO VOLUME" "toWeek" "toYear" "toYearWeek" "toYYYYMM" "toYYYYMMDD" "toYYYYMMDDhhmmss" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"to_utc_timestamp" "TRACKING ONLY" "TRAILING" "TRANSACTION" @@ -4468,17 +4500,18 @@ "translate" "translateUTF8" "TRIGGER" -"trim" "TRIM" +"trim" "trimBoth" "trimLeft" "trimRight" "TRUE" "trunc" -"truncate" "TRUNCATE" +"truncate" "tryBase58Decode" "tryBase64Decode" +"tryBase64URLDecode" "tryDecrypt" "tryIdnaEncode" "tryPunycodeDecode" @@ -4486,8 +4519,8 @@ "tumble" "tumbleEnd" "tumbleStart" -"tuple" "Tuple" +"tuple" "tupleConcat" "tupleDivide" "tupleDivideByNumber" @@ -4502,15 +4535,14 @@ "tupleModuloByNumber" "tupleMultiply" "tupleMultiplyByNumber" +"tupleNames" "tupleNegate" "tuplePlus" "tupleToNameValuePairs" "TYPE" "TYPEOF" "ucase" -"UInt128" "UInt16" -"UInt256" "UInt32" "UInt64" "UInt8" @@ -4643,48 +4675,32 @@ "USE" "user" "USING" -"UTC_timestamp" "UTCTimestamp" +"UTC_timestamp" "UUID" "UUIDNumToString" "UUIDStringToNum" -"validateNestedArraySizes" +"UUIDToNum" +"UUIDv7ToDateTime" "VALID UNTIL" +"validateNestedArraySizes" "VALUES" -"VARBINARY" -"VARCHAR" -"VARCHAR2" -"Variant" "variantElement" "variantType" "varPop" -"VAR_POP" "varPopArgMax" -"VAR_POPArgMax" "varPopArgMin" -"VAR_POPArgMin" "varPopArray" -"VAR_POPArray" "varPopDistinct" -"VAR_POPDistinct" "varPopForEach" -"VAR_POPForEach" "varPopIf" -"VAR_POPIf" "varPopMap" -"VAR_POPMap" "varPopMerge" -"VAR_POPMerge" "varPopNull" -"VAR_POPNull" "varPopOrDefault" -"VAR_POPOrDefault" "varPopOrNull" -"VAR_POPOrNull" "varPopResample" -"VAR_POPResample" "varPopSimpleState" -"VAR_POPSimpleState" "varPopStable" "varPopStableArgMax" "varPopStableArgMin" @@ -4701,35 +4717,20 @@ "varPopStableSimpleState" "varPopStableState" "varPopState" -"VAR_POPState" "varSamp" -"VAR_SAMP" "varSampArgMax" -"VAR_SAMPArgMax" "varSampArgMin" -"VAR_SAMPArgMin" "varSampArray" -"VAR_SAMPArray" "varSampDistinct" -"VAR_SAMPDistinct" "varSampForEach" -"VAR_SAMPForEach" "varSampIf" -"VAR_SAMPIf" "varSampMap" -"VAR_SAMPMap" "varSampMerge" -"VAR_SAMPMerge" "varSampNull" -"VAR_SAMPNull" "varSampOrDefault" -"VAR_SAMPOrDefault" "varSampOrNull" -"VAR_SAMPOrNull" "varSampResample" -"VAR_SAMPResample" "varSampSimpleState" -"VAR_SAMPSimpleState" "varSampStable" "varSampStableArgMax" "varSampStableArgMin" @@ -4746,8 +4747,37 @@ "varSampStableSimpleState" "varSampStableState" "varSampState" -"VAR_SAMPState" "VARYING" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" "vectorDifference" "vectorSum" "version" @@ -4763,8 +4793,8 @@ "visitParamHas" "WATCH" "WATERMARK" -"week" "WEEK" +"week" "WEEKS" "welchTTest" "welchTTestArgMax" @@ -4783,8 +4813,8 @@ "welchTTestState" "WHEN" "WHERE" -"width_bucket" "widthBucket" +"width_bucket" "WINDOW" "windowFunnel" "windowFunnelArgMax" @@ -4802,15 +4832,15 @@ "windowFunnelSimpleState" "windowFunnelState" "windowID" -"WITH" "WITH ADMIN OPTION" "WITH CHECK" "WITH FILL" "WITH GRANT OPTION" -"with_itemindex" "WITH NAME" "WITH REPLACE OPTION" "WITH TIES" +"WITH" +"WITH_ITEMINDEX" "WK" "wkt" "wordShingleMinHash" @@ -4845,3 +4875,11 @@ "YYYYMMDDToDate32" "ZKPATH" "zookeeperSessionUptime" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" diff --git a/tests/fuzz/dictionaries/datatypes.dict b/tests/fuzz/dictionaries/datatypes.dict index a01a94fd3e3..e562595fb67 100644 --- a/tests/fuzz/dictionaries/datatypes.dict +++ b/tests/fuzz/dictionaries/datatypes.dict @@ -1,135 +1,4283 @@ -"AggregateFunction" -"Array" -"BIGINT" -"BIGINT SIGNED" -"BIGINT UNSIGNED" -"BINARY" -"BINARY LARGE OBJECT" -"BINARY VARYING" -"BIT" -"BLOB" -"BYTE" -"BYTEA" -"Bool" -"CHAR" -"CHAR LARGE OBJECT" -"CHAR VARYING" -"CHARACTER" -"CHARACTER LARGE OBJECT" -"CHARACTER VARYING" -"CLOB" -"DEC" -"DOUBLE" -"DOUBLE PRECISION" -"Date" -"Date32" -"DateTime" -"DateTime32" -"DateTime64" -"Decimal" -"Decimal128" -"Decimal256" -"Decimal32" -"Decimal64" -"ENUM" -"Enum" -"Enum16" -"Enum8" -"FIXED" -"FLOAT" -"FixedString" -"Float32" -"Float64" -"GEOMETRY" -"INET4" -"INET6" -"INT" -"INT SIGNED" -"INT UNSIGNED" -"INT1" -"INT1 SIGNED" -"INT1 UNSIGNED" -"INTEGER" -"INTEGER SIGNED" -"INTEGER UNSIGNED" -"IPv4" -"IPv6" -"Int128" -"Int16" -"Int256" -"Int32" -"Int64" -"Int8" -"IntervalDay" -"IntervalHour" -"IntervalMicrosecond" -"IntervalMillisecond" -"IntervalMinute" -"IntervalMonth" -"IntervalNanosecond" -"IntervalQuarter" -"IntervalSecond" -"IntervalWeek" -"IntervalYear" -"JSON" -"LONGBLOB" -"LONGTEXT" -"LowCardinality" -"MEDIUMBLOB" -"MEDIUMINT" -"MEDIUMINT SIGNED" -"MEDIUMINT UNSIGNED" -"MEDIUMTEXT" -"Map" -"MultiPolygon" -"NATIONAL CHAR" -"NATIONAL CHAR VARYING" -"NATIONAL CHARACTER" -"NATIONAL CHARACTER LARGE OBJECT" -"NATIONAL CHARACTER VARYING" -"NCHAR" -"NCHAR LARGE OBJECT" -"NCHAR VARYING" -"NUMERIC" -"NVARCHAR" -"Nested" -"Nothing" -"Nullable" -"Object" -"Point" -"Polygon" -"REAL" -"Ring" -"SET" -"SIGNED" -"SINGLE" -"SMALLINT" -"SMALLINT SIGNED" -"SMALLINT UNSIGNED" -"SimpleAggregateFunction" -"String" -"TEXT" -"TIME" -"TIMESTAMP" -"TINYBLOB" -"TINYINT" -"TINYINT SIGNED" -"TINYINT UNSIGNED" -"TINYTEXT" -"Tuple" -"UInt128" -"UInt16" -"UInt256" -"UInt32" -"UInt64" -"UInt8" -"UNSIGNED" -"UUID" -"VARBINARY" -"VARCHAR" -"VARCHAR2" -"Variant" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"CAST" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"DATABASE" +"DATE" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"FORMAT_BYTES" +"FQDN" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"HOUR" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" +"JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" +"JSONExtractString" +"JSONExtractUInt" +"JSONHas" +"JSONKey" +"JSONLength" +"JSONMergePatch" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LAST_DAY" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LpDistance" +"LpNorm" +"LpNormalize" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MD4" +"MD5" +"MILLISECOND" +"MINUTE" +"MONTH" +"OCTET_LENGTH" +"QUARTER" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"SCHEMA" +"SECOND" +"SHA1" +"SHA224" +"SHA256" +"SHA384" +"SHA512" +"SHA512_256" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"SUBSTRING_INDEX" +"SVG" +"TIMESTAMP_DIFF" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"ULIDStringToDateTime" +"URLHash" +"URLHierarchy" +"URLPathHierarchy" +"UTCTimestamp" +"UTC_timestamp" +"UUIDNumToString" +"UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" "YEAR" -"bool" -"boolean" -"Dynamic" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" +"addDate" +"addDays" +"addHours" +"addInterval" +"addMicroseconds" +"addMilliseconds" +"addMinutes" +"addMonths" +"addNanoseconds" +"addQuarters" +"addSeconds" +"addTupleOfIntervals" +"addWeeks" +"addYears" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" +"aes_decrypt_mysql" +"aes_encrypt_mysql" +"age" +"aggThrow" +"aggThrowArgMax" +"aggThrowArgMin" +"aggThrowArray" +"aggThrowDistinct" +"aggThrowForEach" +"aggThrowIf" +"aggThrowMap" +"aggThrowMerge" +"aggThrowNull" +"aggThrowOrDefault" +"aggThrowOrNull" +"aggThrowResample" +"aggThrowSimpleState" +"aggThrowState" +"alphaTokens" +"analysisOfVariance" +"analysisOfVarianceArgMax" +"analysisOfVarianceArgMin" +"analysisOfVarianceArray" +"analysisOfVarianceDistinct" +"analysisOfVarianceForEach" +"analysisOfVarianceIf" +"analysisOfVarianceMap" +"analysisOfVarianceMerge" +"analysisOfVarianceNull" +"analysisOfVarianceOrDefault" +"analysisOfVarianceOrNull" +"analysisOfVarianceResample" +"analysisOfVarianceSimpleState" +"analysisOfVarianceState" +"and" +"anova" +"anovaArgMax" +"anovaArgMin" +"anovaArray" +"anovaDistinct" +"anovaForEach" +"anovaIf" +"anovaMap" +"anovaMerge" +"anovaNull" +"anovaOrDefault" +"anovaOrNull" +"anovaResample" +"anovaSimpleState" +"anovaState" +"any" +"anyArgMax" +"anyArgMin" +"anyArray" +"anyDistinct" +"anyForEach" +"anyHeavy" +"anyHeavyArgMax" +"anyHeavyArgMin" +"anyHeavyArray" +"anyHeavyDistinct" +"anyHeavyForEach" +"anyHeavyIf" +"anyHeavyMap" +"anyHeavyMerge" +"anyHeavyNull" +"anyHeavyOrDefault" +"anyHeavyOrNull" +"anyHeavyResample" +"anyHeavySimpleState" +"anyHeavyState" +"anyIf" +"anyLast" +"anyLastArgMax" +"anyLastArgMin" +"anyLastArray" +"anyLastDistinct" +"anyLastForEach" +"anyLastIf" +"anyLastMap" +"anyLastMerge" +"anyLastNull" +"anyLastOrDefault" +"anyLastOrNull" +"anyLastResample" +"anyLastSimpleState" +"anyLastState" +"anyLast_respect_nulls" +"anyLast_respect_nullsArgMax" +"anyLast_respect_nullsArgMin" +"anyLast_respect_nullsArray" +"anyLast_respect_nullsDistinct" +"anyLast_respect_nullsForEach" +"anyLast_respect_nullsIf" +"anyLast_respect_nullsMap" +"anyLast_respect_nullsMerge" +"anyLast_respect_nullsNull" +"anyLast_respect_nullsOrDefault" +"anyLast_respect_nullsOrNull" +"anyLast_respect_nullsResample" +"anyLast_respect_nullsSimpleState" +"anyLast_respect_nullsState" +"anyMap" +"anyMerge" +"anyNull" +"anyOrDefault" +"anyOrNull" +"anyResample" +"anySimpleState" +"anyState" +"any_respect_nulls" +"any_respect_nullsArgMax" +"any_respect_nullsArgMin" +"any_respect_nullsArray" +"any_respect_nullsDistinct" +"any_respect_nullsForEach" +"any_respect_nullsIf" +"any_respect_nullsMap" +"any_respect_nullsMerge" +"any_respect_nullsNull" +"any_respect_nullsOrDefault" +"any_respect_nullsOrNull" +"any_respect_nullsResample" +"any_respect_nullsSimpleState" +"any_respect_nullsState" +"any_value" +"any_valueArgMax" +"any_valueArgMin" +"any_valueArray" +"any_valueDistinct" +"any_valueForEach" +"any_valueIf" +"any_valueMap" +"any_valueMerge" +"any_valueNull" +"any_valueOrDefault" +"any_valueOrNull" +"any_valueResample" +"any_valueSimpleState" +"any_valueState" +"any_value_respect_nulls" +"any_value_respect_nullsArgMax" +"any_value_respect_nullsArgMin" +"any_value_respect_nullsArray" +"any_value_respect_nullsDistinct" +"any_value_respect_nullsForEach" +"any_value_respect_nullsIf" +"any_value_respect_nullsMap" +"any_value_respect_nullsMerge" +"any_value_respect_nullsNull" +"any_value_respect_nullsOrDefault" +"any_value_respect_nullsOrNull" +"any_value_respect_nullsResample" +"any_value_respect_nullsSimpleState" +"any_value_respect_nullsState" +"appendTrailingCharIfAbsent" +"approx_top_count" +"approx_top_countArgMax" +"approx_top_countArgMin" +"approx_top_countArray" +"approx_top_countDistinct" +"approx_top_countForEach" +"approx_top_countIf" +"approx_top_countMap" +"approx_top_countMerge" +"approx_top_countNull" +"approx_top_countOrDefault" +"approx_top_countOrNull" +"approx_top_countResample" +"approx_top_countSimpleState" +"approx_top_countState" +"approx_top_k" +"approx_top_kArgMax" +"approx_top_kArgMin" +"approx_top_kArray" +"approx_top_kDistinct" +"approx_top_kForEach" +"approx_top_kIf" +"approx_top_kMap" +"approx_top_kMerge" +"approx_top_kNull" +"approx_top_kOrDefault" +"approx_top_kOrNull" +"approx_top_kResample" +"approx_top_kSimpleState" +"approx_top_kState" +"approx_top_sum" +"approx_top_sumArgMax" +"approx_top_sumArgMin" +"approx_top_sumArray" +"approx_top_sumDistinct" +"approx_top_sumForEach" +"approx_top_sumIf" +"approx_top_sumMap" +"approx_top_sumMerge" +"approx_top_sumNull" +"approx_top_sumOrDefault" +"approx_top_sumOrNull" +"approx_top_sumResample" +"approx_top_sumSimpleState" +"approx_top_sumState" +"argMax" +"argMaxArgMax" +"argMaxArgMin" +"argMaxArray" +"argMaxDistinct" +"argMaxForEach" +"argMaxIf" +"argMaxMap" +"argMaxMerge" +"argMaxNull" +"argMaxOrDefault" +"argMaxOrNull" +"argMaxResample" +"argMaxSimpleState" +"argMaxState" +"argMin" +"argMinArgMax" +"argMinArgMin" +"argMinArray" +"argMinDistinct" +"argMinForEach" +"argMinIf" +"argMinMap" +"argMinMerge" +"argMinNull" +"argMinOrDefault" +"argMinOrNull" +"argMinResample" +"argMinSimpleState" +"argMinState" +"array" +"arrayAUC" +"arrayAll" +"arrayAvg" +"arrayCompact" +"arrayConcat" +"arrayCount" +"arrayCumSum" +"arrayCumSumNonNegative" +"arrayDifference" +"arrayDistinct" +"arrayDotProduct" +"arrayElement" +"arrayEnumerate" +"arrayEnumerateDense" +"arrayEnumerateDenseRanked" +"arrayEnumerateUniq" +"arrayEnumerateUniqRanked" +"arrayExists" +"arrayFill" +"arrayFilter" +"arrayFirst" +"arrayFirstIndex" +"arrayFirstOrNull" +"arrayFlatten" +"arrayFold" +"arrayIntersect" +"arrayJaccardIndex" +"arrayJoin" +"arrayLast" +"arrayLastIndex" +"arrayLastOrNull" +"arrayMap" +"arrayMax" +"arrayMin" +"arrayPartialReverseSort" +"arrayPartialShuffle" +"arrayPartialSort" +"arrayPopBack" +"arrayPopFront" +"arrayProduct" +"arrayPushBack" +"arrayPushFront" +"arrayRandomSample" +"arrayReduce" +"arrayReduceInRanges" +"arrayResize" +"arrayReverse" +"arrayReverseFill" +"arrayReverseSort" +"arrayReverseSplit" +"arrayRotateLeft" +"arrayRotateRight" +"arrayShiftLeft" +"arrayShiftRight" +"arrayShingles" +"arrayShuffle" +"arraySlice" +"arraySort" +"arraySplit" +"arrayStringConcat" +"arraySum" +"arrayUniq" +"arrayWithConstant" +"arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" +"ascii" +"asin" +"asinh" +"assumeNotNull" +"atan" +"atan2" +"atanh" +"avg" +"avgArgMax" +"avgArgMin" +"avgArray" +"avgDistinct" +"avgForEach" +"avgIf" +"avgMap" +"avgMerge" +"avgNull" +"avgOrDefault" +"avgOrNull" +"avgResample" +"avgSimpleState" +"avgState" +"avgWeighted" +"avgWeightedArgMax" +"avgWeightedArgMin" +"avgWeightedArray" +"avgWeightedDistinct" +"avgWeightedForEach" +"avgWeightedIf" +"avgWeightedMap" +"avgWeightedMerge" +"avgWeightedNull" +"avgWeightedOrDefault" +"avgWeightedOrNull" +"avgWeightedResample" +"avgWeightedSimpleState" +"avgWeightedState" +"bar" +"base58Decode" +"base58Encode" +"base64Decode" +"base64Encode" +"base64URLDecode" +"base64URLEncode" +"basename" +"bin" +"bitAnd" +"bitCount" +"bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" +"bitmapAnd" +"bitmapAndCardinality" +"bitmapAndnot" +"bitmapAndnotCardinality" +"bitmapBuild" +"bitmapCardinality" +"bitmapContains" +"bitmapHasAll" +"bitmapHasAny" +"bitmapMax" +"bitmapMin" +"bitmapOr" +"bitmapOrCardinality" +"bitmapSubsetInRange" +"bitmapSubsetLimit" +"bitmapToArray" +"bitmapTransform" +"bitmapXor" +"bitmapXorCardinality" +"bitmaskToArray" +"bitmaskToList" +"blockNumber" +"blockSerializedSize" +"blockSize" +"boundingRatio" +"boundingRatioArgMax" +"boundingRatioArgMin" +"boundingRatioArray" +"boundingRatioDistinct" +"boundingRatioForEach" +"boundingRatioIf" +"boundingRatioMap" +"boundingRatioMerge" +"boundingRatioNull" +"boundingRatioOrDefault" +"boundingRatioOrNull" +"boundingRatioResample" +"boundingRatioSimpleState" +"boundingRatioState" +"buildId" +"byteHammingDistance" +"byteSize" +"byteSlice" +"byteSwap" +"caseWithExpr" +"caseWithExpression" +"caseWithoutExpr" +"caseWithoutExpression" +"catboostEvaluate" +"categoricalInformationValue" +"categoricalInformationValueArgMax" +"categoricalInformationValueArgMin" +"categoricalInformationValueArray" +"categoricalInformationValueDistinct" +"categoricalInformationValueForEach" +"categoricalInformationValueIf" +"categoricalInformationValueMap" +"categoricalInformationValueMerge" +"categoricalInformationValueNull" +"categoricalInformationValueOrDefault" +"categoricalInformationValueOrNull" +"categoricalInformationValueResample" +"categoricalInformationValueSimpleState" +"categoricalInformationValueState" +"cbrt" +"ceil" +"ceiling" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" +"char" +"cityHash64" +"clamp" +"coalesce" +"concat" +"concatAssumeInjective" +"concatWithSeparator" +"concatWithSeparatorAssumeInjective" +"concat_ws" +"connectionId" +"connection_id" +"contingency" +"contingencyArgMax" +"contingencyArgMin" +"contingencyArray" +"contingencyDistinct" +"contingencyForEach" +"contingencyIf" +"contingencyMap" +"contingencyMerge" +"contingencyNull" +"contingencyOrDefault" +"contingencyOrNull" +"contingencyResample" +"contingencySimpleState" +"contingencyState" +"convertCharset" +"corr" +"corrArgMax" +"corrArgMin" +"corrArray" +"corrDistinct" +"corrForEach" +"corrIf" +"corrMap" +"corrMatrix" +"corrMatrixArgMax" +"corrMatrixArgMin" +"corrMatrixArray" +"corrMatrixDistinct" +"corrMatrixForEach" +"corrMatrixIf" +"corrMatrixMap" +"corrMatrixMerge" +"corrMatrixNull" +"corrMatrixOrDefault" +"corrMatrixOrNull" +"corrMatrixResample" +"corrMatrixSimpleState" +"corrMatrixState" +"corrMerge" +"corrNull" +"corrOrDefault" +"corrOrNull" +"corrResample" +"corrSimpleState" +"corrStable" +"corrStableArgMax" +"corrStableArgMin" +"corrStableArray" +"corrStableDistinct" +"corrStableForEach" +"corrStableIf" +"corrStableMap" +"corrStableMerge" +"corrStableNull" +"corrStableOrDefault" +"corrStableOrNull" +"corrStableResample" +"corrStableSimpleState" +"corrStableState" +"corrState" +"cos" +"cosh" +"cosineDistance" +"count" +"countArgMax" +"countArgMin" +"countArray" +"countDigits" +"countDistinct" +"countEqual" +"countForEach" +"countIf" +"countMap" +"countMatches" +"countMatchesCaseInsensitive" +"countMerge" +"countNull" +"countOrDefault" +"countOrNull" +"countResample" +"countSimpleState" +"countState" +"countSubstrings" +"countSubstringsCaseInsensitive" +"countSubstringsCaseInsensitiveUTF8" +"covarPop" +"covarPopArgMax" +"covarPopArgMin" +"covarPopArray" +"covarPopDistinct" +"covarPopForEach" +"covarPopIf" +"covarPopMap" +"covarPopMatrix" +"covarPopMatrixArgMax" +"covarPopMatrixArgMin" +"covarPopMatrixArray" +"covarPopMatrixDistinct" +"covarPopMatrixForEach" +"covarPopMatrixIf" +"covarPopMatrixMap" +"covarPopMatrixMerge" +"covarPopMatrixNull" +"covarPopMatrixOrDefault" +"covarPopMatrixOrNull" +"covarPopMatrixResample" +"covarPopMatrixSimpleState" +"covarPopMatrixState" +"covarPopMerge" +"covarPopNull" +"covarPopOrDefault" +"covarPopOrNull" +"covarPopResample" +"covarPopSimpleState" +"covarPopStable" +"covarPopStableArgMax" +"covarPopStableArgMin" +"covarPopStableArray" +"covarPopStableDistinct" +"covarPopStableForEach" +"covarPopStableIf" +"covarPopStableMap" +"covarPopStableMerge" +"covarPopStableNull" +"covarPopStableOrDefault" +"covarPopStableOrNull" +"covarPopStableResample" +"covarPopStableSimpleState" +"covarPopStableState" +"covarPopState" +"covarSamp" +"covarSampArgMax" +"covarSampArgMin" +"covarSampArray" +"covarSampDistinct" +"covarSampForEach" +"covarSampIf" +"covarSampMap" +"covarSampMatrix" +"covarSampMatrixArgMax" +"covarSampMatrixArgMin" +"covarSampMatrixArray" +"covarSampMatrixDistinct" +"covarSampMatrixForEach" +"covarSampMatrixIf" +"covarSampMatrixMap" +"covarSampMatrixMerge" +"covarSampMatrixNull" +"covarSampMatrixOrDefault" +"covarSampMatrixOrNull" +"covarSampMatrixResample" +"covarSampMatrixSimpleState" +"covarSampMatrixState" +"covarSampMerge" +"covarSampNull" +"covarSampOrDefault" +"covarSampOrNull" +"covarSampResample" +"covarSampSimpleState" +"covarSampStable" +"covarSampStableArgMax" +"covarSampStableArgMin" +"covarSampStableArray" +"covarSampStableDistinct" +"covarSampStableForEach" +"covarSampStableIf" +"covarSampStableMap" +"covarSampStableMerge" +"covarSampStableNull" +"covarSampStableOrDefault" +"covarSampStableOrNull" +"covarSampStableResample" +"covarSampStableSimpleState" +"covarSampStableState" +"covarSampState" +"cramersV" +"cramersVArgMax" +"cramersVArgMin" +"cramersVArray" +"cramersVBiasCorrected" +"cramersVBiasCorrectedArgMax" +"cramersVBiasCorrectedArgMin" +"cramersVBiasCorrectedArray" +"cramersVBiasCorrectedDistinct" +"cramersVBiasCorrectedForEach" +"cramersVBiasCorrectedIf" +"cramersVBiasCorrectedMap" +"cramersVBiasCorrectedMerge" +"cramersVBiasCorrectedNull" +"cramersVBiasCorrectedOrDefault" +"cramersVBiasCorrectedOrNull" +"cramersVBiasCorrectedResample" +"cramersVBiasCorrectedSimpleState" +"cramersVBiasCorrectedState" +"cramersVDistinct" +"cramersVForEach" +"cramersVIf" +"cramersVMap" +"cramersVMerge" +"cramersVNull" +"cramersVOrDefault" +"cramersVOrNull" +"cramersVResample" +"cramersVSimpleState" +"cramersVState" +"curdate" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"current_user" +"cutFragment" +"cutIPv6" +"cutQueryString" +"cutQueryStringAndFragment" +"cutToFirstSignificantSubdomain" +"cutToFirstSignificantSubdomainCustom" +"cutToFirstSignificantSubdomainCustomRFC" +"cutToFirstSignificantSubdomainCustomWithWWW" +"cutToFirstSignificantSubdomainCustomWithWWWRFC" +"cutToFirstSignificantSubdomainRFC" +"cutToFirstSignificantSubdomainWithWWW" +"cutToFirstSignificantSubdomainWithWWWRFC" +"cutURLParameter" +"cutWWW" +"damerauLevenshteinDistance" +"dateDiff" +"dateName" +"dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" +"dateTimeToSnowflake" +"dateTimeToSnowflakeID" +"dateTrunc" +"date_diff" +"decodeHTMLComponent" +"decodeURLComponent" +"decodeURLFormComponent" +"decodeXMLComponent" +"decrypt" +"defaultProfiles" +"defaultRoles" +"defaultValueOfArgumentType" +"defaultValueOfTypeName" +"degrees" +"deltaSum" +"deltaSumArgMax" +"deltaSumArgMin" +"deltaSumArray" +"deltaSumDistinct" +"deltaSumForEach" +"deltaSumIf" +"deltaSumMap" +"deltaSumMerge" +"deltaSumNull" +"deltaSumOrDefault" +"deltaSumOrNull" +"deltaSumResample" +"deltaSumSimpleState" +"deltaSumState" +"deltaSumTimestamp" +"deltaSumTimestampArgMax" +"deltaSumTimestampArgMin" +"deltaSumTimestampArray" +"deltaSumTimestampDistinct" +"deltaSumTimestampForEach" +"deltaSumTimestampIf" +"deltaSumTimestampMap" +"deltaSumTimestampMerge" +"deltaSumTimestampNull" +"deltaSumTimestampOrDefault" +"deltaSumTimestampOrNull" +"deltaSumTimestampResample" +"deltaSumTimestampSimpleState" +"deltaSumTimestampState" +"demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" +"dense_rank" +"dense_rankArgMax" +"dense_rankArgMin" +"dense_rankArray" +"dense_rankDistinct" +"dense_rankForEach" +"dense_rankIf" +"dense_rankMap" +"dense_rankMerge" +"dense_rankNull" +"dense_rankOrDefault" +"dense_rankOrNull" +"dense_rankResample" +"dense_rankSimpleState" +"dense_rankState" +"detectCharset" +"detectLanguage" +"detectLanguageMixed" +"detectLanguageUnknown" +"detectProgrammingLanguage" +"detectTonality" +"dictGet" +"dictGetAll" +"dictGetChildren" +"dictGetDate" +"dictGetDateOrDefault" +"dictGetDateTime" +"dictGetDateTimeOrDefault" +"dictGetDescendants" +"dictGetFloat32" +"dictGetFloat32OrDefault" +"dictGetFloat64" +"dictGetFloat64OrDefault" +"dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" +"dictGetInt16" +"dictGetInt16OrDefault" +"dictGetInt32" +"dictGetInt32OrDefault" +"dictGetInt64" +"dictGetInt64OrDefault" +"dictGetInt8" +"dictGetInt8OrDefault" +"dictGetOrDefault" +"dictGetOrNull" +"dictGetString" +"dictGetStringOrDefault" +"dictGetUInt16" +"dictGetUInt16OrDefault" +"dictGetUInt32" +"dictGetUInt32OrDefault" +"dictGetUInt64" +"dictGetUInt64OrDefault" +"dictGetUInt8" +"dictGetUInt8OrDefault" +"dictGetUUID" +"dictGetUUIDOrDefault" +"dictHas" +"dictIsIn" +"displayName" +"distanceL1" +"distanceL2" +"distanceL2Squared" +"distanceLinf" +"distanceLp" +"divide" +"divideDecimal" +"domain" +"domainRFC" +"domainWithoutWWW" +"domainWithoutWWWRFC" +"dotProduct" +"dumpColumnStructure" +"dynamicElement" +"dynamicType" +"e" +"editDistance" +"editDistanceUTF8" +"empty" +"emptyArrayDate" +"emptyArrayDateTime" +"emptyArrayFloat32" +"emptyArrayFloat64" +"emptyArrayInt16" +"emptyArrayInt32" +"emptyArrayInt64" +"emptyArrayInt8" +"emptyArrayString" +"emptyArrayToSingle" +"emptyArrayUInt16" +"emptyArrayUInt32" +"emptyArrayUInt64" +"emptyArrayUInt8" +"enabledProfiles" +"enabledRoles" +"encodeURLComponent" +"encodeURLFormComponent" +"encodeXMLComponent" +"encrypt" +"endsWith" +"endsWithUTF8" +"entropy" +"entropyArgMax" +"entropyArgMin" +"entropyArray" +"entropyDistinct" +"entropyForEach" +"entropyIf" +"entropyMap" +"entropyMerge" +"entropyNull" +"entropyOrDefault" +"entropyOrNull" +"entropyResample" +"entropySimpleState" +"entropyState" +"equals" +"erf" +"erfc" +"errorCodeToName" +"evalMLMethod" +"exp" +"exp10" +"exp2" +"exponentialMovingAverage" +"exponentialMovingAverageArgMax" +"exponentialMovingAverageArgMin" +"exponentialMovingAverageArray" +"exponentialMovingAverageDistinct" +"exponentialMovingAverageForEach" +"exponentialMovingAverageIf" +"exponentialMovingAverageMap" +"exponentialMovingAverageMerge" +"exponentialMovingAverageNull" +"exponentialMovingAverageOrDefault" +"exponentialMovingAverageOrNull" +"exponentialMovingAverageResample" +"exponentialMovingAverageSimpleState" +"exponentialMovingAverageState" +"exponentialTimeDecayedAvg" +"exponentialTimeDecayedAvgArgMax" +"exponentialTimeDecayedAvgArgMin" +"exponentialTimeDecayedAvgArray" +"exponentialTimeDecayedAvgDistinct" +"exponentialTimeDecayedAvgForEach" +"exponentialTimeDecayedAvgIf" +"exponentialTimeDecayedAvgMap" +"exponentialTimeDecayedAvgMerge" +"exponentialTimeDecayedAvgNull" +"exponentialTimeDecayedAvgOrDefault" +"exponentialTimeDecayedAvgOrNull" +"exponentialTimeDecayedAvgResample" +"exponentialTimeDecayedAvgSimpleState" +"exponentialTimeDecayedAvgState" +"exponentialTimeDecayedCount" +"exponentialTimeDecayedCountArgMax" +"exponentialTimeDecayedCountArgMin" +"exponentialTimeDecayedCountArray" +"exponentialTimeDecayedCountDistinct" +"exponentialTimeDecayedCountForEach" +"exponentialTimeDecayedCountIf" +"exponentialTimeDecayedCountMap" +"exponentialTimeDecayedCountMerge" +"exponentialTimeDecayedCountNull" +"exponentialTimeDecayedCountOrDefault" +"exponentialTimeDecayedCountOrNull" +"exponentialTimeDecayedCountResample" +"exponentialTimeDecayedCountSimpleState" +"exponentialTimeDecayedCountState" +"exponentialTimeDecayedMax" +"exponentialTimeDecayedMaxArgMax" +"exponentialTimeDecayedMaxArgMin" +"exponentialTimeDecayedMaxArray" +"exponentialTimeDecayedMaxDistinct" +"exponentialTimeDecayedMaxForEach" +"exponentialTimeDecayedMaxIf" +"exponentialTimeDecayedMaxMap" +"exponentialTimeDecayedMaxMerge" +"exponentialTimeDecayedMaxNull" +"exponentialTimeDecayedMaxOrDefault" +"exponentialTimeDecayedMaxOrNull" +"exponentialTimeDecayedMaxResample" +"exponentialTimeDecayedMaxSimpleState" +"exponentialTimeDecayedMaxState" +"exponentialTimeDecayedSum" +"exponentialTimeDecayedSumArgMax" +"exponentialTimeDecayedSumArgMin" +"exponentialTimeDecayedSumArray" +"exponentialTimeDecayedSumDistinct" +"exponentialTimeDecayedSumForEach" +"exponentialTimeDecayedSumIf" +"exponentialTimeDecayedSumMap" +"exponentialTimeDecayedSumMerge" +"exponentialTimeDecayedSumNull" +"exponentialTimeDecayedSumOrDefault" +"exponentialTimeDecayedSumOrNull" +"exponentialTimeDecayedSumResample" +"exponentialTimeDecayedSumSimpleState" +"exponentialTimeDecayedSumState" +"extract" +"extractAll" +"extractAllGroups" +"extractAllGroupsHorizontal" +"extractAllGroupsVertical" +"extractGroups" +"extractKeyValuePairs" +"extractKeyValuePairsWithEscaping" +"extractTextFromHTML" +"extractURLParameter" +"extractURLParameterNames" +"extractURLParameters" +"factorial" +"farmFingerprint64" +"farmHash64" +"file" +"filesystemAvailable" +"filesystemCapacity" +"filesystemUnreserved" +"finalizeAggregation" +"firstLine" +"firstSignificantSubdomain" +"firstSignificantSubdomainCustom" +"firstSignificantSubdomainCustomRFC" +"firstSignificantSubdomainRFC" +"first_value" +"first_valueArgMax" +"first_valueArgMin" +"first_valueArray" +"first_valueDistinct" +"first_valueForEach" +"first_valueIf" +"first_valueMap" +"first_valueMerge" +"first_valueNull" +"first_valueOrDefault" +"first_valueOrNull" +"first_valueResample" +"first_valueSimpleState" +"first_valueState" +"first_value_respect_nulls" +"first_value_respect_nullsArgMax" +"first_value_respect_nullsArgMin" +"first_value_respect_nullsArray" +"first_value_respect_nullsDistinct" +"first_value_respect_nullsForEach" +"first_value_respect_nullsIf" +"first_value_respect_nullsMap" +"first_value_respect_nullsMerge" +"first_value_respect_nullsNull" +"first_value_respect_nullsOrDefault" +"first_value_respect_nullsOrNull" +"first_value_respect_nullsResample" +"first_value_respect_nullsSimpleState" +"first_value_respect_nullsState" +"flameGraph" +"flameGraphArgMax" +"flameGraphArgMin" +"flameGraphArray" +"flameGraphDistinct" +"flameGraphForEach" +"flameGraphIf" +"flameGraphMap" +"flameGraphMerge" +"flameGraphNull" +"flameGraphOrDefault" +"flameGraphOrNull" +"flameGraphResample" +"flameGraphSimpleState" +"flameGraphState" +"flatten" +"flattenTuple" +"floor" +"format" +"formatDateTime" +"formatDateTimeInJodaSyntax" +"formatQuery" +"formatQueryOrNull" +"formatQuerySingleLine" +"formatQuerySingleLineOrNull" +"formatReadableDecimalSize" +"formatReadableQuantity" +"formatReadableSize" +"formatReadableTimeDelta" +"formatRow" +"formatRowNoNewline" +"fragment" +"fromDaysSinceYearZero" +"fromDaysSinceYearZero32" +"fromModifiedJulianDay" +"fromModifiedJulianDayOrNull" +"fromUTCTimestamp" +"fromUnixTimestamp" +"fromUnixTimestamp64Micro" +"fromUnixTimestamp64Milli" +"fromUnixTimestamp64Nano" +"fromUnixTimestampInJodaSyntax" +"from_utc_timestamp" +"fullHostName" +"fuzzBits" +"gccMurmurHash" +"gcd" +"generateRandomStructure" +"generateSnowflakeID" +"generateULID" +"generateUUIDv4" +"generateUUIDv7" +"geoDistance" +"geoToH3" +"geoToS2" +"geohashDecode" +"geohashEncode" +"geohashesInBox" +"getClientHTTPHeader" +"getMacro" +"getOSKernelVersion" +"getServerPort" +"getSetting" +"getSizeOfEnumType" +"getSubcolumn" +"getTypeSerializationStreams" +"globalIn" +"globalInIgnoreSet" +"globalNotIn" +"globalNotInIgnoreSet" +"globalNotNullIn" +"globalNotNullInIgnoreSet" +"globalNullIn" +"globalNullInIgnoreSet" +"globalVariable" +"greatCircleAngle" +"greatCircleDistance" +"greater" +"greaterOrEquals" +"greatest" +"groupArray" +"groupArrayArgMax" +"groupArrayArgMin" +"groupArrayArray" +"groupArrayDistinct" +"groupArrayForEach" +"groupArrayIf" +"groupArrayInsertAt" +"groupArrayInsertAtArgMax" +"groupArrayInsertAtArgMin" +"groupArrayInsertAtArray" +"groupArrayInsertAtDistinct" +"groupArrayInsertAtForEach" +"groupArrayInsertAtIf" +"groupArrayInsertAtMap" +"groupArrayInsertAtMerge" +"groupArrayInsertAtNull" +"groupArrayInsertAtOrDefault" +"groupArrayInsertAtOrNull" +"groupArrayInsertAtResample" +"groupArrayInsertAtSimpleState" +"groupArrayInsertAtState" +"groupArrayIntersect" +"groupArrayIntersectArgMax" +"groupArrayIntersectArgMin" +"groupArrayIntersectArray" +"groupArrayIntersectDistinct" +"groupArrayIntersectForEach" +"groupArrayIntersectIf" +"groupArrayIntersectMap" +"groupArrayIntersectMerge" +"groupArrayIntersectNull" +"groupArrayIntersectOrDefault" +"groupArrayIntersectOrNull" +"groupArrayIntersectResample" +"groupArrayIntersectSimpleState" +"groupArrayIntersectState" +"groupArrayLast" +"groupArrayLastArgMax" +"groupArrayLastArgMin" +"groupArrayLastArray" +"groupArrayLastDistinct" +"groupArrayLastForEach" +"groupArrayLastIf" +"groupArrayLastMap" +"groupArrayLastMerge" +"groupArrayLastNull" +"groupArrayLastOrDefault" +"groupArrayLastOrNull" +"groupArrayLastResample" +"groupArrayLastSimpleState" +"groupArrayLastState" +"groupArrayMap" +"groupArrayMerge" +"groupArrayMovingAvg" +"groupArrayMovingAvgArgMax" +"groupArrayMovingAvgArgMin" +"groupArrayMovingAvgArray" +"groupArrayMovingAvgDistinct" +"groupArrayMovingAvgForEach" +"groupArrayMovingAvgIf" +"groupArrayMovingAvgMap" +"groupArrayMovingAvgMerge" +"groupArrayMovingAvgNull" +"groupArrayMovingAvgOrDefault" +"groupArrayMovingAvgOrNull" +"groupArrayMovingAvgResample" +"groupArrayMovingAvgSimpleState" +"groupArrayMovingAvgState" +"groupArrayMovingSum" +"groupArrayMovingSumArgMax" +"groupArrayMovingSumArgMin" +"groupArrayMovingSumArray" +"groupArrayMovingSumDistinct" +"groupArrayMovingSumForEach" +"groupArrayMovingSumIf" +"groupArrayMovingSumMap" +"groupArrayMovingSumMerge" +"groupArrayMovingSumNull" +"groupArrayMovingSumOrDefault" +"groupArrayMovingSumOrNull" +"groupArrayMovingSumResample" +"groupArrayMovingSumSimpleState" +"groupArrayMovingSumState" +"groupArrayNull" +"groupArrayOrDefault" +"groupArrayOrNull" +"groupArrayResample" +"groupArraySample" +"groupArraySampleArgMax" +"groupArraySampleArgMin" +"groupArraySampleArray" +"groupArraySampleDistinct" +"groupArraySampleForEach" +"groupArraySampleIf" +"groupArraySampleMap" +"groupArraySampleMerge" +"groupArraySampleNull" +"groupArraySampleOrDefault" +"groupArraySampleOrNull" +"groupArraySampleResample" +"groupArraySampleSimpleState" +"groupArraySampleState" +"groupArraySimpleState" +"groupArraySorted" +"groupArraySortedArgMax" +"groupArraySortedArgMin" +"groupArraySortedArray" +"groupArraySortedDistinct" +"groupArraySortedForEach" +"groupArraySortedIf" +"groupArraySortedMap" +"groupArraySortedMerge" +"groupArraySortedNull" +"groupArraySortedOrDefault" +"groupArraySortedOrNull" +"groupArraySortedResample" +"groupArraySortedSimpleState" +"groupArraySortedState" +"groupArrayState" +"groupBitAnd" +"groupBitAndArgMax" +"groupBitAndArgMin" +"groupBitAndArray" +"groupBitAndDistinct" +"groupBitAndForEach" +"groupBitAndIf" +"groupBitAndMap" +"groupBitAndMerge" +"groupBitAndNull" +"groupBitAndOrDefault" +"groupBitAndOrNull" +"groupBitAndResample" +"groupBitAndSimpleState" +"groupBitAndState" +"groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" +"groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" +"groupBitmap" +"groupBitmapAnd" +"groupBitmapAndArgMax" +"groupBitmapAndArgMin" +"groupBitmapAndArray" +"groupBitmapAndDistinct" +"groupBitmapAndForEach" +"groupBitmapAndIf" +"groupBitmapAndMap" +"groupBitmapAndMerge" +"groupBitmapAndNull" +"groupBitmapAndOrDefault" +"groupBitmapAndOrNull" +"groupBitmapAndResample" +"groupBitmapAndSimpleState" +"groupBitmapAndState" +"groupBitmapArgMax" +"groupBitmapArgMin" +"groupBitmapArray" +"groupBitmapDistinct" +"groupBitmapForEach" +"groupBitmapIf" +"groupBitmapMap" +"groupBitmapMerge" +"groupBitmapNull" +"groupBitmapOr" +"groupBitmapOrArgMax" +"groupBitmapOrArgMin" +"groupBitmapOrArray" +"groupBitmapOrDefault" +"groupBitmapOrDistinct" +"groupBitmapOrForEach" +"groupBitmapOrIf" +"groupBitmapOrMap" +"groupBitmapOrMerge" +"groupBitmapOrNull" +"groupBitmapOrNull" +"groupBitmapOrOrDefault" +"groupBitmapOrOrNull" +"groupBitmapOrResample" +"groupBitmapOrSimpleState" +"groupBitmapOrState" +"groupBitmapResample" +"groupBitmapSimpleState" +"groupBitmapState" +"groupBitmapXor" +"groupBitmapXorArgMax" +"groupBitmapXorArgMin" +"groupBitmapXorArray" +"groupBitmapXorDistinct" +"groupBitmapXorForEach" +"groupBitmapXorIf" +"groupBitmapXorMap" +"groupBitmapXorMerge" +"groupBitmapXorNull" +"groupBitmapXorOrDefault" +"groupBitmapXorOrNull" +"groupBitmapXorResample" +"groupBitmapXorSimpleState" +"groupBitmapXorState" +"groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" +"groupUniqArray" +"groupUniqArrayArgMax" +"groupUniqArrayArgMin" +"groupUniqArrayArray" +"groupUniqArrayDistinct" +"groupUniqArrayForEach" +"groupUniqArrayIf" +"groupUniqArrayMap" +"groupUniqArrayMerge" +"groupUniqArrayNull" +"groupUniqArrayOrDefault" +"groupUniqArrayOrNull" +"groupUniqArrayResample" +"groupUniqArraySimpleState" +"groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" +"h3CellAreaM2" +"h3CellAreaRads2" +"h3Distance" +"h3EdgeAngle" +"h3EdgeLengthKm" +"h3EdgeLengthM" +"h3ExactEdgeLengthKm" +"h3ExactEdgeLengthM" +"h3ExactEdgeLengthRads" +"h3GetBaseCell" +"h3GetDestinationIndexFromUnidirectionalEdge" +"h3GetFaces" +"h3GetIndexesFromUnidirectionalEdge" +"h3GetOriginIndexFromUnidirectionalEdge" +"h3GetPentagonIndexes" +"h3GetRes0Indexes" +"h3GetResolution" +"h3GetUnidirectionalEdge" +"h3GetUnidirectionalEdgeBoundary" +"h3GetUnidirectionalEdgesFromHexagon" +"h3HexAreaKm2" +"h3HexAreaM2" +"h3HexRing" +"h3IndexesAreNeighbors" +"h3IsPentagon" +"h3IsResClassIII" +"h3IsValid" +"h3Line" +"h3NumHexagons" +"h3PointDistKm" +"h3PointDistM" +"h3PointDistRads" +"h3ToCenterChild" +"h3ToChildren" +"h3ToGeo" +"h3ToGeoBoundary" +"h3ToParent" +"h3ToString" +"h3UnidirectionalEdgeIsValid" +"h3kRing" +"halfMD5" +"has" +"hasAll" +"hasAny" +"hasColumnInTable" +"hasSubsequence" +"hasSubsequenceCaseInsensitive" +"hasSubsequenceCaseInsensitiveUTF8" +"hasSubsequenceUTF8" +"hasSubstr" +"hasThreadFuzzer" +"hasToken" +"hasTokenCaseInsensitive" +"hasTokenCaseInsensitiveOrNull" +"hasTokenOrNull" +"hex" +"hilbertDecode" +"hilbertEncode" +"histogram" +"histogramArgMax" +"histogramArgMin" +"histogramArray" +"histogramDistinct" +"histogramForEach" +"histogramIf" +"histogramMap" +"histogramMerge" +"histogramNull" +"histogramOrDefault" +"histogramOrNull" +"histogramResample" +"histogramSimpleState" +"histogramState" +"hiveHash" +"hop" +"hopEnd" +"hopStart" +"hostName" +"hostname" +"hypot" +"identity" +"idnaDecode" +"idnaEncode" +"if" +"ifNotFinite" +"ifNull" +"ignore" +"ilike" +"in" +"inIgnoreSet" +"indexHint" +"indexOf" +"initcap" +"initcapUTF8" +"initialQueryID" +"initial_query_id" +"initializeAggregation" +"instr" +"intDiv" +"intDivOrZero" +"intExp10" +"intExp2" +"intHash32" +"intHash64" +"intervalLengthSum" +"intervalLengthSumArgMax" +"intervalLengthSumArgMin" +"intervalLengthSumArray" +"intervalLengthSumDistinct" +"intervalLengthSumForEach" +"intervalLengthSumIf" +"intervalLengthSumMap" +"intervalLengthSumMerge" +"intervalLengthSumNull" +"intervalLengthSumOrDefault" +"intervalLengthSumOrNull" +"intervalLengthSumResample" +"intervalLengthSumSimpleState" +"intervalLengthSumState" +"isConstant" +"isDecimalOverflow" +"isFinite" +"isIPAddressInRange" +"isIPv4String" +"isIPv6String" +"isInfinite" +"isNaN" +"isNotDistinctFrom" +"isNotNull" +"isNull" +"isNullable" +"isValidJSON" +"isValidUTF8" +"isZeroOrNull" +"jaroSimilarity" +"jaroWinklerSimilarity" +"javaHash" +"javaHashUTF16LE" +"joinGet" +"joinGetOrNull" +"jsonMergePatch" +"jumpConsistentHash" +"kafkaMurmurHash" +"kolmogorovSmirnovTest" +"kolmogorovSmirnovTestArgMax" +"kolmogorovSmirnovTestArgMin" +"kolmogorovSmirnovTestArray" +"kolmogorovSmirnovTestDistinct" +"kolmogorovSmirnovTestForEach" +"kolmogorovSmirnovTestIf" +"kolmogorovSmirnovTestMap" +"kolmogorovSmirnovTestMerge" +"kolmogorovSmirnovTestNull" +"kolmogorovSmirnovTestOrDefault" +"kolmogorovSmirnovTestOrNull" +"kolmogorovSmirnovTestResample" +"kolmogorovSmirnovTestSimpleState" +"kolmogorovSmirnovTestState" +"kostikConsistentHash" +"kql_array_sort_asc" +"kql_array_sort_desc" +"kurtPop" +"kurtPopArgMax" +"kurtPopArgMin" +"kurtPopArray" +"kurtPopDistinct" +"kurtPopForEach" +"kurtPopIf" +"kurtPopMap" +"kurtPopMerge" +"kurtPopNull" +"kurtPopOrDefault" +"kurtPopOrNull" +"kurtPopResample" +"kurtPopSimpleState" +"kurtPopState" +"kurtSamp" +"kurtSampArgMax" +"kurtSampArgMin" +"kurtSampArray" +"kurtSampDistinct" +"kurtSampForEach" +"kurtSampIf" +"kurtSampMap" +"kurtSampMerge" +"kurtSampNull" +"kurtSampOrDefault" +"kurtSampOrNull" +"kurtSampResample" +"kurtSampSimpleState" +"kurtSampState" +"lagInFrame" +"lagInFrameArgMax" +"lagInFrameArgMin" +"lagInFrameArray" +"lagInFrameDistinct" +"lagInFrameForEach" +"lagInFrameIf" +"lagInFrameMap" +"lagInFrameMerge" +"lagInFrameNull" +"lagInFrameOrDefault" +"lagInFrameOrNull" +"lagInFrameResample" +"lagInFrameSimpleState" +"lagInFrameState" +"largestTriangleThreeBuckets" +"largestTriangleThreeBucketsArgMax" +"largestTriangleThreeBucketsArgMin" +"largestTriangleThreeBucketsArray" +"largestTriangleThreeBucketsDistinct" +"largestTriangleThreeBucketsForEach" +"largestTriangleThreeBucketsIf" +"largestTriangleThreeBucketsMap" +"largestTriangleThreeBucketsMerge" +"largestTriangleThreeBucketsNull" +"largestTriangleThreeBucketsOrDefault" +"largestTriangleThreeBucketsOrNull" +"largestTriangleThreeBucketsResample" +"largestTriangleThreeBucketsSimpleState" +"largestTriangleThreeBucketsState" +"last_value" +"last_valueArgMax" +"last_valueArgMin" +"last_valueArray" +"last_valueDistinct" +"last_valueForEach" +"last_valueIf" +"last_valueMap" +"last_valueMerge" +"last_valueNull" +"last_valueOrDefault" +"last_valueOrNull" +"last_valueResample" +"last_valueSimpleState" +"last_valueState" +"last_value_respect_nulls" +"last_value_respect_nullsArgMax" +"last_value_respect_nullsArgMin" +"last_value_respect_nullsArray" +"last_value_respect_nullsDistinct" +"last_value_respect_nullsForEach" +"last_value_respect_nullsIf" +"last_value_respect_nullsMap" +"last_value_respect_nullsMerge" +"last_value_respect_nullsNull" +"last_value_respect_nullsOrDefault" +"last_value_respect_nullsOrNull" +"last_value_respect_nullsResample" +"last_value_respect_nullsSimpleState" +"last_value_respect_nullsState" +"lcase" +"lcm" +"leadInFrame" +"leadInFrameArgMax" +"leadInFrameArgMin" +"leadInFrameArray" +"leadInFrameDistinct" +"leadInFrameForEach" +"leadInFrameIf" +"leadInFrameMap" +"leadInFrameMerge" +"leadInFrameNull" +"leadInFrameOrDefault" +"leadInFrameOrNull" +"leadInFrameResample" +"leadInFrameSimpleState" +"leadInFrameState" +"least" +"left" +"leftPad" +"leftPadUTF8" +"leftUTF8" +"lemmatize" +"length" +"lengthUTF8" +"less" +"lessOrEquals" +"levenshteinDistance" +"levenshteinDistanceUTF8" +"lgamma" +"like" +"ln" +"locate" +"log" +"log10" +"log1p" +"log2" +"logTrace" +"lowCardinalityIndices" +"lowCardinalityKeys" +"lower" +"lowerUTF8" +"lpad" +"ltrim" +"lttb" +"lttbArgMax" +"lttbArgMin" +"lttbArray" +"lttbDistinct" +"lttbForEach" +"lttbIf" +"lttbMap" +"lttbMerge" +"lttbNull" +"lttbOrDefault" +"lttbOrNull" +"lttbResample" +"lttbSimpleState" +"lttbState" +"makeDate" +"makeDate32" +"makeDateTime" +"makeDateTime64" +"mannWhitneyUTest" +"mannWhitneyUTestArgMax" +"mannWhitneyUTestArgMin" +"mannWhitneyUTestArray" +"mannWhitneyUTestDistinct" +"mannWhitneyUTestForEach" +"mannWhitneyUTestIf" +"mannWhitneyUTestMap" +"mannWhitneyUTestMerge" +"mannWhitneyUTestNull" +"mannWhitneyUTestOrDefault" +"mannWhitneyUTestOrNull" +"mannWhitneyUTestResample" +"mannWhitneyUTestSimpleState" +"mannWhitneyUTestState" +"map" +"mapAdd" +"mapAll" +"mapApply" +"mapConcat" +"mapContains" +"mapContainsKeyLike" +"mapExists" +"mapExtractKeyLike" +"mapFilter" +"mapFromArrays" +"mapFromString" +"mapKeys" +"mapPartialReverseSort" +"mapPartialSort" +"mapPopulateSeries" +"mapReverseSort" +"mapSort" +"mapSubtract" +"mapUpdate" +"mapValues" +"match" +"materialize" +"max" +"max2" +"maxArgMax" +"maxArgMin" +"maxArray" +"maxDistinct" +"maxForEach" +"maxIf" +"maxIntersections" +"maxIntersectionsArgMax" +"maxIntersectionsArgMin" +"maxIntersectionsArray" +"maxIntersectionsDistinct" +"maxIntersectionsForEach" +"maxIntersectionsIf" +"maxIntersectionsMap" +"maxIntersectionsMerge" +"maxIntersectionsNull" +"maxIntersectionsOrDefault" +"maxIntersectionsOrNull" +"maxIntersectionsPosition" +"maxIntersectionsPositionArgMax" +"maxIntersectionsPositionArgMin" +"maxIntersectionsPositionArray" +"maxIntersectionsPositionDistinct" +"maxIntersectionsPositionForEach" +"maxIntersectionsPositionIf" +"maxIntersectionsPositionMap" +"maxIntersectionsPositionMerge" +"maxIntersectionsPositionNull" +"maxIntersectionsPositionOrDefault" +"maxIntersectionsPositionOrNull" +"maxIntersectionsPositionResample" +"maxIntersectionsPositionSimpleState" +"maxIntersectionsPositionState" +"maxIntersectionsResample" +"maxIntersectionsSimpleState" +"maxIntersectionsState" +"maxMap" +"maxMappedArrays" +"maxMappedArraysArgMax" +"maxMappedArraysArgMin" +"maxMappedArraysArray" +"maxMappedArraysDistinct" +"maxMappedArraysForEach" +"maxMappedArraysIf" +"maxMappedArraysMap" +"maxMappedArraysMerge" +"maxMappedArraysNull" +"maxMappedArraysOrDefault" +"maxMappedArraysOrNull" +"maxMappedArraysResample" +"maxMappedArraysSimpleState" +"maxMappedArraysState" +"maxMerge" +"maxNull" +"maxOrDefault" +"maxOrNull" +"maxResample" +"maxSimpleState" +"maxState" +"meanZTest" +"meanZTestArgMax" +"meanZTestArgMin" +"meanZTestArray" +"meanZTestDistinct" +"meanZTestForEach" +"meanZTestIf" +"meanZTestMap" +"meanZTestMerge" +"meanZTestNull" +"meanZTestOrDefault" +"meanZTestOrNull" +"meanZTestResample" +"meanZTestSimpleState" +"meanZTestState" +"median" +"medianArgMax" +"medianArgMin" +"medianArray" +"medianBFloat16" +"medianBFloat16ArgMax" +"medianBFloat16ArgMin" +"medianBFloat16Array" +"medianBFloat16Distinct" +"medianBFloat16ForEach" +"medianBFloat16If" +"medianBFloat16Map" +"medianBFloat16Merge" +"medianBFloat16Null" +"medianBFloat16OrDefault" +"medianBFloat16OrNull" +"medianBFloat16Resample" +"medianBFloat16SimpleState" +"medianBFloat16State" +"medianBFloat16Weighted" +"medianBFloat16WeightedArgMax" +"medianBFloat16WeightedArgMin" +"medianBFloat16WeightedArray" +"medianBFloat16WeightedDistinct" +"medianBFloat16WeightedForEach" +"medianBFloat16WeightedIf" +"medianBFloat16WeightedMap" +"medianBFloat16WeightedMerge" +"medianBFloat16WeightedNull" +"medianBFloat16WeightedOrDefault" +"medianBFloat16WeightedOrNull" +"medianBFloat16WeightedResample" +"medianBFloat16WeightedSimpleState" +"medianBFloat16WeightedState" +"medianDD" +"medianDDArgMax" +"medianDDArgMin" +"medianDDArray" +"medianDDDistinct" +"medianDDForEach" +"medianDDIf" +"medianDDMap" +"medianDDMerge" +"medianDDNull" +"medianDDOrDefault" +"medianDDOrNull" +"medianDDResample" +"medianDDSimpleState" +"medianDDState" +"medianDeterministic" +"medianDeterministicArgMax" +"medianDeterministicArgMin" +"medianDeterministicArray" +"medianDeterministicDistinct" +"medianDeterministicForEach" +"medianDeterministicIf" +"medianDeterministicMap" +"medianDeterministicMerge" +"medianDeterministicNull" +"medianDeterministicOrDefault" +"medianDeterministicOrNull" +"medianDeterministicResample" +"medianDeterministicSimpleState" +"medianDeterministicState" +"medianDistinct" +"medianExact" +"medianExactArgMax" +"medianExactArgMin" +"medianExactArray" +"medianExactDistinct" +"medianExactForEach" +"medianExactHigh" +"medianExactHighArgMax" +"medianExactHighArgMin" +"medianExactHighArray" +"medianExactHighDistinct" +"medianExactHighForEach" +"medianExactHighIf" +"medianExactHighMap" +"medianExactHighMerge" +"medianExactHighNull" +"medianExactHighOrDefault" +"medianExactHighOrNull" +"medianExactHighResample" +"medianExactHighSimpleState" +"medianExactHighState" +"medianExactIf" +"medianExactLow" +"medianExactLowArgMax" +"medianExactLowArgMin" +"medianExactLowArray" +"medianExactLowDistinct" +"medianExactLowForEach" +"medianExactLowIf" +"medianExactLowMap" +"medianExactLowMerge" +"medianExactLowNull" +"medianExactLowOrDefault" +"medianExactLowOrNull" +"medianExactLowResample" +"medianExactLowSimpleState" +"medianExactLowState" +"medianExactMap" +"medianExactMerge" +"medianExactNull" +"medianExactOrDefault" +"medianExactOrNull" +"medianExactResample" +"medianExactSimpleState" +"medianExactState" +"medianExactWeighted" +"medianExactWeightedArgMax" +"medianExactWeightedArgMin" +"medianExactWeightedArray" +"medianExactWeightedDistinct" +"medianExactWeightedForEach" +"medianExactWeightedIf" +"medianExactWeightedMap" +"medianExactWeightedMerge" +"medianExactWeightedNull" +"medianExactWeightedOrDefault" +"medianExactWeightedOrNull" +"medianExactWeightedResample" +"medianExactWeightedSimpleState" +"medianExactWeightedState" +"medianForEach" +"medianGK" +"medianGKArgMax" +"medianGKArgMin" +"medianGKArray" +"medianGKDistinct" +"medianGKForEach" +"medianGKIf" +"medianGKMap" +"medianGKMerge" +"medianGKNull" +"medianGKOrDefault" +"medianGKOrNull" +"medianGKResample" +"medianGKSimpleState" +"medianGKState" +"medianIf" +"medianInterpolatedWeighted" +"medianInterpolatedWeightedArgMax" +"medianInterpolatedWeightedArgMin" +"medianInterpolatedWeightedArray" +"medianInterpolatedWeightedDistinct" +"medianInterpolatedWeightedForEach" +"medianInterpolatedWeightedIf" +"medianInterpolatedWeightedMap" +"medianInterpolatedWeightedMerge" +"medianInterpolatedWeightedNull" +"medianInterpolatedWeightedOrDefault" +"medianInterpolatedWeightedOrNull" +"medianInterpolatedWeightedResample" +"medianInterpolatedWeightedSimpleState" +"medianInterpolatedWeightedState" +"medianMap" +"medianMerge" +"medianNull" +"medianOrDefault" +"medianOrNull" +"medianResample" +"medianSimpleState" +"medianState" +"medianTDigest" +"medianTDigestArgMax" +"medianTDigestArgMin" +"medianTDigestArray" +"medianTDigestDistinct" +"medianTDigestForEach" +"medianTDigestIf" +"medianTDigestMap" +"medianTDigestMerge" +"medianTDigestNull" +"medianTDigestOrDefault" +"medianTDigestOrNull" +"medianTDigestResample" +"medianTDigestSimpleState" +"medianTDigestState" +"medianTDigestWeighted" +"medianTDigestWeightedArgMax" +"medianTDigestWeightedArgMin" +"medianTDigestWeightedArray" +"medianTDigestWeightedDistinct" +"medianTDigestWeightedForEach" +"medianTDigestWeightedIf" +"medianTDigestWeightedMap" +"medianTDigestWeightedMerge" +"medianTDigestWeightedNull" +"medianTDigestWeightedOrDefault" +"medianTDigestWeightedOrNull" +"medianTDigestWeightedResample" +"medianTDigestWeightedSimpleState" +"medianTDigestWeightedState" +"medianTiming" +"medianTimingArgMax" +"medianTimingArgMin" +"medianTimingArray" +"medianTimingDistinct" +"medianTimingForEach" +"medianTimingIf" +"medianTimingMap" +"medianTimingMerge" +"medianTimingNull" +"medianTimingOrDefault" +"medianTimingOrNull" +"medianTimingResample" +"medianTimingSimpleState" +"medianTimingState" +"medianTimingWeighted" +"medianTimingWeightedArgMax" +"medianTimingWeightedArgMin" +"medianTimingWeightedArray" +"medianTimingWeightedDistinct" +"medianTimingWeightedForEach" +"medianTimingWeightedIf" +"medianTimingWeightedMap" +"medianTimingWeightedMerge" +"medianTimingWeightedNull" +"medianTimingWeightedOrDefault" +"medianTimingWeightedOrNull" +"medianTimingWeightedResample" +"medianTimingWeightedSimpleState" +"medianTimingWeightedState" +"metroHash64" +"mid" +"min" +"min2" +"minArgMax" +"minArgMin" +"minArray" +"minDistinct" +"minForEach" +"minIf" +"minMap" +"minMappedArrays" +"minMappedArraysArgMax" +"minMappedArraysArgMin" +"minMappedArraysArray" +"minMappedArraysDistinct" +"minMappedArraysForEach" +"minMappedArraysIf" +"minMappedArraysMap" +"minMappedArraysMerge" +"minMappedArraysNull" +"minMappedArraysOrDefault" +"minMappedArraysOrNull" +"minMappedArraysResample" +"minMappedArraysSimpleState" +"minMappedArraysState" +"minMerge" +"minNull" +"minOrDefault" +"minOrNull" +"minResample" +"minSampleSizeContinous" +"minSampleSizeContinuous" +"minSampleSizeConversion" +"minSimpleState" +"minState" +"minus" +"mismatches" +"mod" +"modulo" +"moduloLegacy" +"moduloOrZero" +"monthName" +"mortonDecode" +"mortonEncode" +"multiFuzzyMatchAllIndices" +"multiFuzzyMatchAny" +"multiFuzzyMatchAnyIndex" +"multiIf" +"multiMatchAllIndices" +"multiMatchAny" +"multiMatchAnyIndex" +"multiSearchAllPositions" +"multiSearchAllPositionsCaseInsensitive" +"multiSearchAllPositionsCaseInsensitiveUTF8" +"multiSearchAllPositionsUTF8" +"multiSearchAny" +"multiSearchAnyCaseInsensitive" +"multiSearchAnyCaseInsensitiveUTF8" +"multiSearchAnyUTF8" +"multiSearchFirstIndex" +"multiSearchFirstIndexCaseInsensitive" +"multiSearchFirstIndexCaseInsensitiveUTF8" +"multiSearchFirstIndexUTF8" +"multiSearchFirstPosition" +"multiSearchFirstPositionCaseInsensitive" +"multiSearchFirstPositionCaseInsensitiveUTF8" +"multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" +"murmurHash2_32" +"murmurHash2_64" +"murmurHash3_128" +"murmurHash3_32" +"murmurHash3_64" +"negate" +"neighbor" +"nested" +"netloc" +"ngramDistance" +"ngramDistanceCaseInsensitive" +"ngramDistanceCaseInsensitiveUTF8" +"ngramDistanceUTF8" +"ngramMinHash" +"ngramMinHashArg" +"ngramMinHashArgCaseInsensitive" +"ngramMinHashArgCaseInsensitiveUTF8" +"ngramMinHashArgUTF8" +"ngramMinHashCaseInsensitive" +"ngramMinHashCaseInsensitiveUTF8" +"ngramMinHashUTF8" +"ngramSearch" +"ngramSearchCaseInsensitive" +"ngramSearchCaseInsensitiveUTF8" +"ngramSearchUTF8" +"ngramSimHash" +"ngramSimHashCaseInsensitive" +"ngramSimHashCaseInsensitiveUTF8" +"ngramSimHashUTF8" +"ngrams" +"nonNegativeDerivative" +"nonNegativeDerivativeArgMax" +"nonNegativeDerivativeArgMin" +"nonNegativeDerivativeArray" +"nonNegativeDerivativeDistinct" +"nonNegativeDerivativeForEach" +"nonNegativeDerivativeIf" +"nonNegativeDerivativeMap" +"nonNegativeDerivativeMerge" +"nonNegativeDerivativeNull" +"nonNegativeDerivativeOrDefault" +"nonNegativeDerivativeOrNull" +"nonNegativeDerivativeResample" +"nonNegativeDerivativeSimpleState" +"nonNegativeDerivativeState" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" +"normalizeL1" +"normalizeL2" +"normalizeLinf" +"normalizeLp" +"normalizeQuery" +"normalizeQueryKeepNames" +"normalizeUTF8NFC" +"normalizeUTF8NFD" +"normalizeUTF8NFKC" +"normalizeUTF8NFKD" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" +"not" +"notEmpty" +"notEquals" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" +"nothing" +"nothingArgMax" +"nothingArgMin" +"nothingArray" +"nothingDistinct" +"nothingForEach" +"nothingIf" +"nothingMap" +"nothingMerge" +"nothingNull" +"nothingNull" +"nothingNullArgMax" +"nothingNullArgMin" +"nothingNullArray" +"nothingNullDistinct" +"nothingNullForEach" +"nothingNullIf" +"nothingNullMap" +"nothingNullMerge" +"nothingNullNull" +"nothingNullOrDefault" +"nothingNullOrNull" +"nothingNullResample" +"nothingNullSimpleState" +"nothingNullState" +"nothingOrDefault" +"nothingOrNull" +"nothingResample" +"nothingSimpleState" +"nothingState" +"nothingUInt64" +"nothingUInt64ArgMax" +"nothingUInt64ArgMin" +"nothingUInt64Array" +"nothingUInt64Distinct" +"nothingUInt64ForEach" +"nothingUInt64If" +"nothingUInt64Map" +"nothingUInt64Merge" +"nothingUInt64Null" +"nothingUInt64OrDefault" +"nothingUInt64OrNull" +"nothingUInt64Resample" +"nothingUInt64SimpleState" +"nothingUInt64State" +"now" +"now64" +"nowInBlock" +"nth_value" +"nth_valueArgMax" +"nth_valueArgMin" +"nth_valueArray" +"nth_valueDistinct" +"nth_valueForEach" +"nth_valueIf" +"nth_valueMap" +"nth_valueMerge" +"nth_valueNull" +"nth_valueOrDefault" +"nth_valueOrNull" +"nth_valueResample" +"nth_valueSimpleState" +"nth_valueState" +"ntile" +"ntileArgMax" +"ntileArgMin" +"ntileArray" +"ntileDistinct" +"ntileForEach" +"ntileIf" +"ntileMap" +"ntileMerge" +"ntileNull" +"ntileOrDefault" +"ntileOrNull" +"ntileResample" +"ntileSimpleState" +"ntileState" +"nullIf" +"nullIn" +"nullInIgnoreSet" +"or" +"parseDateTime" +"parseDateTime32BestEffort" +"parseDateTime32BestEffortOrNull" +"parseDateTime32BestEffortOrZero" +"parseDateTime64BestEffort" +"parseDateTime64BestEffortOrNull" +"parseDateTime64BestEffortOrZero" +"parseDateTime64BestEffortUS" +"parseDateTime64BestEffortUSOrNull" +"parseDateTime64BestEffortUSOrZero" +"parseDateTimeBestEffort" +"parseDateTimeBestEffortOrNull" +"parseDateTimeBestEffortOrZero" +"parseDateTimeBestEffortUS" +"parseDateTimeBestEffortUSOrNull" +"parseDateTimeBestEffortUSOrZero" +"parseDateTimeInJodaSyntax" +"parseDateTimeInJodaSyntaxOrNull" +"parseDateTimeInJodaSyntaxOrZero" +"parseDateTimeOrNull" +"parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" +"parseTimeDelta" +"partitionID" +"partitionId" +"path" +"pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" +"pi" +"plus" +"pmod" +"pointInEllipses" +"pointInPolygon" +"polygonAreaCartesian" +"polygonAreaSpherical" +"polygonConvexHullCartesian" +"polygonPerimeterCartesian" +"polygonPerimeterSpherical" +"polygonsDistanceCartesian" +"polygonsDistanceSpherical" +"polygonsEqualsCartesian" +"polygonsIntersectionCartesian" +"polygonsIntersectionSpherical" +"polygonsSymDifferenceCartesian" +"polygonsSymDifferenceSpherical" +"polygonsUnionCartesian" +"polygonsUnionSpherical" +"polygonsWithinCartesian" +"polygonsWithinSpherical" +"port" +"portRFC" +"position" +"positionCaseInsensitive" +"positionCaseInsensitiveUTF8" +"positionUTF8" +"positiveModulo" +"positive_modulo" +"pow" +"power" +"printf" +"proportionsZTest" +"protocol" +"punycodeDecode" +"punycodeEncode" +"quantile" +"quantileArgMax" +"quantileArgMin" +"quantileArray" +"quantileBFloat16" +"quantileBFloat16ArgMax" +"quantileBFloat16ArgMin" +"quantileBFloat16Array" +"quantileBFloat16Distinct" +"quantileBFloat16ForEach" +"quantileBFloat16If" +"quantileBFloat16Map" +"quantileBFloat16Merge" +"quantileBFloat16Null" +"quantileBFloat16OrDefault" +"quantileBFloat16OrNull" +"quantileBFloat16Resample" +"quantileBFloat16SimpleState" +"quantileBFloat16State" +"quantileBFloat16Weighted" +"quantileBFloat16WeightedArgMax" +"quantileBFloat16WeightedArgMin" +"quantileBFloat16WeightedArray" +"quantileBFloat16WeightedDistinct" +"quantileBFloat16WeightedForEach" +"quantileBFloat16WeightedIf" +"quantileBFloat16WeightedMap" +"quantileBFloat16WeightedMerge" +"quantileBFloat16WeightedNull" +"quantileBFloat16WeightedOrDefault" +"quantileBFloat16WeightedOrNull" +"quantileBFloat16WeightedResample" +"quantileBFloat16WeightedSimpleState" +"quantileBFloat16WeightedState" +"quantileDD" +"quantileDDArgMax" +"quantileDDArgMin" +"quantileDDArray" +"quantileDDDistinct" +"quantileDDForEach" +"quantileDDIf" +"quantileDDMap" +"quantileDDMerge" +"quantileDDNull" +"quantileDDOrDefault" +"quantileDDOrNull" +"quantileDDResample" +"quantileDDSimpleState" +"quantileDDState" +"quantileDeterministic" +"quantileDeterministicArgMax" +"quantileDeterministicArgMin" +"quantileDeterministicArray" +"quantileDeterministicDistinct" +"quantileDeterministicForEach" +"quantileDeterministicIf" +"quantileDeterministicMap" +"quantileDeterministicMerge" +"quantileDeterministicNull" +"quantileDeterministicOrDefault" +"quantileDeterministicOrNull" +"quantileDeterministicResample" +"quantileDeterministicSimpleState" +"quantileDeterministicState" +"quantileDistinct" +"quantileExact" +"quantileExactArgMax" +"quantileExactArgMin" +"quantileExactArray" +"quantileExactDistinct" +"quantileExactExclusive" +"quantileExactExclusiveArgMax" +"quantileExactExclusiveArgMin" +"quantileExactExclusiveArray" +"quantileExactExclusiveDistinct" +"quantileExactExclusiveForEach" +"quantileExactExclusiveIf" +"quantileExactExclusiveMap" +"quantileExactExclusiveMerge" +"quantileExactExclusiveNull" +"quantileExactExclusiveOrDefault" +"quantileExactExclusiveOrNull" +"quantileExactExclusiveResample" +"quantileExactExclusiveSimpleState" +"quantileExactExclusiveState" +"quantileExactForEach" +"quantileExactHigh" +"quantileExactHighArgMax" +"quantileExactHighArgMin" +"quantileExactHighArray" +"quantileExactHighDistinct" +"quantileExactHighForEach" +"quantileExactHighIf" +"quantileExactHighMap" +"quantileExactHighMerge" +"quantileExactHighNull" +"quantileExactHighOrDefault" +"quantileExactHighOrNull" +"quantileExactHighResample" +"quantileExactHighSimpleState" +"quantileExactHighState" +"quantileExactIf" +"quantileExactInclusive" +"quantileExactInclusiveArgMax" +"quantileExactInclusiveArgMin" +"quantileExactInclusiveArray" +"quantileExactInclusiveDistinct" +"quantileExactInclusiveForEach" +"quantileExactInclusiveIf" +"quantileExactInclusiveMap" +"quantileExactInclusiveMerge" +"quantileExactInclusiveNull" +"quantileExactInclusiveOrDefault" +"quantileExactInclusiveOrNull" +"quantileExactInclusiveResample" +"quantileExactInclusiveSimpleState" +"quantileExactInclusiveState" +"quantileExactLow" +"quantileExactLowArgMax" +"quantileExactLowArgMin" +"quantileExactLowArray" +"quantileExactLowDistinct" +"quantileExactLowForEach" +"quantileExactLowIf" +"quantileExactLowMap" +"quantileExactLowMerge" +"quantileExactLowNull" +"quantileExactLowOrDefault" +"quantileExactLowOrNull" +"quantileExactLowResample" +"quantileExactLowSimpleState" +"quantileExactLowState" +"quantileExactMap" +"quantileExactMerge" +"quantileExactNull" +"quantileExactOrDefault" +"quantileExactOrNull" +"quantileExactResample" +"quantileExactSimpleState" +"quantileExactState" +"quantileExactWeighted" +"quantileExactWeightedArgMax" +"quantileExactWeightedArgMin" +"quantileExactWeightedArray" +"quantileExactWeightedDistinct" +"quantileExactWeightedForEach" +"quantileExactWeightedIf" +"quantileExactWeightedMap" +"quantileExactWeightedMerge" +"quantileExactWeightedNull" +"quantileExactWeightedOrDefault" +"quantileExactWeightedOrNull" +"quantileExactWeightedResample" +"quantileExactWeightedSimpleState" +"quantileExactWeightedState" +"quantileForEach" +"quantileGK" +"quantileGKArgMax" +"quantileGKArgMin" +"quantileGKArray" +"quantileGKDistinct" +"quantileGKForEach" +"quantileGKIf" +"quantileGKMap" +"quantileGKMerge" +"quantileGKNull" +"quantileGKOrDefault" +"quantileGKOrNull" +"quantileGKResample" +"quantileGKSimpleState" +"quantileGKState" +"quantileIf" +"quantileInterpolatedWeighted" +"quantileInterpolatedWeightedArgMax" +"quantileInterpolatedWeightedArgMin" +"quantileInterpolatedWeightedArray" +"quantileInterpolatedWeightedDistinct" +"quantileInterpolatedWeightedForEach" +"quantileInterpolatedWeightedIf" +"quantileInterpolatedWeightedMap" +"quantileInterpolatedWeightedMerge" +"quantileInterpolatedWeightedNull" +"quantileInterpolatedWeightedOrDefault" +"quantileInterpolatedWeightedOrNull" +"quantileInterpolatedWeightedResample" +"quantileInterpolatedWeightedSimpleState" +"quantileInterpolatedWeightedState" +"quantileMap" +"quantileMerge" +"quantileNull" +"quantileOrDefault" +"quantileOrNull" +"quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" +"quantiles" +"quantilesArgMax" +"quantilesArgMin" +"quantilesArray" +"quantilesBFloat16" +"quantilesBFloat16ArgMax" +"quantilesBFloat16ArgMin" +"quantilesBFloat16Array" +"quantilesBFloat16Distinct" +"quantilesBFloat16ForEach" +"quantilesBFloat16If" +"quantilesBFloat16Map" +"quantilesBFloat16Merge" +"quantilesBFloat16Null" +"quantilesBFloat16OrDefault" +"quantilesBFloat16OrNull" +"quantilesBFloat16Resample" +"quantilesBFloat16SimpleState" +"quantilesBFloat16State" +"quantilesBFloat16Weighted" +"quantilesBFloat16WeightedArgMax" +"quantilesBFloat16WeightedArgMin" +"quantilesBFloat16WeightedArray" +"quantilesBFloat16WeightedDistinct" +"quantilesBFloat16WeightedForEach" +"quantilesBFloat16WeightedIf" +"quantilesBFloat16WeightedMap" +"quantilesBFloat16WeightedMerge" +"quantilesBFloat16WeightedNull" +"quantilesBFloat16WeightedOrDefault" +"quantilesBFloat16WeightedOrNull" +"quantilesBFloat16WeightedResample" +"quantilesBFloat16WeightedSimpleState" +"quantilesBFloat16WeightedState" +"quantilesDD" +"quantilesDDArgMax" +"quantilesDDArgMin" +"quantilesDDArray" +"quantilesDDDistinct" +"quantilesDDForEach" +"quantilesDDIf" +"quantilesDDMap" +"quantilesDDMerge" +"quantilesDDNull" +"quantilesDDOrDefault" +"quantilesDDOrNull" +"quantilesDDResample" +"quantilesDDSimpleState" +"quantilesDDState" +"quantilesDeterministic" +"quantilesDeterministicArgMax" +"quantilesDeterministicArgMin" +"quantilesDeterministicArray" +"quantilesDeterministicDistinct" +"quantilesDeterministicForEach" +"quantilesDeterministicIf" +"quantilesDeterministicMap" +"quantilesDeterministicMerge" +"quantilesDeterministicNull" +"quantilesDeterministicOrDefault" +"quantilesDeterministicOrNull" +"quantilesDeterministicResample" +"quantilesDeterministicSimpleState" +"quantilesDeterministicState" +"quantilesDistinct" +"quantilesExact" +"quantilesExactArgMax" +"quantilesExactArgMin" +"quantilesExactArray" +"quantilesExactDistinct" +"quantilesExactExclusive" +"quantilesExactExclusiveArgMax" +"quantilesExactExclusiveArgMin" +"quantilesExactExclusiveArray" +"quantilesExactExclusiveDistinct" +"quantilesExactExclusiveForEach" +"quantilesExactExclusiveIf" +"quantilesExactExclusiveMap" +"quantilesExactExclusiveMerge" +"quantilesExactExclusiveNull" +"quantilesExactExclusiveOrDefault" +"quantilesExactExclusiveOrNull" +"quantilesExactExclusiveResample" +"quantilesExactExclusiveSimpleState" +"quantilesExactExclusiveState" +"quantilesExactForEach" +"quantilesExactHigh" +"quantilesExactHighArgMax" +"quantilesExactHighArgMin" +"quantilesExactHighArray" +"quantilesExactHighDistinct" +"quantilesExactHighForEach" +"quantilesExactHighIf" +"quantilesExactHighMap" +"quantilesExactHighMerge" +"quantilesExactHighNull" +"quantilesExactHighOrDefault" +"quantilesExactHighOrNull" +"quantilesExactHighResample" +"quantilesExactHighSimpleState" +"quantilesExactHighState" +"quantilesExactIf" +"quantilesExactInclusive" +"quantilesExactInclusiveArgMax" +"quantilesExactInclusiveArgMin" +"quantilesExactInclusiveArray" +"quantilesExactInclusiveDistinct" +"quantilesExactInclusiveForEach" +"quantilesExactInclusiveIf" +"quantilesExactInclusiveMap" +"quantilesExactInclusiveMerge" +"quantilesExactInclusiveNull" +"quantilesExactInclusiveOrDefault" +"quantilesExactInclusiveOrNull" +"quantilesExactInclusiveResample" +"quantilesExactInclusiveSimpleState" +"quantilesExactInclusiveState" +"quantilesExactLow" +"quantilesExactLowArgMax" +"quantilesExactLowArgMin" +"quantilesExactLowArray" +"quantilesExactLowDistinct" +"quantilesExactLowForEach" +"quantilesExactLowIf" +"quantilesExactLowMap" +"quantilesExactLowMerge" +"quantilesExactLowNull" +"quantilesExactLowOrDefault" +"quantilesExactLowOrNull" +"quantilesExactLowResample" +"quantilesExactLowSimpleState" +"quantilesExactLowState" +"quantilesExactMap" +"quantilesExactMerge" +"quantilesExactNull" +"quantilesExactOrDefault" +"quantilesExactOrNull" +"quantilesExactResample" +"quantilesExactSimpleState" +"quantilesExactState" +"quantilesExactWeighted" +"quantilesExactWeightedArgMax" +"quantilesExactWeightedArgMin" +"quantilesExactWeightedArray" +"quantilesExactWeightedDistinct" +"quantilesExactWeightedForEach" +"quantilesExactWeightedIf" +"quantilesExactWeightedMap" +"quantilesExactWeightedMerge" +"quantilesExactWeightedNull" +"quantilesExactWeightedOrDefault" +"quantilesExactWeightedOrNull" +"quantilesExactWeightedResample" +"quantilesExactWeightedSimpleState" +"quantilesExactWeightedState" +"quantilesForEach" +"quantilesGK" +"quantilesGKArgMax" +"quantilesGKArgMin" +"quantilesGKArray" +"quantilesGKDistinct" +"quantilesGKForEach" +"quantilesGKIf" +"quantilesGKMap" +"quantilesGKMerge" +"quantilesGKNull" +"quantilesGKOrDefault" +"quantilesGKOrNull" +"quantilesGKResample" +"quantilesGKSimpleState" +"quantilesGKState" +"quantilesIf" +"quantilesInterpolatedWeighted" +"quantilesInterpolatedWeightedArgMax" +"quantilesInterpolatedWeightedArgMin" +"quantilesInterpolatedWeightedArray" +"quantilesInterpolatedWeightedDistinct" +"quantilesInterpolatedWeightedForEach" +"quantilesInterpolatedWeightedIf" +"quantilesInterpolatedWeightedMap" +"quantilesInterpolatedWeightedMerge" +"quantilesInterpolatedWeightedNull" +"quantilesInterpolatedWeightedOrDefault" +"quantilesInterpolatedWeightedOrNull" +"quantilesInterpolatedWeightedResample" +"quantilesInterpolatedWeightedSimpleState" +"quantilesInterpolatedWeightedState" +"quantilesMap" +"quantilesMerge" +"quantilesNull" +"quantilesOrDefault" +"quantilesOrNull" +"quantilesResample" +"quantilesSimpleState" +"quantilesState" +"quantilesTDigest" +"quantilesTDigestArgMax" +"quantilesTDigestArgMin" +"quantilesTDigestArray" +"quantilesTDigestDistinct" +"quantilesTDigestForEach" +"quantilesTDigestIf" +"quantilesTDigestMap" +"quantilesTDigestMerge" +"quantilesTDigestNull" +"quantilesTDigestOrDefault" +"quantilesTDigestOrNull" +"quantilesTDigestResample" +"quantilesTDigestSimpleState" +"quantilesTDigestState" +"quantilesTDigestWeighted" +"quantilesTDigestWeightedArgMax" +"quantilesTDigestWeightedArgMin" +"quantilesTDigestWeightedArray" +"quantilesTDigestWeightedDistinct" +"quantilesTDigestWeightedForEach" +"quantilesTDigestWeightedIf" +"quantilesTDigestWeightedMap" +"quantilesTDigestWeightedMerge" +"quantilesTDigestWeightedNull" +"quantilesTDigestWeightedOrDefault" +"quantilesTDigestWeightedOrNull" +"quantilesTDigestWeightedResample" +"quantilesTDigestWeightedSimpleState" +"quantilesTDigestWeightedState" +"quantilesTiming" +"quantilesTimingArgMax" +"quantilesTimingArgMin" +"quantilesTimingArray" +"quantilesTimingDistinct" +"quantilesTimingForEach" +"quantilesTimingIf" +"quantilesTimingMap" +"quantilesTimingMerge" +"quantilesTimingNull" +"quantilesTimingOrDefault" +"quantilesTimingOrNull" +"quantilesTimingResample" +"quantilesTimingSimpleState" +"quantilesTimingState" +"quantilesTimingWeighted" +"quantilesTimingWeightedArgMax" +"quantilesTimingWeightedArgMin" +"quantilesTimingWeightedArray" +"quantilesTimingWeightedDistinct" +"quantilesTimingWeightedForEach" +"quantilesTimingWeightedIf" +"quantilesTimingWeightedMap" +"quantilesTimingWeightedMerge" +"quantilesTimingWeightedNull" +"quantilesTimingWeightedOrDefault" +"quantilesTimingWeightedOrNull" +"quantilesTimingWeightedResample" +"quantilesTimingWeightedSimpleState" +"quantilesTimingWeightedState" +"queryID" +"queryString" +"queryStringAndFragment" +"query_id" +"radians" +"rand" +"rand32" +"rand64" +"randBernoulli" +"randBinomial" +"randCanonical" +"randChiSquared" +"randConstant" +"randExponential" +"randFisherF" +"randLogNormal" +"randNegativeBinomial" +"randNormal" +"randPoisson" +"randStudentT" +"randUniform" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" +"range" +"rank" +"rankArgMax" +"rankArgMin" +"rankArray" +"rankCorr" +"rankCorrArgMax" +"rankCorrArgMin" +"rankCorrArray" +"rankCorrDistinct" +"rankCorrForEach" +"rankCorrIf" +"rankCorrMap" +"rankCorrMerge" +"rankCorrNull" +"rankCorrOrDefault" +"rankCorrOrNull" +"rankCorrResample" +"rankCorrSimpleState" +"rankCorrState" +"rankDistinct" +"rankForEach" +"rankIf" +"rankMap" +"rankMerge" +"rankNull" +"rankOrDefault" +"rankOrNull" +"rankResample" +"rankSimpleState" +"rankState" +"readWKTLineString" +"readWKTMultiLineString" +"readWKTMultiPolygon" +"readWKTPoint" +"readWKTPolygon" +"readWKTRing" +"regexpExtract" +"regexpQuoteMeta" +"regionHierarchy" +"regionIn" +"regionToArea" +"regionToCity" +"regionToContinent" +"regionToCountry" +"regionToDistrict" +"regionToName" +"regionToPopulation" +"regionToTopContinent" +"reinterpret" +"reinterpretAsDate" +"reinterpretAsDateTime" +"reinterpretAsFixedString" +"reinterpretAsFloat32" +"reinterpretAsFloat64" +"reinterpretAsInt128" +"reinterpretAsInt16" +"reinterpretAsInt256" +"reinterpretAsInt32" +"reinterpretAsInt64" +"reinterpretAsInt8" +"reinterpretAsString" +"reinterpretAsUInt128" +"reinterpretAsUInt16" +"reinterpretAsUInt256" +"reinterpretAsUInt32" +"reinterpretAsUInt64" +"reinterpretAsUInt8" +"reinterpretAsUUID" +"repeat" +"replace" +"replaceAll" +"replaceOne" +"replaceRegexpAll" +"replaceRegexpOne" +"replicate" +"retention" +"retentionArgMax" +"retentionArgMin" +"retentionArray" +"retentionDistinct" +"retentionForEach" +"retentionIf" +"retentionMap" +"retentionMerge" +"retentionNull" +"retentionOrDefault" +"retentionOrNull" +"retentionResample" +"retentionSimpleState" +"retentionState" +"reverse" +"reverseUTF8" +"revision" +"right" +"rightPad" +"rightPadUTF8" +"rightUTF8" +"round" +"roundAge" +"roundBankers" +"roundDown" +"roundDuration" +"roundToExp2" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"row_number" +"row_numberArgMax" +"row_numberArgMin" +"row_numberArray" +"row_numberDistinct" +"row_numberForEach" +"row_numberIf" +"row_numberMap" +"row_numberMerge" +"row_numberNull" +"row_numberOrDefault" +"row_numberOrNull" +"row_numberResample" +"row_numberSimpleState" +"row_numberState" +"rpad" +"rtrim" +"runningAccumulate" +"runningConcurrency" +"runningDifference" +"runningDifferenceStartingWithFirstValue" +"s2CapContains" +"s2CapUnion" +"s2CellsIntersect" +"s2GetNeighbors" +"s2RectAdd" +"s2RectContains" +"s2RectIntersection" +"s2RectUnion" +"s2ToGeo" +"scalarProduct" +"sequenceCount" +"sequenceCountArgMax" +"sequenceCountArgMin" +"sequenceCountArray" +"sequenceCountDistinct" +"sequenceCountForEach" +"sequenceCountIf" +"sequenceCountMap" +"sequenceCountMerge" +"sequenceCountNull" +"sequenceCountOrDefault" +"sequenceCountOrNull" +"sequenceCountResample" +"sequenceCountSimpleState" +"sequenceCountState" +"sequenceMatch" +"sequenceMatchArgMax" +"sequenceMatchArgMin" +"sequenceMatchArray" +"sequenceMatchDistinct" +"sequenceMatchForEach" +"sequenceMatchIf" +"sequenceMatchMap" +"sequenceMatchMerge" +"sequenceMatchNull" +"sequenceMatchOrDefault" +"sequenceMatchOrNull" +"sequenceMatchResample" +"sequenceMatchSimpleState" +"sequenceMatchState" +"sequenceNextNode" +"sequenceNextNodeArgMax" +"sequenceNextNodeArgMin" +"sequenceNextNodeArray" +"sequenceNextNodeDistinct" +"sequenceNextNodeForEach" +"sequenceNextNodeIf" +"sequenceNextNodeMap" +"sequenceNextNodeMerge" +"sequenceNextNodeNull" +"sequenceNextNodeOrDefault" +"sequenceNextNodeOrNull" +"sequenceNextNodeResample" +"sequenceNextNodeSimpleState" +"sequenceNextNodeState" +"seriesDecomposeSTL" +"seriesOutliersDetectTukey" +"seriesPeriodDetectFFT" +"serverTimeZone" +"serverTimezone" +"serverUUID" +"shardCount" +"shardNum" +"showCertificate" +"sigmoid" +"sign" +"simpleJSONExtractBool" +"simpleJSONExtractFloat" +"simpleJSONExtractInt" +"simpleJSONExtractRaw" +"simpleJSONExtractString" +"simpleJSONExtractUInt" +"simpleJSONHas" +"simpleLinearRegression" +"simpleLinearRegressionArgMax" +"simpleLinearRegressionArgMin" +"simpleLinearRegressionArray" +"simpleLinearRegressionDistinct" +"simpleLinearRegressionForEach" +"simpleLinearRegressionIf" +"simpleLinearRegressionMap" +"simpleLinearRegressionMerge" +"simpleLinearRegressionNull" +"simpleLinearRegressionOrDefault" +"simpleLinearRegressionOrNull" +"simpleLinearRegressionResample" +"simpleLinearRegressionSimpleState" +"simpleLinearRegressionState" +"sin" +"singleValueOrNull" +"singleValueOrNullArgMax" +"singleValueOrNullArgMin" +"singleValueOrNullArray" +"singleValueOrNullDistinct" +"singleValueOrNullForEach" +"singleValueOrNullIf" +"singleValueOrNullMap" +"singleValueOrNullMerge" +"singleValueOrNullNull" +"singleValueOrNullOrDefault" +"singleValueOrNullOrNull" +"singleValueOrNullResample" +"singleValueOrNullSimpleState" +"singleValueOrNullState" +"sinh" +"sipHash128" +"sipHash128Keyed" +"sipHash128Reference" +"sipHash128ReferenceKeyed" +"sipHash64" +"sipHash64Keyed" +"skewPop" +"skewPopArgMax" +"skewPopArgMin" +"skewPopArray" +"skewPopDistinct" +"skewPopForEach" +"skewPopIf" +"skewPopMap" +"skewPopMerge" +"skewPopNull" +"skewPopOrDefault" +"skewPopOrNull" +"skewPopResample" +"skewPopSimpleState" +"skewPopState" +"skewSamp" +"skewSampArgMax" +"skewSampArgMin" +"skewSampArray" +"skewSampDistinct" +"skewSampForEach" +"skewSampIf" +"skewSampMap" +"skewSampMerge" +"skewSampNull" +"skewSampOrDefault" +"skewSampOrNull" +"skewSampResample" +"skewSampSimpleState" +"skewSampState" +"sleep" +"sleepEachRow" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" +"snowflakeToDateTime" +"snowflakeToDateTime64" +"soundex" +"space" +"sparkBar" +"sparkBarArgMax" +"sparkBarArgMin" +"sparkBarArray" +"sparkBarDistinct" +"sparkBarForEach" +"sparkBarIf" +"sparkBarMap" +"sparkBarMerge" +"sparkBarNull" +"sparkBarOrDefault" +"sparkBarOrNull" +"sparkBarResample" +"sparkBarSimpleState" +"sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" +"sparkbarState" +"splitByAlpha" +"splitByChar" +"splitByNonAlpha" +"splitByRegexp" +"splitByString" +"splitByWhitespace" +"sqid" +"sqidDecode" +"sqidEncode" +"sqrt" +"startsWith" +"startsWithUTF8" +"stddevPop" +"stddevPopArgMax" +"stddevPopArgMin" +"stddevPopArray" +"stddevPopDistinct" +"stddevPopForEach" +"stddevPopIf" +"stddevPopMap" +"stddevPopMerge" +"stddevPopNull" +"stddevPopOrDefault" +"stddevPopOrNull" +"stddevPopResample" +"stddevPopSimpleState" +"stddevPopStable" +"stddevPopStableArgMax" +"stddevPopStableArgMin" +"stddevPopStableArray" +"stddevPopStableDistinct" +"stddevPopStableForEach" +"stddevPopStableIf" +"stddevPopStableMap" +"stddevPopStableMerge" +"stddevPopStableNull" +"stddevPopStableOrDefault" +"stddevPopStableOrNull" +"stddevPopStableResample" +"stddevPopStableSimpleState" +"stddevPopStableState" +"stddevPopState" +"stddevSamp" +"stddevSampArgMax" +"stddevSampArgMin" +"stddevSampArray" +"stddevSampDistinct" +"stddevSampForEach" +"stddevSampIf" +"stddevSampMap" +"stddevSampMerge" +"stddevSampNull" +"stddevSampOrDefault" +"stddevSampOrNull" +"stddevSampResample" +"stddevSampSimpleState" +"stddevSampStable" +"stddevSampStableArgMax" +"stddevSampStableArgMin" +"stddevSampStableArray" +"stddevSampStableDistinct" +"stddevSampStableForEach" +"stddevSampStableIf" +"stddevSampStableMap" +"stddevSampStableMerge" +"stddevSampStableNull" +"stddevSampStableOrDefault" +"stddevSampStableOrNull" +"stddevSampStableResample" +"stddevSampStableSimpleState" +"stddevSampStableState" +"stddevSampState" +"stem" +"stochasticLinearRegression" +"stochasticLinearRegressionArgMax" +"stochasticLinearRegressionArgMin" +"stochasticLinearRegressionArray" +"stochasticLinearRegressionDistinct" +"stochasticLinearRegressionForEach" +"stochasticLinearRegressionIf" +"stochasticLinearRegressionMap" +"stochasticLinearRegressionMerge" +"stochasticLinearRegressionNull" +"stochasticLinearRegressionOrDefault" +"stochasticLinearRegressionOrNull" +"stochasticLinearRegressionResample" +"stochasticLinearRegressionSimpleState" +"stochasticLinearRegressionState" +"stochasticLogisticRegression" +"stochasticLogisticRegressionArgMax" +"stochasticLogisticRegressionArgMin" +"stochasticLogisticRegressionArray" +"stochasticLogisticRegressionDistinct" +"stochasticLogisticRegressionForEach" +"stochasticLogisticRegressionIf" +"stochasticLogisticRegressionMap" +"stochasticLogisticRegressionMerge" +"stochasticLogisticRegressionNull" +"stochasticLogisticRegressionOrDefault" +"stochasticLogisticRegressionOrNull" +"stochasticLogisticRegressionResample" +"stochasticLogisticRegressionSimpleState" +"stochasticLogisticRegressionState" +"str_to_date" +"str_to_map" +"stringJaccardIndex" +"stringJaccardIndexUTF8" +"stringToH3" +"structureToCapnProtoSchema" +"structureToProtobufSchema" +"studentTTest" +"studentTTestArgMax" +"studentTTestArgMin" +"studentTTestArray" +"studentTTestDistinct" +"studentTTestForEach" +"studentTTestIf" +"studentTTestMap" +"studentTTestMerge" +"studentTTestNull" +"studentTTestOrDefault" +"studentTTestOrNull" +"studentTTestResample" +"studentTTestSimpleState" +"studentTTestState" +"subBitmap" +"subDate" +"substr" +"substring" +"substringIndex" +"substringIndexUTF8" +"substringUTF8" +"subtractDays" +"subtractHours" +"subtractInterval" +"subtractMicroseconds" +"subtractMilliseconds" +"subtractMinutes" +"subtractMonths" +"subtractNanoseconds" +"subtractQuarters" +"subtractSeconds" +"subtractTupleOfIntervals" +"subtractWeeks" +"subtractYears" +"sum" +"sumArgMax" +"sumArgMin" +"sumArray" +"sumCount" +"sumCountArgMax" +"sumCountArgMin" +"sumCountArray" +"sumCountDistinct" +"sumCountForEach" +"sumCountIf" +"sumCountMap" +"sumCountMerge" +"sumCountNull" +"sumCountOrDefault" +"sumCountOrNull" +"sumCountResample" +"sumCountSimpleState" +"sumCountState" +"sumDistinct" +"sumForEach" +"sumIf" +"sumKahan" +"sumKahanArgMax" +"sumKahanArgMin" +"sumKahanArray" +"sumKahanDistinct" +"sumKahanForEach" +"sumKahanIf" +"sumKahanMap" +"sumKahanMerge" +"sumKahanNull" +"sumKahanOrDefault" +"sumKahanOrNull" +"sumKahanResample" +"sumKahanSimpleState" +"sumKahanState" +"sumMap" +"sumMapFiltered" +"sumMapFilteredArgMax" +"sumMapFilteredArgMin" +"sumMapFilteredArray" +"sumMapFilteredDistinct" +"sumMapFilteredForEach" +"sumMapFilteredIf" +"sumMapFilteredMap" +"sumMapFilteredMerge" +"sumMapFilteredNull" +"sumMapFilteredOrDefault" +"sumMapFilteredOrNull" +"sumMapFilteredResample" +"sumMapFilteredSimpleState" +"sumMapFilteredState" +"sumMapFilteredWithOverflow" +"sumMapFilteredWithOverflowArgMax" +"sumMapFilteredWithOverflowArgMin" +"sumMapFilteredWithOverflowArray" +"sumMapFilteredWithOverflowDistinct" +"sumMapFilteredWithOverflowForEach" +"sumMapFilteredWithOverflowIf" +"sumMapFilteredWithOverflowMap" +"sumMapFilteredWithOverflowMerge" +"sumMapFilteredWithOverflowNull" +"sumMapFilteredWithOverflowOrDefault" +"sumMapFilteredWithOverflowOrNull" +"sumMapFilteredWithOverflowResample" +"sumMapFilteredWithOverflowSimpleState" +"sumMapFilteredWithOverflowState" +"sumMapWithOverflow" +"sumMapWithOverflowArgMax" +"sumMapWithOverflowArgMin" +"sumMapWithOverflowArray" +"sumMapWithOverflowDistinct" +"sumMapWithOverflowForEach" +"sumMapWithOverflowIf" +"sumMapWithOverflowMap" +"sumMapWithOverflowMerge" +"sumMapWithOverflowNull" +"sumMapWithOverflowOrDefault" +"sumMapWithOverflowOrNull" +"sumMapWithOverflowResample" +"sumMapWithOverflowSimpleState" +"sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" +"sumMerge" +"sumNull" +"sumOrDefault" +"sumOrNull" +"sumResample" +"sumSimpleState" +"sumState" +"sumWithOverflow" +"sumWithOverflowArgMax" +"sumWithOverflowArgMin" +"sumWithOverflowArray" +"sumWithOverflowDistinct" +"sumWithOverflowForEach" +"sumWithOverflowIf" +"sumWithOverflowMap" +"sumWithOverflowMerge" +"sumWithOverflowNull" +"sumWithOverflowOrDefault" +"sumWithOverflowOrNull" +"sumWithOverflowResample" +"sumWithOverflowSimpleState" +"sumWithOverflowState" +"svg" +"synonyms" +"tan" +"tanh" +"tcpPort" +"tgamma" +"theilsU" +"theilsUArgMax" +"theilsUArgMin" +"theilsUArray" +"theilsUDistinct" +"theilsUForEach" +"theilsUIf" +"theilsUMap" +"theilsUMerge" +"theilsUNull" +"theilsUOrDefault" +"theilsUOrNull" +"theilsUResample" +"theilsUSimpleState" +"theilsUState" +"throwIf" +"tid" +"timeDiff" +"timeSlot" +"timeSlots" +"timeZone" +"timeZoneOf" +"timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" +"timezoneOffset" +"toBool" +"toColumnTypeName" +"toDate" +"toDate32" +"toDate32OrDefault" +"toDate32OrNull" +"toDate32OrZero" +"toDateOrDefault" +"toDateOrNull" +"toDateOrZero" +"toDateTime" +"toDateTime32" +"toDateTime64" +"toDateTime64OrDefault" +"toDateTime64OrNull" +"toDateTime64OrZero" +"toDateTimeOrDefault" +"toDateTimeOrNull" +"toDateTimeOrZero" +"toDayOfMonth" +"toDayOfWeek" +"toDayOfYear" +"toDaysSinceYearZero" +"toDecimal128" +"toDecimal128OrDefault" +"toDecimal128OrNull" +"toDecimal128OrZero" +"toDecimal256" +"toDecimal256OrDefault" +"toDecimal256OrNull" +"toDecimal256OrZero" +"toDecimal32" +"toDecimal32OrDefault" +"toDecimal32OrNull" +"toDecimal32OrZero" +"toDecimal64" +"toDecimal64OrDefault" +"toDecimal64OrNull" +"toDecimal64OrZero" +"toDecimalString" +"toFixedString" +"toFloat32" +"toFloat32OrDefault" +"toFloat32OrNull" +"toFloat32OrZero" +"toFloat64" +"toFloat64OrDefault" +"toFloat64OrNull" +"toFloat64OrZero" +"toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" +"toInt128" +"toInt128OrDefault" +"toInt128OrNull" +"toInt128OrZero" +"toInt16" +"toInt16OrDefault" +"toInt16OrNull" +"toInt16OrZero" +"toInt256" +"toInt256OrDefault" +"toInt256OrNull" +"toInt256OrZero" +"toInt32" +"toInt32OrDefault" +"toInt32OrNull" +"toInt32OrZero" +"toInt64" +"toInt64OrDefault" +"toInt64OrNull" +"toInt64OrZero" +"toInt8" +"toInt8OrDefault" +"toInt8OrNull" +"toInt8OrZero" +"toIntervalDay" +"toIntervalHour" +"toIntervalMicrosecond" +"toIntervalMillisecond" +"toIntervalMinute" +"toIntervalMonth" +"toIntervalNanosecond" +"toIntervalQuarter" +"toIntervalSecond" +"toIntervalWeek" +"toIntervalYear" +"toJSONString" +"toLastDayOfMonth" +"toLastDayOfWeek" +"toLowCardinality" +"toMillisecond" +"toMinute" +"toModifiedJulianDay" +"toModifiedJulianDayOrNull" +"toMonday" +"toMonth" +"toNullable" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" +"topK" +"topKArgMax" +"topKArgMin" +"topKArray" +"topKDistinct" +"topKForEach" +"topKIf" +"topKMap" +"topKMerge" +"topKNull" +"topKOrDefault" +"topKOrNull" +"topKResample" +"topKSimpleState" +"topKState" +"topKWeighted" +"topKWeightedArgMax" +"topKWeightedArgMin" +"topKWeightedArray" +"topKWeightedDistinct" +"topKWeightedForEach" +"topKWeightedIf" +"topKWeightedMap" +"topKWeightedMerge" +"topKWeightedNull" +"topKWeightedOrDefault" +"topKWeightedOrNull" +"topKWeightedResample" +"topKWeightedSimpleState" +"topKWeightedState" +"topLevelDomain" +"topLevelDomainRFC" +"transactionID" +"transactionLatestSnapshot" +"transactionOldestSnapshot" +"transform" +"translate" +"translateUTF8" +"trim" +"trimBoth" +"trimLeft" +"trimRight" +"trunc" +"truncate" +"tryBase58Decode" +"tryBase64Decode" +"tryBase64URLDecode" +"tryDecrypt" +"tryIdnaEncode" +"tryPunycodeDecode" +"tumble" +"tumbleEnd" +"tumbleStart" +"tuple" +"tupleConcat" +"tupleDivide" +"tupleDivideByNumber" +"tupleElement" +"tupleHammingDistance" +"tupleIntDiv" +"tupleIntDivByNumber" +"tupleIntDivOrZero" +"tupleIntDivOrZeroByNumber" +"tupleMinus" +"tupleModulo" +"tupleModuloByNumber" +"tupleMultiply" +"tupleMultiplyByNumber" +"tupleNames" +"tupleNegate" +"tuplePlus" +"tupleToNameValuePairs" +"ucase" +"unbin" +"unhex" +"uniq" +"uniqArgMax" +"uniqArgMin" +"uniqArray" +"uniqCombined" +"uniqCombined64" +"uniqCombined64ArgMax" +"uniqCombined64ArgMin" +"uniqCombined64Array" +"uniqCombined64Distinct" +"uniqCombined64ForEach" +"uniqCombined64If" +"uniqCombined64Map" +"uniqCombined64Merge" +"uniqCombined64Null" +"uniqCombined64OrDefault" +"uniqCombined64OrNull" +"uniqCombined64Resample" +"uniqCombined64SimpleState" +"uniqCombined64State" +"uniqCombinedArgMax" +"uniqCombinedArgMin" +"uniqCombinedArray" +"uniqCombinedDistinct" +"uniqCombinedForEach" +"uniqCombinedIf" +"uniqCombinedMap" +"uniqCombinedMerge" +"uniqCombinedNull" +"uniqCombinedOrDefault" +"uniqCombinedOrNull" +"uniqCombinedResample" +"uniqCombinedSimpleState" +"uniqCombinedState" +"uniqDistinct" +"uniqExact" +"uniqExactArgMax" +"uniqExactArgMin" +"uniqExactArray" +"uniqExactDistinct" +"uniqExactForEach" +"uniqExactIf" +"uniqExactMap" +"uniqExactMerge" +"uniqExactNull" +"uniqExactOrDefault" +"uniqExactOrNull" +"uniqExactResample" +"uniqExactSimpleState" +"uniqExactState" +"uniqForEach" +"uniqHLL12" +"uniqHLL12ArgMax" +"uniqHLL12ArgMin" +"uniqHLL12Array" +"uniqHLL12Distinct" +"uniqHLL12ForEach" +"uniqHLL12If" +"uniqHLL12Map" +"uniqHLL12Merge" +"uniqHLL12Null" +"uniqHLL12OrDefault" +"uniqHLL12OrNull" +"uniqHLL12Resample" +"uniqHLL12SimpleState" +"uniqHLL12State" +"uniqIf" +"uniqMap" +"uniqMerge" +"uniqNull" +"uniqOrDefault" +"uniqOrNull" +"uniqResample" +"uniqSimpleState" +"uniqState" +"uniqTheta" +"uniqThetaArgMax" +"uniqThetaArgMin" +"uniqThetaArray" +"uniqThetaDistinct" +"uniqThetaForEach" +"uniqThetaIf" +"uniqThetaIntersect" +"uniqThetaMap" +"uniqThetaMerge" +"uniqThetaNot" +"uniqThetaNull" +"uniqThetaOrDefault" +"uniqThetaOrNull" +"uniqThetaResample" +"uniqThetaSimpleState" +"uniqThetaState" +"uniqThetaUnion" +"uniqUpTo" +"uniqUpToArgMax" +"uniqUpToArgMin" +"uniqUpToArray" +"uniqUpToDistinct" +"uniqUpToForEach" +"uniqUpToIf" +"uniqUpToMap" +"uniqUpToMerge" +"uniqUpToNull" +"uniqUpToOrDefault" +"uniqUpToOrNull" +"uniqUpToResample" +"uniqUpToSimpleState" +"uniqUpToState" +"upper" +"upperUTF8" +"uptime" +"user" +"validateNestedArraySizes" +"varPop" +"varPopArgMax" +"varPopArgMin" +"varPopArray" +"varPopDistinct" +"varPopForEach" +"varPopIf" +"varPopMap" +"varPopMerge" +"varPopNull" +"varPopOrDefault" +"varPopOrNull" +"varPopResample" +"varPopSimpleState" +"varPopStable" +"varPopStableArgMax" +"varPopStableArgMin" +"varPopStableArray" +"varPopStableDistinct" +"varPopStableForEach" +"varPopStableIf" +"varPopStableMap" +"varPopStableMerge" +"varPopStableNull" +"varPopStableOrDefault" +"varPopStableOrNull" +"varPopStableResample" +"varPopStableSimpleState" +"varPopStableState" +"varPopState" +"varSamp" +"varSampArgMax" +"varSampArgMin" +"varSampArray" +"varSampDistinct" +"varSampForEach" +"varSampIf" +"varSampMap" +"varSampMerge" +"varSampNull" +"varSampOrDefault" +"varSampOrNull" +"varSampResample" +"varSampSimpleState" +"varSampStable" +"varSampStableArgMax" +"varSampStableArgMin" +"varSampStableArray" +"varSampStableDistinct" +"varSampStableForEach" +"varSampStableIf" +"varSampStableMap" +"varSampStableMerge" +"varSampStableNull" +"varSampStableOrDefault" +"varSampStableOrNull" +"varSampStableResample" +"varSampStableSimpleState" +"varSampStableState" +"varSampState" +"variantElement" +"variantType" +"vectorDifference" +"vectorSum" +"version" +"visibleWidth" +"visitParamExtractBool" +"visitParamExtractFloat" +"visitParamExtractInt" +"visitParamExtractRaw" +"visitParamExtractString" +"visitParamExtractUInt" +"visitParamHas" +"week" +"welchTTest" +"welchTTestArgMax" +"welchTTestArgMin" +"welchTTestArray" +"welchTTestDistinct" +"welchTTestForEach" +"welchTTestIf" +"welchTTestMap" +"welchTTestMerge" +"welchTTestNull" +"welchTTestOrDefault" +"welchTTestOrNull" +"welchTTestResample" +"welchTTestSimpleState" +"welchTTestState" +"widthBucket" +"width_bucket" +"windowFunnel" +"windowFunnelArgMax" +"windowFunnelArgMin" +"windowFunnelArray" +"windowFunnelDistinct" +"windowFunnelForEach" +"windowFunnelIf" +"windowFunnelMap" +"windowFunnelMerge" +"windowFunnelNull" +"windowFunnelOrDefault" +"windowFunnelOrNull" +"windowFunnelResample" +"windowFunnelSimpleState" +"windowFunnelState" +"windowID" +"wkt" +"wordShingleMinHash" +"wordShingleMinHashArg" +"wordShingleMinHashArgCaseInsensitive" +"wordShingleMinHashArgCaseInsensitiveUTF8" +"wordShingleMinHashArgUTF8" +"wordShingleMinHashCaseInsensitive" +"wordShingleMinHashCaseInsensitiveUTF8" +"wordShingleMinHashUTF8" +"wordShingleSimHash" +"wordShingleSimHashCaseInsensitive" +"wordShingleSimHashCaseInsensitiveUTF8" +"wordShingleSimHashUTF8" +"wyHash64" +"xor" +"xxHash32" +"xxHash64" +"xxh3" +"yandexConsistentHash" +"yearweek" +"yesterday" +"zookeeperSessionUptime" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 6f2a88c22fa..e562595fb67 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -126,6 +126,7 @@ "JSONHas" "JSONKey" "JSONLength" +"JSONMergePatch" "JSONType" "JSON_ARRAY_LENGTH" "JSON_EXISTS" @@ -227,6 +228,8 @@ "UTC_timestamp" "UUIDNumToString" "UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" "VAR_POP" "VAR_POPArgMax" "VAR_POPArgMin" @@ -263,6 +266,7 @@ "YYYYMMDDhhmmssToDateTime" "YYYYMMDDhhmmssToDateTime64" "_CAST" +"__actionName" "__bitBoolMaskAnd" "__bitBoolMaskOr" "__bitSwapLastTwo" @@ -660,6 +664,8 @@ "base58Encode" "base64Decode" "base64Encode" +"base64URLDecode" +"base64URLEncode" "basename" "bin" "bitAnd" @@ -744,8 +750,15 @@ "cbrt" "ceil" "ceiling" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" "char" "cityHash64" +"clamp" "coalesce" "concat" "concatAssumeInjective" @@ -970,6 +983,7 @@ "current_date" "current_schemas" "current_timestamp" +"current_user" "cutFragment" "cutIPv6" "cutQueryString" @@ -988,7 +1002,9 @@ "dateDiff" "dateName" "dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" "dateTimeToSnowflake" +"dateTimeToSnowflakeID" "dateTrunc" "date_diff" "decodeHTMLComponent" @@ -1032,6 +1048,21 @@ "deltaSumTimestampSimpleState" "deltaSumTimestampState" "demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" "dense_rank" "dense_rankArgMax" "dense_rankArgMin" @@ -1108,8 +1139,11 @@ "domainWithoutWWWRFC" "dotProduct" "dumpColumnStructure" +"dynamicElement" +"dynamicType" "e" "editDistance" +"editDistanceUTF8" "empty" "emptyArrayDate" "emptyArrayDateTime" @@ -1334,14 +1368,17 @@ "gccMurmurHash" "gcd" "generateRandomStructure" +"generateSnowflakeID" "generateULID" "generateUUIDv4" +"generateUUIDv7" "geoDistance" "geoToH3" "geoToS2" "geohashDecode" "geohashEncode" "geohashesInBox" +"getClientHTTPHeader" "getMacro" "getOSKernelVersion" "getServerPort" @@ -1589,6 +1626,20 @@ "groupBitmapXorSimpleState" "groupBitmapXorState" "groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" "groupUniqArray" "groupUniqArrayArgMax" "groupUniqArrayArgMin" @@ -1604,6 +1655,21 @@ "groupUniqArrayResample" "groupUniqArraySimpleState" "groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" "h3CellAreaM2" "h3CellAreaRads2" "h3Distance" @@ -1660,6 +1726,8 @@ "hasTokenCaseInsensitiveOrNull" "hasTokenOrNull" "hex" +"hilbertDecode" +"hilbertEncode" "histogram" "histogramArgMax" "histogramArgMin" @@ -1881,6 +1949,7 @@ "less" "lessOrEquals" "levenshteinDistance" +"levenshteinDistanceUTF8" "lgamma" "like" "ln" @@ -2498,10 +2567,44 @@ "parseDateTimeInJodaSyntaxOrZero" "parseDateTimeOrNull" "parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" "parseTimeDelta" +"partitionID" "partitionId" "path" "pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" "pi" "plus" "pmod" @@ -2533,6 +2636,7 @@ "positive_modulo" "pow" "power" +"printf" "proportionsZTest" "protocol" "punycodeDecode" @@ -3103,6 +3207,8 @@ "rankResample" "rankSimpleState" "rankState" +"readWKTLineString" +"readWKTMultiLineString" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" @@ -3340,6 +3446,8 @@ "skewSampState" "sleep" "sleepEachRow" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" "snowflakeToDateTime" "snowflakeToDateTime64" "soundex" @@ -3902,6 +4010,7 @@ "truncate" "tryBase58Decode" "tryBase64Decode" +"tryBase64URLDecode" "tryDecrypt" "tryIdnaEncode" "tryPunycodeDecode" @@ -3923,6 +4032,7 @@ "tupleModuloByNumber" "tupleMultiply" "tupleMultiplyByNumber" +"tupleNames" "tupleNegate" "tuplePlus" "tupleToNameValuePairs" diff --git a/tests/fuzz/dictionaries/key_words.dict b/tests/fuzz/dictionaries/keywords.dict similarity index 95% rename from tests/fuzz/dictionaries/key_words.dict rename to tests/fuzz/dictionaries/keywords.dict index db517a2382c..abaaf9e53b5 100644 --- a/tests/fuzz/dictionaries/key_words.dict +++ b/tests/fuzz/dictionaries/keywords.dict @@ -3,7 +3,7 @@ "ADD CONSTRAINT" "ADD INDEX" "ADD PROJECTION" -"ADD STATISTIC" +"ADD STATISTICS" "ADMIN OPTION FOR" "AFTER" "ALGORITHM" @@ -76,7 +76,7 @@ "CLEAR COLUMN" "CLEAR INDEX" "CLEAR PROJECTION" -"CLEAR STATISTIC" +"CLEAR STATISTICS" "CLUSTER" "CLUSTERS" "CN" @@ -110,6 +110,8 @@ "CURRENTUSER" "CURRENT_USER" "D" +"DATA" +"DATA INNER UUID" "DATABASE" "DATABASES" "DATE" @@ -147,7 +149,7 @@ "DROP PART" "DROP PARTITION" "DROP PROJECTION" -"DROP STATISTIC" +"DROP STATISTICS" "DROP TABLE" "DROP TEMPORARY TABLE" "ELSE" @@ -247,6 +249,7 @@ "IS NULL" "IS_OBJECT_ID" "JOIN" +"JWT" "KERBEROS" "KEY" "KEY BY" @@ -277,13 +280,15 @@ "MATERIALIZE COLUMN" "MATERIALIZE INDEX" "MATERIALIZE PROJECTION" -"MATERIALIZE STATISTIC" +"MATERIALIZE STATISTICS" "MATERIALIZE TTL" "MATERIALIZED" "MAX" "MCS" "MEMORY" "MERGES" +"METRICS" +"METRICS INNER UUID" "MI" "MICROSECOND" "MICROSECONDS" @@ -297,12 +302,14 @@ "MODIFY" "MODIFY COLUMN" "MODIFY COMMENT" +"MODIFY DEFINER" "MODIFY ORDER BY" "MODIFY QUERY" "MODIFY REFRESH" "MODIFY SAMPLE BY" "MODIFY SETTING" "MODIFY SQL SECURITY" +"MODIFY STATISTICS" "MODIFY TTL" "MONTH" "MONTHS" @@ -373,6 +380,7 @@ "Protobuf" "Q" "QQ" +"QUALIFY" "QUARTER" "QUARTERS" "QUERY" @@ -384,6 +392,7 @@ "READONLY" "REALM" "RECOMPRESS" +"RECURSIVE" "REFERENCES" "REFRESH" "REGEXP" @@ -415,6 +424,7 @@ "SALT" "SAMPLE" "SAMPLE BY" +"SAN" "SCHEME" "SECOND" "SECONDS" @@ -460,7 +470,8 @@ "SS" "SSH_KEY" "SSL_CERTIFICATE" -"STATISTIC" +"START TRANSACTION" +"STATISTICS" "STEP" "STORAGE" "STRICT" @@ -475,6 +486,8 @@ "TABLE" "TABLE OVERRIDE" "TABLES" +"TAGS" +"TAGS INNER UUID" "TEMPORARY" "TEMPORARY TABLE" "TEST" @@ -529,6 +542,7 @@ "WITH NAME" "WITH REPLACE OPTION" "WITH TIES" +"WITH_ITEMINDEX" "WK" "WRITABLE" "WW" @@ -540,4 +554,3 @@ "bagexpansion" "base_backup" "cluster_host_ids" -"with_itemindex" From 710cf1a223c7b6d3755316beb1904ef841f2e51e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 13:55:57 +0000 Subject: [PATCH 0882/1722] Calculate statistics for paths in shared data --- src/Columns/ColumnObject.cpp | 86 ++++++++++--- src/Columns/ColumnObject.h | 39 ++++-- src/DataTypes/DataTypesBinaryEncoding.h | 106 +++++++-------- .../Serializations/SerializationObject.cpp | 121 ++++++++++++++++-- .../Serializations/SerializationObject.h | 3 +- src/Interpreters/Squashing.cpp | 1 - ...9_json_type_horizontal_merges.reference.j2 | 40 ++++++ .../03209_json_type_horizontal_merges.sql.j2 | 15 +++ ...209_json_type_vertical_merges.reference.j2 | 40 ++++++ .../03209_json_type_vertical_merges.sql.j2 | 15 +++ .../03222_json_squashing.reference | 102 +++++++++++++++ .../0_stateless/03222_json_squashing.sql | 82 ++++++++++++ 12 files changed, 555 insertions(+), 95 deletions(-) create mode 100644 tests/queries/0_stateless/03222_json_squashing.reference create mode 100644 tests/queries/0_stateless/03222_json_squashing.sql diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 852327cfc17..0645c10340d 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -38,10 +38,12 @@ ColumnObject::ColumnObject( std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, + size_t global_max_dynamic_paths_, size_t max_dynamic_types_, - const Statistics & statistics_) + const StatisticsPtr & statistics_) : shared_data(std::move(shared_data_)) , max_dynamic_paths(max_dynamic_paths_) + , global_max_dynamic_paths(global_max_dynamic_paths_) , max_dynamic_types(max_dynamic_types_) , statistics(statistics_) { @@ -60,7 +62,7 @@ ColumnObject::ColumnObject( ColumnObject::ColumnObject( std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_) - : max_dynamic_paths(max_dynamic_paths_), max_dynamic_types(max_dynamic_types_) + : max_dynamic_paths(max_dynamic_paths_), global_max_dynamic_paths(max_dynamic_paths_), max_dynamic_types(max_dynamic_types_) { typed_paths.reserve(typed_paths_.size()); for (auto & [path, column] : typed_paths_) @@ -81,8 +83,9 @@ ColumnObject::Ptr ColumnObject::create( const std::unordered_map & dynamic_paths_, const ColumnPtr & shared_data_, size_t max_dynamic_paths_, + size_t global_max_dynamic_paths_, size_t max_dynamic_types_, - const ColumnObject::Statistics & statistics_) + const ColumnObject::StatisticsPtr & statistics_) { std::unordered_map mutable_typed_paths; mutable_typed_paths.reserve(typed_paths_.size()); @@ -99,6 +102,7 @@ ColumnObject::Ptr ColumnObject::create( std::move(mutable_dynamic_paths), shared_data_->assumeMutable(), max_dynamic_paths_, + global_max_dynamic_paths_, max_dynamic_types_, statistics_); } @@ -108,10 +112,11 @@ ColumnObject::MutablePtr ColumnObject::create( std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, + size_t global_max_dynamic_paths_, size_t max_dynamic_types_, - const ColumnObject::Statistics & statistics_) + const ColumnObject::StatisticsPtr & statistics_) { - return Base::create(std::move(typed_paths_), std::move(dynamic_paths_), std::move(shared_data_), max_dynamic_paths_, max_dynamic_types_, statistics_); + return Base::create(std::move(typed_paths_), std::move(dynamic_paths_), std::move(shared_data_), max_dynamic_paths_, global_max_dynamic_paths_, max_dynamic_types_, statistics_); } ColumnObject::MutablePtr ColumnObject::create(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_) @@ -153,6 +158,7 @@ MutableColumnPtr ColumnObject::cloneEmpty() const std::move(empty_dynamic_paths), shared_data->cloneEmpty(), max_dynamic_paths, + global_max_dynamic_paths, max_dynamic_types, statistics); } @@ -174,6 +180,7 @@ MutableColumnPtr ColumnObject::cloneResized(size_t size) const std::move(resized_dynamic_paths), shared_data->cloneResized(size), max_dynamic_paths, + global_max_dynamic_paths, max_dynamic_types, statistics); } @@ -263,6 +270,14 @@ void ColumnObject::addNewDynamicPath(const std::string_view path) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add new dynamic path as the limit ({}) on dynamic paths is reached", max_dynamic_paths); } +void ColumnObject::setMaxDynamicPaths(size_t max_dynamic_paths_) +{ + if (!empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Setting specific max_dynamic_paths parameter is allowed only for empty object column"); + + max_dynamic_paths = max_dynamic_paths_; +} + void ColumnObject::setDynamicPaths(const std::vector & paths) { if (paths.size() > max_dynamic_paths) @@ -836,7 +851,7 @@ ColumnPtr ColumnObject::filter(const Filter & filt, ssize_t result_size_hint) co filtered_dynamic_paths[path] = column->filter(filt, result_size_hint); auto filtered_shared_data = shared_data->filter(filt, result_size_hint); - return ColumnObject::create(filtered_typed_paths, filtered_dynamic_paths, filtered_shared_data, max_dynamic_paths, max_dynamic_types); + return ColumnObject::create(filtered_typed_paths, filtered_dynamic_paths, filtered_shared_data, max_dynamic_paths, global_max_dynamic_paths, max_dynamic_types); } void ColumnObject::expand(const Filter & mask, bool inverted) @@ -861,7 +876,7 @@ ColumnPtr ColumnObject::permute(const Permutation & perm, size_t limit) const permuted_dynamic_paths[path] = column->permute(perm, limit); auto permuted_shared_data = shared_data->permute(perm, limit); - return ColumnObject::create(permuted_typed_paths, permuted_dynamic_paths, permuted_shared_data, max_dynamic_paths, max_dynamic_types); + return ColumnObject::create(permuted_typed_paths, permuted_dynamic_paths, permuted_shared_data, max_dynamic_paths, global_max_dynamic_paths, max_dynamic_types); } ColumnPtr ColumnObject::index(const IColumn & indexes, size_t limit) const @@ -877,7 +892,7 @@ ColumnPtr ColumnObject::index(const IColumn & indexes, size_t limit) const indexed_dynamic_paths[path] = column->index(indexes, limit); auto indexed_shared_data = shared_data->index(indexes, limit); - return ColumnObject::create(indexed_typed_paths, indexed_dynamic_paths, indexed_shared_data, max_dynamic_paths, max_dynamic_types); + return ColumnObject::create(indexed_typed_paths, indexed_dynamic_paths, indexed_shared_data, max_dynamic_paths, global_max_dynamic_paths, max_dynamic_types); } ColumnPtr ColumnObject::replicate(const Offsets & replicate_offsets) const @@ -893,7 +908,7 @@ ColumnPtr ColumnObject::replicate(const Offsets & replicate_offsets) const replicated_dynamic_paths[path] = column->replicate(replicate_offsets); auto replicated_shared_data = shared_data->replicate(replicate_offsets); - return ColumnObject::create(replicated_typed_paths, replicated_dynamic_paths, replicated_shared_data, max_dynamic_paths, max_dynamic_types); + return ColumnObject::create(replicated_typed_paths, replicated_dynamic_paths, replicated_shared_data, max_dynamic_paths, global_max_dynamic_paths, max_dynamic_types); } MutableColumns ColumnObject::scatter(ColumnIndex num_columns, const Selector & selector) const @@ -924,7 +939,7 @@ MutableColumns ColumnObject::scatter(ColumnIndex num_columns, const Selector & s MutableColumns result_columns; result_columns.reserve(num_columns); for (size_t i = 0; i != num_columns; ++i) - result_columns.emplace_back(ColumnObject::create(std::move(scattered_typed_paths[i]), std::move(scattered_dynamic_paths[i]), std::move(scattered_shared_data_columns[i]), max_dynamic_paths, max_dynamic_types)); + result_columns.emplace_back(ColumnObject::create(std::move(scattered_typed_paths[i]), std::move(scattered_dynamic_paths[i]), std::move(scattered_shared_data_columns[i]), max_dynamic_paths, global_max_dynamic_paths, max_dynamic_types)); return result_columns; } @@ -1075,6 +1090,7 @@ ColumnPtr ColumnObject::compress() const my_compressed_dynamic_paths = std::move(compressed_dynamic_paths), my_compressed_shared_data = std::move(compressed_shared_data), my_max_dynamic_paths = max_dynamic_paths, + my_global_max_dynamic_paths = global_max_dynamic_paths, my_max_dynamic_types = max_dynamic_types, my_statistics = statistics]() mutable { @@ -1089,7 +1105,7 @@ ColumnPtr ColumnObject::compress() const decompressed_dynamic_paths[path] = column->decompress(); auto decompressed_shared_data = my_compressed_shared_data->decompress(); - return ColumnObject::create(decompressed_typed_paths, decompressed_dynamic_paths, decompressed_shared_data, my_max_dynamic_paths, my_max_dynamic_types, my_statistics); + return ColumnObject::create(decompressed_typed_paths, decompressed_dynamic_paths, decompressed_shared_data, my_max_dynamic_paths, my_global_max_dynamic_paths, my_max_dynamic_types, my_statistics); }; return ColumnCompressed::create(size(), byte_size, decompress); @@ -1268,14 +1284,35 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou auto it = path_to_total_number_of_non_null_values.find(path); if (it == path_to_total_number_of_non_null_values.end()) it = path_to_total_number_of_non_null_values.emplace(path, 0).first; - auto statistics_it = source_statistics.data.find(path); - size_t size = statistics_it == source_statistics.data.end() ? (column_ptr->size() - column_ptr->getNumberOfDefaultRows()) : statistics_it->second; + size_t size = column_ptr->size() - column_ptr->getNumberOfDefaultRows(); + if (source_statistics) + { + auto statistics_it = source_statistics->dynamic_paths_statistics.find(path); + if (statistics_it != source_statistics->dynamic_paths_statistics.end()) + size = statistics_it->second; + } it->second += size; } + + /// Add paths from shared data statistics. It can helo extracting frequent paths + /// from shared data to dynamic paths. + if (source_statistics) + { + for (const auto & [path, size] : source_statistics->shared_data_paths_statistics) + { + auto it = path_to_total_number_of_non_null_values.find(path); + if (it == path_to_total_number_of_non_null_values.end()) + it = path_to_total_number_of_non_null_values.emplace(path, 0).first; + it->second += size; + } + } } + /// Reset current state. dynamic_paths.clear(); dynamic_paths_ptrs.clear(); + max_dynamic_paths = global_max_dynamic_paths; + Statistics new_statistics(Statistics::Source::MERGE); /// Check if the number of all dynamic paths exceeds the limit. if (path_to_total_number_of_non_null_values.size() > max_dynamic_paths) @@ -1288,10 +1325,18 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou std::sort(paths_with_sizes.begin(), paths_with_sizes.end(), std::greater()); /// Fill dynamic_paths with first max_dynamic_paths paths in sorted list. - for (size_t i = 0; i != max_dynamic_paths; ++i) + for (const auto & [size, path] : paths_with_sizes) { - dynamic_paths.emplace(paths_with_sizes[i].second, ColumnDynamic::create(max_dynamic_types)); - dynamic_paths_ptrs.emplace(paths_with_sizes[i].second, assert_cast(dynamic_paths.find(paths_with_sizes[i].second)->second.get())); + if (dynamic_paths.size() < max_dynamic_paths) + { + dynamic_paths.emplace(path, ColumnDynamic::create(max_dynamic_types)); + dynamic_paths_ptrs.emplace(path, assert_cast(dynamic_paths.find(path)->second.get())); + } + /// Add all remaining paths into shared data statistics until we reach its max size; + else if (new_statistics.shared_data_paths_statistics.size() < Statistics::MAX_SHARED_DATA_STATISTICS_SIZE) + { + new_statistics.shared_data_paths_statistics.emplace(path, size); + } } } /// Use all dynamic paths from all source columns. @@ -1305,10 +1350,13 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou } /// Fill statistics for the merged part. - statistics.data.clear(); - statistics.source = Statistics::Source::MERGE; for (const auto & [path, _] : dynamic_paths) - statistics.data[path] = path_to_total_number_of_non_null_values[path]; + new_statistics.dynamic_paths_statistics[path] = path_to_total_number_of_non_null_values[path]; + statistics = std::make_shared(std::move(new_statistics)); + + /// Set max_dynamic_paths to the number of selected dynamic paths. + /// It's needed to avoid adding new unexpected dynamic paths during inserts into this column during merge. + max_dynamic_paths = dynamic_paths.size(); /// Now we have the resulting set of dynamic paths that will be used in all merged columns. /// As we use Dynamic column for dynamic paths, we should call takeDynamicStructureFromSourceColumns diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 38c71e94e12..8412a50281a 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -26,12 +26,21 @@ public: MERGE, /// Statistics were calculated during merge of several MergeTree parts. }; + explicit Statistics(Source source_) : source(source_) {} + /// Source of the statistics. Source source; - /// Statistics data: (path) -> (total number of not-null values). - std::unordered_map data; + /// Statistics for dynamic paths: (path) -> (total number of not-null values). + std::unordered_map dynamic_paths_statistics; + /// Statistics for paths in shared data: path) -> (total number of not-null values). + /// We don't store statistics for all paths in shared data but only for some subset of them + /// (is 10000 a good limit? It should not be expensive to store 10000 paths per part) + static const size_t MAX_SHARED_DATA_STATISTICS_SIZE = 10000; + std::unordered_map shared_data_paths_statistics; }; + using StatisticsPtr = std::shared_ptr; + private: friend class COWHelper, ColumnObject>; @@ -41,8 +50,9 @@ private: std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, + size_t global_max_dynamic_paths_, size_t max_dynamic_types_, - const Statistics & statistics_ = {}); + const StatisticsPtr & statistics_ = {}); /// Use StringHashForHeterogeneousLookup hash for hash maps to be able to use std::string_view in find() method. using PathToColumnMap = std::unordered_map; @@ -58,16 +68,18 @@ public: const std::unordered_map & dynamic_paths_, const ColumnPtr & shared_data_, size_t max_dynamic_paths_, + size_t global_max_dynamic_paths_, size_t max_dynamic_types_, - const Statistics & statistics_ = {}); + const StatisticsPtr & statistics_ = {}); static MutablePtr create( std::unordered_map typed_paths_, std::unordered_map dynamic_paths_, MutableColumnPtr shared_data_, size_t max_dynamic_paths_, + size_t global_max_dynamic_paths_, size_t max_dynamic_types_, - const Statistics & statistics_ = {}); + const StatisticsPtr & statistics_ = {}); static MutablePtr create(std::unordered_map typed_paths_, size_t max_dynamic_paths_, size_t max_dynamic_types_); @@ -171,7 +183,7 @@ public: const PathToDynamicColumnPtrMap & getDynamicPathsPtrs() const { return dynamic_paths_ptrs; } PathToDynamicColumnPtrMap & getDynamicPathsPtrs() { return dynamic_paths_ptrs; } - const Statistics & getStatistics() const { return statistics; } + const StatisticsPtr & getStatistics() const { return statistics; } const ColumnPtr & getSharedDataPtr() const { return shared_data; } ColumnPtr & getSharedDataPtr() { return shared_data; } @@ -199,6 +211,7 @@ public: size_t getMaxDynamicTypes() const { return max_dynamic_types; } size_t getMaxDynamicPaths() const { return max_dynamic_paths; } + size_t getGlobalMaxDynamicPaths() const { return global_max_dynamic_paths; } /// Try to add new dynamic path. Returns pointer to the new dynamic /// path column or nullptr if limit on dynamic paths is reached. @@ -207,7 +220,8 @@ public: void addNewDynamicPath(const std::string_view path); void setDynamicPaths(const std::vector & paths); - void setStatistics(const Statistics & statistics_) { statistics = statistics_; } + void setMaxDynamicPaths(size_t max_dynamic_paths_); + void setStatistics(const StatisticsPtr & statistics_) { statistics = statistics_; } void serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const std::string_view path, const IColumn & column, size_t n); void deserializeValueFromSharedData(const ColumnString * shared_data_values, size_t n, IColumn & column) const; @@ -238,12 +252,17 @@ private: WrappedPtr shared_data; /// Maximum number of dynamic paths. If this limit is reached, all new paths will be inserted into shared data. + /// This limit can be different for different instances of Object column. For example, we can decrease it + /// in takeDynamicStructureFromSourceColumns before merge. size_t max_dynamic_paths; + /// Global limit on number of dynamic paths for all column instances of this Object type. It's the limit specified + /// in the type definition (for example 'JSON(max_dynamic_paths=N)'). max_dynamic_paths is always not greater than this limit. + size_t global_max_dynamic_paths; /// Maximum number of dynamic types for each dynamic path. Used while creating Dynamic columns for new dynamic paths. size_t max_dynamic_types; - /// Statistics on the number of non-null values for each dynamic path in the MergeTree data part. - /// Calculated during merges or reading from MergeTree. Used to determine the set of dynamic paths for the merged part. - Statistics statistics; + /// Statistics on the number of non-null values for each dynamic path and for some shared data paths in the MergeTree data part. + /// Calculated during serializing of data part in MergeTree. Used to determine the set of dynamic paths for the merged part. + StatisticsPtr statistics; }; } diff --git a/src/DataTypes/DataTypesBinaryEncoding.h b/src/DataTypes/DataTypesBinaryEncoding.h index 5ac21471a29..cdfbfee1ccf 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.h +++ b/src/DataTypes/DataTypesBinaryEncoding.h @@ -8,59 +8,59 @@ namespace DB /** Binary encoding for ClickHouse data types: -|---------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| ClickHouse data type | Binary encoding | -|---------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| Nothing | 0x00 | -| UInt8 | 0x01 | -| UInt16 | 0x02 | -| UInt32 | 0x03 | -| UInt64 | 0x04 | -| UInt128 | 0x05 | -| UInt256 | 0x06 | -| Int8 | 0x07 | -| Int16 | 0x08 | -| Int32 | 0x09 | -| Int64 | 0x0A | -| Int128 | 0x0B | -| Int256 | 0x0C | -| Float32 | 0x0D | -| Float64 | 0x0E | -| Date | 0x0F | -| Date32 | 0x10 | -| DateTime | 0x11 | -| DateTime(time_zone) | 0x12 | -| DateTime64(P) | 0x13 | -| DateTime64(P, time_zone) | 0x14 | -| String | 0x15 | -| FixedString(N) | 0x16 | -| Enum8 | 0x17... | -| Enum16 | 0x18...> | -| Decimal32(P, S) | 0x19 | -| Decimal64(P, S) | 0x1A | -| Decimal128(P, S) | 0x1B | -| Decimal256(P, S) | 0x1C | -| UUID | 0x1D | -| Array(T) | 0x1E | -| Tuple(T1, ..., TN) | 0x1F... | -| Tuple(name1 T1, ..., nameN TN) | 0x20... | -| Set| 0x21 | -| Interval | 0x22 | -| Nullable(T) | 0x23 | -| Function | 0x24... | -| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | -| LowCardinality(T) | 0x26 | -| Map(K, V) | 0x27 | -| IPv4 | 0x28 | -| IPv6 | 0x29 | -| Variant(T1, ..., TN) | 0x2A... | -| Dynamic(max_types=N) | 0x2B | -| Custom type (Ring, Polygon, etc) | 0x2C | -| Bool | 0x2D | -| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | -| Nested(name1 T1, ..., nameN TN) | 0x2F... | -| JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP REGEXP skip_path_regexp) | 0x30......... | -|---------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +|---------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| ClickHouse data type | Binary encoding | +|---------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Nothing | 0x00 | +| UInt8 | 0x01 | +| UInt16 | 0x02 | +| UInt32 | 0x03 | +| UInt64 | 0x04 | +| UInt128 | 0x05 | +| UInt256 | 0x06 | +| Int8 | 0x07 | +| Int16 | 0x08 | +| Int32 | 0x09 | +| Int64 | 0x0A | +| Int128 | 0x0B | +| Int256 | 0x0C | +| Float32 | 0x0D | +| Float64 | 0x0E | +| Date | 0x0F | +| Date32 | 0x10 | +| DateTime | 0x11 | +| DateTime(time_zone) | 0x12 | +| DateTime64(P) | 0x13 | +| DateTime64(P, time_zone) | 0x14 | +| String | 0x15 | +| FixedString(N) | 0x16 | +| Enum8 | 0x17... | +| Enum16 | 0x18...> | +| Decimal32(P, S) | 0x19 | +| Decimal64(P, S) | 0x1A | +| Decimal128(P, S) | 0x1B | +| Decimal256(P, S) | 0x1C | +| UUID | 0x1D | +| Array(T) | 0x1E | +| Tuple(T1, ..., TN) | 0x1F... | +| Tuple(name1 T1, ..., nameN TN) | 0x20... | +| Set| 0x21 | +| Interval | 0x22 | +| Nullable(T) | 0x23 | +| Function | 0x24... | +| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | +| LowCardinality(T) | 0x26 | +| Map(K, V) | 0x27 | +| IPv4 | 0x28 | +| IPv6 | 0x29 | +| Variant(T1, ..., TN) | 0x2A... | +| Dynamic(max_types=N) | 0x2B | +| Custom type (Ring, Polygon, etc) | 0x2C | +| Bool | 0x2D | +| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | +| Nested(name1 T1, ..., nameN TN) | 0x2F... | +| JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP REGEXP skip_path_regexp) | 0x30......... | +|---------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| Interval kind binary encoding: |---------------|-----------------| diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 68d25eaeaff..a1580ffedd4 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -75,14 +75,20 @@ void SerializationObject::ObjectSerializationVersion::checkVersion(UInt64 versio struct SerializeBinaryBulkStateObject: public ISerialization::SerializeBinaryBulkState { SerializationObject::ObjectSerializationVersion serialization_version; + size_t max_dynamic_paths; std::vector sorted_dynamic_paths; std::unordered_map typed_path_states; std::unordered_map dynamic_path_states; ISerialization::SerializeBinaryBulkStatePtr shared_data_state; - /// Paths statistics. Map (dynamic path) -> (number of non-null values in this path). - ColumnObject::Statistics statistics = { .source = ColumnObject::Statistics::Source::READ, .data = {} }; + /// Paths statistics. + ColumnObject::Statistics statistics; + /// If true, statistics will be recalculated during serialization. + bool recalculate_statistics = false; - explicit SerializeBinaryBulkStateObject(UInt64 serialization_version_) : serialization_version(serialization_version_) {} + explicit SerializeBinaryBulkStateObject(UInt64 serialization_version_) + : serialization_version(serialization_version_), statistics(ColumnObject::Statistics::Source::READ) + { + } }; struct DeserializeBinaryBulkStateObject : public ISerialization::DeserializeBinaryBulkState @@ -190,8 +196,11 @@ void SerializationObject::serializeBinaryBulkStatePrefix( UInt64 serialization_version = ObjectSerializationVersion::Value::BASIC; writeBinaryLittleEndian(serialization_version, *stream); - /// Write all dynamic paths in sorted order. auto object_state = std::make_shared(serialization_version); + object_state->max_dynamic_paths = column_object.getMaxDynamicPaths(); + /// Write max_dynamic_paths parameter. + writeBinaryLittleEndian(object_state->max_dynamic_paths, *stream); + /// Write all dynamic paths in sorted order. object_state->sorted_dynamic_paths.reserve(dynamic_paths.size()); for (const auto & [path, _] : dynamic_paths) object_state->sorted_dynamic_paths.push_back(path); @@ -204,6 +213,7 @@ void SerializationObject::serializeBinaryBulkStatePrefix( if (settings.object_and_dynamic_write_statistics == SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::PREFIX) { const auto & statistics = column_object.getStatistics(); + /// First, write statistics for dynamic paths. for (const auto & path : object_state->sorted_dynamic_paths) { size_t number_of_non_null_values = 0; @@ -213,13 +223,52 @@ void SerializationObject::serializeBinaryBulkStatePrefix( /// - statistics read from the data part during deserialization of Object column (Statistics::Source::READ). /// We can rely only on statistics calculated during the merge, because column with statistics that was read /// during deserialization from some data part could be filtered/limited/transformed/etc and so the statistics can be outdated. - if (!statistics.data.empty() && statistics.source == ColumnObject::Statistics::Source::MERGE) - number_of_non_null_values = statistics.data.at(path); + if (statistics && statistics->source == ColumnObject::Statistics::Source::MERGE) + number_of_non_null_values = statistics->dynamic_paths_statistics.at(path); /// Otherwise we can use only path column from current object column. else number_of_non_null_values = (dynamic_paths.at(path)->size() - dynamic_paths.at(path)->getNumberOfDefaultRows()); writeVarUInt(number_of_non_null_values, *stream); } + + /// Second, write statistics for paths in shared data. + /// Check if we have statistics calculated during merge of some data parts (Statistics::Source::MERGE). + if (statistics && statistics->source == ColumnObject::Statistics::Source::MERGE) + { + writeVarUInt(statistics->shared_data_paths_statistics.size(), *stream); + for (const auto & [path, size] : statistics->shared_data_paths_statistics) + { + writeStringBinary(path, *stream); + writeVarUInt(size, *stream); + } + } + /// If we don't have statistics for shared data from merge, calculate it from the column. + else + { + std::unordered_map shared_data_paths_statistics; + const auto [shared_data_paths, _] = column_object.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + { + auto path = shared_data_paths->getDataAt(i).toView(); + if (auto it = shared_data_paths_statistics.find(path); it != shared_data_paths_statistics.end()) + ++it->second; + else if (shared_data_paths_statistics.size() < ColumnObject::Statistics::MAX_SHARED_DATA_STATISTICS_SIZE) + shared_data_paths_statistics.emplace(path, 1); + } + + writeVarUInt(shared_data_paths_statistics.size(), *stream); + for (const auto & [path, size] : shared_data_paths_statistics) + { + writeStringBinary(path, *stream); + writeVarUInt(size, *stream); + } + } + } + /// Otherwise statistics will be written in the suffix, in this case we will recalculate + /// statistics during serialization to make it more precise. + else + { + object_state->recalculate_statistics = true; } settings.path.push_back(Substream::ObjectData); @@ -304,6 +353,8 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationObject::deserializeOb UInt64 serialization_version; readBinaryLittleEndian(serialization_version, *structure_stream); auto structure_state = std::make_shared(serialization_version); + /// Read max_dynamic_paths parameter. + readBinaryLittleEndian(structure_state->max_dynamic_paths, *structure_stream); /// Read the sorted list of dynamic paths. size_t dynamic_paths_size; readVarUInt(dynamic_paths_size, *structure_stream); @@ -319,8 +370,24 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationObject::deserializeOb /// Read statistics if needed. if (settings.object_and_dynamic_read_statistics) { + ColumnObject::Statistics statistics(ColumnObject::Statistics::Source::READ); + statistics.dynamic_paths_statistics.reserve(structure_state->sorted_dynamic_paths.size()); + /// First, read dynamic paths statistics. for (const auto & path : structure_state->sorted_dynamic_paths) - readVarUInt(structure_state->statistics.data[path], *structure_stream); + readVarUInt(statistics.dynamic_paths_statistics[path], *structure_stream); + + /// Second, read shared data paths statistics. + size_t size; + readVarUInt(size, *structure_stream); + statistics.shared_data_paths_statistics.reserve(size); + String path; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(path, *structure_stream); + readVarUInt(statistics.shared_data_paths_statistics[path], *structure_stream); + } + + structure_state->statistics = std::make_shared(std::move(statistics)); } state = std::move(structure_state); @@ -362,14 +429,37 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( auto it = dynamic_paths.find(path); if (it == dynamic_paths.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Dynamic structure mismatch for Object column: dynamic path '{}' is not found in the column", path); - size_t number_of_non_null_values = 0; - dynamic_serialization_typed->serializeBinaryBulkWithMultipleStreamsAndCountTotalSizeOfVariants(*it->second, offset, limit, settings, object_state->dynamic_path_states[path], number_of_non_null_values); - object_state->statistics.data[path] += number_of_non_null_values; + if (object_state->recalculate_statistics) + { + size_t number_of_non_null_values = 0; + dynamic_serialization_typed->serializeBinaryBulkWithMultipleStreamsAndCountTotalSizeOfVariants(*it->second, offset, limit, settings, object_state->dynamic_path_states[path], number_of_non_null_values); + object_state->statistics.dynamic_paths_statistics[path] += number_of_non_null_values; + } + else + { + dynamic_serialization_typed->serializeBinaryBulkWithMultipleStreams(*it->second, offset, limit, settings, object_state->dynamic_path_states[path]); + } settings.path.pop_back(); } settings.path.push_back(Substream::ObjectSharedData); shared_data_serialization->serializeBinaryBulkWithMultipleStreams(*shared_data, offset, limit, settings, object_state->shared_data_state); + if (object_state->recalculate_statistics) + { + /// Calculate statistics for paths in shared data. + const auto [shared_data_paths, _] = column_object.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column_object.getSharedDataOffsets(); + size_t start = shared_data_offsets[offset - 1]; + size_t end = limit == 0 || offset + limit > shared_data_offsets.size() ? shared_data_paths->size() : shared_data_offsets[offset + limit - 1]; + for (size_t i = start; i != end; ++i) + { + auto path = shared_data_paths->getDataAt(i).toView(); + if (auto it = object_state->statistics.shared_data_paths_statistics.find(path); it != object_state->statistics.shared_data_paths_statistics.end()) + ++it->second; + else if (object_state->statistics.shared_data_paths_statistics.size() < ColumnObject::Statistics::MAX_SHARED_DATA_STATISTICS_SIZE) + object_state->statistics.shared_data_paths_statistics.emplace(path, 1); + } + } settings.path.pop_back(); settings.path.pop_back(); } @@ -388,8 +478,17 @@ void SerializationObject::serializeBinaryBulkStateSuffix( /// Write statistics in suffix if needed. if (settings.object_and_dynamic_write_statistics == SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::SUFFIX) { + /// First, write dynamic paths statistics. for (const auto & path : object_state->sorted_dynamic_paths) - writeVarUInt(object_state->statistics.data[path], *stream); + writeVarUInt(object_state->statistics.dynamic_paths_statistics[path], *stream); + + /// Second, write shared data paths statistics. + writeVarUInt(object_state->statistics.shared_data_paths_statistics.size(), *stream); + for (const auto & [path, size] : object_state->statistics.shared_data_paths_statistics) + { + writeStringBinary(path, *stream); + writeVarUInt(size, *stream); + } } settings.path.push_back(Substream::ObjectData); diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index faf15aa3260..62ff9849f45 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -82,10 +82,11 @@ private: struct DeserializeBinaryBulkStateObjectStructure : public ISerialization::DeserializeBinaryBulkState { ObjectSerializationVersion structure_version; + size_t max_dynamic_paths; std::vector sorted_dynamic_paths; std::unordered_set dynamic_paths; /// Paths statistics. Map (dynamic path) -> (number of non-null values in this path). - ColumnObject::Statistics statistics = {.source = ColumnObject::Statistics::Source::READ, .data = {}}; + ColumnObject::StatisticsPtr statistics; explicit DeserializeBinaryBulkStateObjectStructure(UInt64 structure_version_) : structure_version(structure_version_) {} }; diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index ad7a5a4c863..95b76c60063 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -146,7 +146,6 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl { /// We know all the data we will insert in advance and can make all necessary pre-allocations. mutable_columns[i]->prepareForSquashing(source_columns_list[i]); -// mutable_columns[i]->reserve(rows); for (auto & source_column : source_columns_list[i]) { auto column = std::move(source_column); diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 index 31ba25b7732..42f24e7f26f 100644 --- a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 @@ -1,3 +1,4 @@ +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; Dynamic paths 100000 a 90000 b @@ -47,6 +48,26 @@ Shared data paths 60000 e 10000 g Dynamic paths +200000 f +100000 a +90000 b +40000 c +Shared data paths +80000 c +70000 d +60000 e +10000 g +Dynamic paths +200000 f +120000 c +100000 a +Shared data paths +90000 b +70000 d +60000 e +10000 g +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; +Dynamic paths 100000 a 90000 b 80000 c @@ -94,3 +115,22 @@ Shared data paths 70000 d 60000 e 10000 g +Dynamic paths +200000 f +100000 a +90000 b +40000 c +Shared data paths +80000 c +70000 d +60000 e +10000 g +Dynamic paths +200000 f +120000 c +100000 a +Shared data paths +90000 b +70000 d +60000 e +10000 g diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 index 6ae9f438432..05b5d8d6095 100644 --- a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 @@ -7,6 +7,8 @@ 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;'] -%} +select '{{ create_command }}'; + {{ create_command }} system stop merges test; @@ -54,6 +56,19 @@ select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by pat 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('c', number)) from numbers(40000); +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 -%} diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 index 31ba25b7732..99257f1d6ce 100644 --- a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 @@ -1,3 +1,4 @@ +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; Dynamic paths 100000 a 90000 b @@ -47,6 +48,26 @@ Shared data paths 60000 e 10000 g Dynamic paths +200000 f +100000 a +90000 b +40000 c +Shared data paths +80000 c +70000 d +60000 e +10000 g +Dynamic paths +200000 f +120000 c +100000 a +Shared data paths +90000 b +70000 d +60000 e +10000 g +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; +Dynamic paths 100000 a 90000 b 80000 c @@ -94,3 +115,22 @@ Shared data paths 70000 d 60000 e 10000 g +Dynamic paths +200000 f +100000 a +90000 b +40000 c +Shared data paths +80000 c +70000 d +60000 e +10000 g +Dynamic paths +200000 f +120000 c +100000 a +Shared data paths +90000 b +70000 d +60000 e +10000 g diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 b/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 index aef36452bb8..d17d8a48537 100644 --- a/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 @@ -7,6 +7,8 @@ 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;'] -%} +select '{{ create_command }}'; + {{ create_command }} system stop merges test; @@ -54,6 +56,19 @@ select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by pat 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('c', number)) from numbers(40000); +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 -%} diff --git a/tests/queries/0_stateless/03222_json_squashing.reference b/tests/queries/0_stateless/03222_json_squashing.reference new file mode 100644 index 00000000000..d0c19d8239a --- /dev/null +++ b/tests/queries/0_stateless/03222_json_squashing.reference @@ -0,0 +1,102 @@ +All paths +a +b +c +d +Dynamic paths +a +b +c +d +Shared data paths +All paths +a +b +c +d +e +f +Dynamic paths +a +b +c +d +e +f +Shared data paths +All paths +a +b +c +d +Dynamic paths +c +d +Shared data paths +a +b +All paths +a +b +c +d +e +f +Dynamic paths +a +b +Shared data paths +c +d +e +f +All paths +a +b +c +d +e +Dynamic paths +a +e +Shared data paths +b +c +d +All paths +b +c +d +e +Dynamic paths +d +e +Shared data paths +b +c +All paths +b +c +d +e +f +g +Dynamic paths +b +c +Shared data paths +d +e +f +g +All paths +b +d +e +f +Dynamic paths +b +f +Shared data paths +d +e diff --git a/tests/queries/0_stateless/03222_json_squashing.sql b/tests/queries/0_stateless/03222_json_squashing.sql new file mode 100644 index 00000000000..b6bda7a702f --- /dev/null +++ b/tests/queries/0_stateless/03222_json_squashing.sql @@ -0,0 +1,82 @@ +-- Tags: long + +set allow_experimental_json_type = 1; +set max_block_size = 1000; + +drop table if exists test; + +create table test (json JSON) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, '{}'::JSON, number < 3000, '{"a" : 42, "b" : "Hello"}'::JSON, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; + +truncate table test; +insert into test select multiIf(number < 1000, '{"a" : 42, "b" : "Hello"}'::JSON, number < 3000, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON, '{"e" : 43, "f" : ["s1", "s2", "s3"]}'::JSON) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; + +drop table test; +create table test (json JSON(max_dynamic_paths=2)) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, '{}'::JSON(max_dynamic_paths=2), number < 3000, '{"a" : 42, "b" : "Hello"}'::JSON(max_dynamic_paths=2), '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2)) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; + +truncate table test; +insert into test select multiIf(number < 1000, '{"a" : 42, "b" : "Hello"}'::JSON(max_dynamic_paths=2), number < 3000, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2), '{"e" : 43, "f" : ["s1", "s2", "s3"]}'::JSON(max_dynamic_paths=2)) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; + +truncate table test; +insert into test select multiIf(number < 1000, '{"a" : 42}'::JSON(max_dynamic_paths=2), number < 3000, '{"b" : "Hello", "c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2), '{"e" : 43}'::JSON(max_dynamic_paths=2)) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; + +drop table test; +create table test (json JSON(max_dynamic_paths=8)) engine=MergeTree order by tuple(); +insert into test select multiIf(number < 1000, '{}'::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"b" : 42, "c" : [1, 2, 3]}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8)) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(arrayJoin(json.a[]))) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(arrayJoin(json.a[]))) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(arrayJoin(json.a[]))) as path from test order by path; + +truncate table test; +insert into test select multiIf(number < 1000, materialize('{"a" : [{"b" : 42, "c" : [1, 2, 3]}]}')::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"f" : "2020-01-01 00:00:00", "g" : "Hello2"}]}')::JSON(max_dynamic_paths=8)) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(arrayJoin(json.a[]))) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(arrayJoin(json.a[]))) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(arrayJoin(json.a[]))) as path from test order by path; + +truncate table test; +insert into test select multiIf(number < 1000, materialize('{"a" : [{"b" : 42}]}')::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"f" : "2020-01-01 00:00:00"}]}')::JSON(max_dynamic_paths=8)) from numbers(1000000); +select 'All paths'; +select distinct arrayJoin(JSONAllPaths(arrayJoin(json.a[]))) as path from test order by path; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(arrayJoin(json.a[]))) as path from test order by path; +select 'Shared data paths'; +select distinct arrayJoin(JSONSharedDataPaths(arrayJoin(json.a[]))) as path from test order by path; + +drop table test; \ No newline at end of file From 0b728e7547b7e1642b37cdc6721fd05a42abccce Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 13:56:08 +0000 Subject: [PATCH 0883/1722] Update docs --- .../data-types/data-types-binary-encoding.md | 103 +++++++++--------- 1 file changed, 52 insertions(+), 51 deletions(-) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index bbd47d6f620..08fb664126a 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -12,58 +12,59 @@ This specification describes the binary format that can be used for binary encod The table below describes how each data type is represented in binary format. Each data type encoding consist of 1 byte that indicates the type and some optional additional information. `var_uint` in the binary encoding means that the size is encoded using Variable-Length Quantity compression. -| ClickHouse data type | Binary encoding | -|-----------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `Nothing` | `0x00` | -| `UInt8` | `0x01` | -| `UInt16` | `0x02` | -| `UInt32` | `0x03` | -| `UInt64` | `0x04` | -| `UInt128` | `0x05` | -| `UInt256` | `0x06` | -| `Int8` | `0x07` | -| `Int16` | `0x08` | -| `Int32` | `0x09` | -| `Int64` | `0x0A` | -| `Int128` | `0x0B` | -| `Int256` | `0x0C` | -| `Float32` | `0x0D` | -| `Float64` | `0x0E` | -| `Date` | `0x0F` | -| `Date32` | `0x10` | -| `DateTime` | `0x11` | -| `DateTime(time_zone)` | `0x12` | -| `DateTime64(P)` | `0x13` | -| `DateTime64(P, time_zone)` | `0x14` | -| `String` | `0x15` | -| `FixedString(N)` | `0x16` | -| `Enum8` | `0x17...` | -| `Enum16` | `0x18...>` | -| `Decimal32(P, S)` | `0x19` | -| `Decimal64(P, S)` | `0x1A` | -| `Decimal128(P, S)` | `0x1B` | -| `Decimal256(P, S)` | `0x1C` | -| `UUID` | `0x1D` | -| `Array(T)` | `0x1E` | -| `Tuple(T1, ..., TN)` | `0x1F...` | -| `Tuple(name1 T1, ..., nameN TN)` | `0x20...` | -| `Set` | `0x21` | -| `Interval` | `0x22` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | -| `Nullable(T)` | `0x23` | -| `Function` | `0x24...` | -| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x25......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `LowCardinality(T)` | `0x26` | -| `Map(K, V)` | `0x27` | -| `IPv4` | `0x28` | -| `IPv6` | `0x29` | -| `Variant(T1, ..., TN)` | `0x2A...` | -| `Dynamic(max_types=N)` | `0x2B` | -| `Custom type` (`Ring`, `Polygon`, etc) | `0x2C` | -| `Bool` | `0x2D` | -| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2E......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `Nested(name1 T1, ..., nameN TN)` | `0x2F...` | -| `JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP REGEXP skip_path_regexp)` | `0x30.........` | +| ClickHouse data type | Binary encoding | +|-----------------------------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `Nothing` | `0x00` | +| `UInt8` | `0x01` | +| `UInt16` | `0x02` | +| `UInt32` | `0x03` | +| `UInt64` | `0x04` | +| `UInt128` | `0x05` | +| `UInt256` | `0x06` | +| `Int8` | `0x07` | +| `Int16` | `0x08` | +| `Int32` | `0x09` | +| `Int64` | `0x0A` | +| `Int128` | `0x0B` | +| `Int256` | `0x0C` | +| `Float32` | `0x0D` | +| `Float64` | `0x0E` | +| `Date` | `0x0F` | +| `Date32` | `0x10` | +| `DateTime` | `0x11` | +| `DateTime(time_zone)` | `0x12` | +| `DateTime64(P)` | `0x13` | +| `DateTime64(P, time_zone)` | `0x14` | +| `String` | `0x15` | +| `FixedString(N)` | `0x16` | +| `Enum8` | `0x17...` | +| `Enum16` | `0x18...>` | +| `Decimal32(P, S)` | `0x19` | +| `Decimal64(P, S)` | `0x1A` | +| `Decimal128(P, S)` | `0x1B` | +| `Decimal256(P, S)` | `0x1C` | +| `UUID` | `0x1D` | +| `Array(T)` | `0x1E` | +| `Tuple(T1, ..., TN)` | `0x1F...` | +| `Tuple(name1 T1, ..., nameN TN)` | `0x20...` | +| `Set` | `0x21` | +| `Interval` | `0x22` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | +| `Nullable(T)` | `0x23` | +| `Function` | `0x24...` | +| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x25......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `LowCardinality(T)` | `0x26` | +| `Map(K, V)` | `0x27` | +| `IPv4` | `0x28` | +| `IPv6` | `0x29` | +| `Variant(T1, ..., TN)` | `0x2A...` | +| `Dynamic(max_types=N)` | `0x2B` | +| `Custom type` (`Ring`, `Polygon`, etc) | `0x2C` | +| `Bool` | `0x2D` | +| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2E......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `Nested(name1 T1, ..., nameN TN)` | `0x2F...` | +| `JSON(max_dynamic_paths=N, max_dynamic_types=M, path Type, SKIP skip_path, SKIP REGEXP skip_path_regexp)` | `0x30.........` | +For type `JSON` byte `uint8_serialization_version` indicates the version of the serialization. Right now the version is always 0 but can change in future if new arguments will be introduced for `JSON` type. ### Interval kind binary encoding From 0414cdbbbf32efe10a92c9dd93ba47743ceeb848 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 13 Aug 2024 15:58:49 +0200 Subject: [PATCH 0884/1722] Fix unpack error --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5946e561949..5bde4686d3a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1995,7 +1995,7 @@ class TestSuite: tag_line = find_tag_line(file) next_line = file.readline() except UnicodeDecodeError: - return [] + return [], {} try: if filepath.endswith(".sql"): for line in file: From b5f7875f574b640dc00d42dea5e721058e9a10fc Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 14:04:04 +0000 Subject: [PATCH 0885/1722] Remove odd new line --- tests/queries/0_stateless/00652_mergetree_mutations.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index 3b0966dd2c3..6be0ebf882f 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -89,4 +89,3 @@ done ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner' ORDER BY mutation_id" ${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner" - From 79e055783931bd544897ec23eb07b7c7c9b09a69 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 13 Aug 2024 14:18:48 +0000 Subject: [PATCH 0886/1722] Write a script to automatically update the dict --- tests/fuzz/README.md | 23 - tests/fuzz/all.dict | 2377 +++++++------ tests/fuzz/dictionaries/datatypes.dict | 4418 +----------------------- tests/fuzz/dictionaries/functions.dict | 4283 ----------------------- tests/fuzz/update_dict.sh | 20 + 5 files changed, 1389 insertions(+), 9732 deletions(-) delete mode 100644 tests/fuzz/README.md create mode 100755 tests/fuzz/update_dict.sh diff --git a/tests/fuzz/README.md b/tests/fuzz/README.md deleted file mode 100644 index 576ad66ed93..00000000000 --- a/tests/fuzz/README.md +++ /dev/null @@ -1,23 +0,0 @@ -The list of functions generated via the following query - -``` - clickhouse client -q "SELECT * FROM (SELECT DISTINCT concat('\"', name, '\"') as res FROM system.functions ORDER BY name UNION ALL SELECT concat('\"', a.name, b.name, '\"') as res FROM system.functions as a CROSS JOIN system.aggregate_function_combinators as b WHERE a.is_aggregate = 1) ORDER BY res" > functions.dict -``` - -The list of datatypes generated via the following query: - -``` - clickhouse client -q "SELECT DISTINCT concat('\"', name, '\"') as res FROM system.data_type_families ORDER BY name" > datatypes.dict -``` - -The list of keywords generated via the following query: - -``` - clickhouse client -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > keywords.dict -``` - -Then merge all dictionaries into one (all.dict) - -``` - cat ./dictionaries/* | LC_ALL=C sort | uniq > all.dict -``` \ No newline at end of file diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 1c3c657d6b0..30af3746fca 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -1,15 +1,971 @@ -"abs" -"accurateCast" -"accurateCastOrDefault" -"accurateCastOrNull" -"acos" -"acosh" "ADD COLUMN" "ADD CONSTRAINT" "ADD INDEX" "ADD PROJECTION" "ADD STATISTICS" "ADD" +"ADMIN OPTION FOR" +"AFTER" +"ALGORITHM" +"ALIAS" +"ALL" +"ALLOWED_LATENESS" +"ALTER COLUMN" +"ALTER DATABASE" +"ALTER LIVE VIEW" +"ALTER POLICY" +"ALTER PROFILE" +"ALTER QUOTA" +"ALTER ROLE" +"ALTER ROW POLICY" +"ALTER SETTINGS PROFILE" +"ALTER TABLE" +"ALTER TEMPORARY TABLE" +"ALTER USER" +"ALTER" +"AND STDOUT" +"AND" +"ANTI" +"ANY" +"APPEND" +"APPLY DELETED MASK" +"APPLY" +"ARRAY JOIN" +"AS" +"ASC" +"ASCENDING" +"ASOF" +"ASSUME" +"AST" +"ASYNC" +"ATTACH PART" +"ATTACH PARTITION" +"ATTACH POLICY" +"ATTACH PROFILE" +"ATTACH QUOTA" +"ATTACH ROLE" +"ATTACH ROW POLICY" +"ATTACH SETTINGS PROFILE" +"ATTACH USER" +"ATTACH" +"AUTO_INCREMENT" +"AZURE" +"AggregateFunction" +"Array" +"BACKUP" +"BCRYPT_HASH" +"BCRYPT_PASSWORD" +"BEGIN TRANSACTION" +"BETWEEN" +"BIDIRECTIONAL" +"BIGINT SIGNED" +"BIGINT UNSIGNED" +"BIGINT" +"BINARY LARGE OBJECT" +"BINARY VARYING" +"BINARY" +"BIT" +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"BLOB" +"BOTH" +"BY" +"BYTE" +"BYTEA" +"Bool" +"CASCADE" +"CASE" +"CAST" +"CHANGE" +"CHANGEABLE_IN_READONLY" +"CHANGED" +"CHAR LARGE OBJECT" +"CHAR VARYING" +"CHAR" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CHARACTER" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"CHECK ALL TABLES" +"CHECK TABLE" +"CHECK" +"CLEANUP" +"CLEAR COLUMN" +"CLEAR INDEX" +"CLEAR PROJECTION" +"CLEAR STATISTICS" +"CLOB" +"CLUSTER" +"CLUSTERS" +"CN" +"CODEC" +"COLLATE" +"COLUMN" +"COLUMNS" +"COMMENT COLUMN" +"COMMENT" +"COMMIT" +"COMPRESSION" +"CONST" +"CONSTRAINT" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"CREATE POLICY" +"CREATE PROFILE" +"CREATE QUOTA" +"CREATE ROLE" +"CREATE ROW POLICY" +"CREATE SETTINGS PROFILE" +"CREATE TABLE" +"CREATE TEMPORARY TABLE" +"CREATE USER" +"CREATE" +"CROSS" +"CUBE" +"CURRENT GRANTS" +"CURRENT QUOTA" +"CURRENT ROLES" +"CURRENT ROW" +"CURRENT TRANSACTION" +"CURRENTUSER" +"CURRENT_USER" +"D" +"DATA INNER UUID" +"DATA" +"DATABASE" +"DATABASES" +"DATE" +"DATEADD" +"DATEDIFF" +"DATESUB" +"DATE_ADD" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_SUB" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"DAYS" +"DD" +"DEC" +"DEDUPLICATE" +"DEFAULT DATABASE" +"DEFAULT ROLE" +"DEFAULT" +"DEFINER" +"DELETE WHERE" +"DELETE" +"DEPENDS ON" +"DESC" +"DESCENDING" +"DESCRIBE" +"DETACH PART" +"DETACH PARTITION" +"DETACH" +"DICTIONARIES" +"DICTIONARY" +"DISK" +"DISTINCT ON" +"DISTINCT" +"DIV" +"DOUBLE PRECISION" +"DOUBLE" +"DOUBLE_SHA1_HASH" +"DOUBLE_SHA1_PASSWORD" +"DROP COLUMN" +"DROP CONSTRAINT" +"DROP DEFAULT" +"DROP DETACHED PART" +"DROP DETACHED PARTITION" +"DROP INDEX" +"DROP PART" +"DROP PARTITION" +"DROP PROJECTION" +"DROP STATISTICS" +"DROP TABLE" +"DROP TEMPORARY TABLE" +"DROP" +"Date" +"Date32" +"DateTime" +"DateTime32" +"DateTime64" +"Decimal" +"Decimal128" +"Decimal256" +"Decimal32" +"Decimal64" +"Dynamic" +"ELSE" +"EMPTY AS" +"EMPTY" +"ENABLED ROLES" +"END" +"ENFORCED" +"ENGINE" +"ENUM" +"EPHEMERAL SEQUENTIAL" +"EPHEMERAL" +"ESTIMATE" +"EVENT" +"EVENTS" +"EVERY" +"EXCEPT DATABASE" +"EXCEPT DATABASES" +"EXCEPT TABLE" +"EXCEPT TABLES" +"EXCEPT" +"EXCHANGE DICTIONARIES" +"EXCHANGE TABLES" +"EXISTS" +"EXPLAIN" +"EXPRESSION" +"EXTENDED" +"EXTERNAL DDL FROM" +"EXTRACT" +"Enum" +"Enum16" +"Enum8" +"FALSE" +"FETCH PART" +"FETCH PARTITION" +"FETCH" +"FIELDS" +"FILE" +"FILESYSTEM CACHE" +"FILESYSTEM CACHES" +"FILTER" +"FINAL" +"FIRST" +"FIXED" +"FLOAT" +"FOLLOWING" +"FOR" +"FOREIGN KEY" +"FOREIGN" +"FORGET PARTITION" +"FORMAT" +"FORMAT_BYTES" +"FQDN" +"FREEZE" +"FROM INFILE" +"FROM SHARD" +"FROM" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"FULL" +"FULLTEXT" +"FUNCTION" +"FixedString" +"Float32" +"Float64" +"ForEach" +"GEOMETRY" +"GLOBAL IN" +"GLOBAL NOT IN" +"GLOBAL" +"GRANT OPTION FOR" +"GRANT" +"GRANTEES" +"GRANULARITY" +"GROUP BY" +"GROUPING SETS" +"GROUPS" +"H" +"HASH" +"HAVING" +"HDFS" +"HH" +"HIERARCHICAL" +"HOST" +"HOUR" +"HOURS" +"HTTP" +"ID" +"IDENTIFIED" +"IF EMPTY" +"IF EXISTS" +"IF NOT EXISTS" +"IGNORE NULLS" +"ILIKE" +"IN PARTITION" +"IN" +"INDEX" +"INDEXES" +"INDICES" +"INET4" +"INET6" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"INHERIT" +"INJECTIVE" +"INNER" +"INSERT INTO" +"INT SIGNED" +"INT UNSIGNED" +"INT" +"INT1 SIGNED" +"INT1 UNSIGNED" +"INT1" +"INTEGER SIGNED" +"INTEGER UNSIGNED" +"INTEGER" +"INTERPOLATE" +"INTERSECT" +"INTERVAL" +"INTO OUTFILE" +"INVISIBLE" +"INVOKER" +"IP" +"IPv4" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"IS NOT DISTINCT FROM" +"IS NOT NULL" +"IS NULL" +"IS_OBJECT_ID" +"Int128" +"Int16" +"Int256" +"Int32" +"Int64" +"Int8" +"IntervalDay" +"IntervalHour" +"IntervalMicrosecond" +"IntervalMillisecond" +"IntervalMinute" +"IntervalMonth" +"IntervalNanosecond" +"IntervalQuarter" +"IntervalSecond" +"IntervalWeek" +"IntervalYear" +"JOIN" +"JSON" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" +"JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" +"JSONExtractString" +"JSONExtractUInt" +"JSONHas" +"JSONKey" +"JSONLength" +"JSONMergePatch" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"JWT" +"KERBEROS" +"KEY BY" +"KEY" +"KEYED BY" +"KEYS" +"KILL" +"KIND" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LARGE OBJECT" +"LAST" +"LAST_DAY" +"LAYOUT" +"LDAP" +"LEADING" +"LEFT ARRAY JOIN" +"LEFT" +"LESS THAN" +"LEVEL" +"LIFETIME" +"LIGHTWEIGHT" +"LIKE" +"LIMIT" +"LINEAR" +"LIST" +"LIVE" +"LOCAL" +"LONGBLOB" +"LONGTEXT" +"LTRIM" +"LineString" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LowCardinality" +"LpDistance" +"LpNorm" +"LpNormalize" +"M" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MATCH" +"MATERIALIZE COLUMN" +"MATERIALIZE INDEX" +"MATERIALIZE PROJECTION" +"MATERIALIZE STATISTICS" +"MATERIALIZE TTL" +"MATERIALIZE" +"MATERIALIZED" +"MAX" +"MCS" +"MD4" +"MD5" +"MEDIUMBLOB" +"MEDIUMINT SIGNED" +"MEDIUMINT UNSIGNED" +"MEDIUMINT" +"MEDIUMTEXT" +"MEMORY" +"MERGES" +"METRICS INNER UUID" +"METRICS" +"MI" +"MICROSECOND" +"MICROSECONDS" +"MILLISECOND" +"MILLISECONDS" +"MIN" +"MINUTE" +"MINUTES" +"MM" +"MOD" +"MODIFY COLUMN" +"MODIFY COMMENT" +"MODIFY DEFINER" +"MODIFY ORDER BY" +"MODIFY QUERY" +"MODIFY REFRESH" +"MODIFY SAMPLE BY" +"MODIFY SETTING" +"MODIFY SQL SECURITY" +"MODIFY STATISTICS" +"MODIFY TTL" +"MODIFY" +"MONTH" +"MONTHS" +"MOVE PART" +"MOVE PARTITION" +"MOVE" +"MS" +"MUTATION" +"Map" +"Merge" +"MultiLineString" +"MultiPolygon" +"N" +"NAME" +"NAMED COLLECTION" +"NANOSECOND" +"NANOSECONDS" +"NATIONAL CHAR VARYING" +"NATIONAL CHAR" +"NATIONAL CHARACTER LARGE OBJECT" +"NATIONAL CHARACTER VARYING" +"NATIONAL CHARACTER" +"NCHAR LARGE OBJECT" +"NCHAR VARYING" +"NCHAR" +"NEXT" +"NO ACTION" +"NO DELAY" +"NO LIMITS" +"NONE" +"NOT BETWEEN" +"NOT IDENTIFIED" +"NOT ILIKE" +"NOT IN" +"NOT KEYED" +"NOT LIKE" +"NOT OVERRIDABLE" +"NOT" +"NO_PASSWORD" +"NS" +"NULL" +"NULLS" +"NUMERIC" +"NVARCHAR" +"Nested" +"Nothing" +"Null" +"Nullable" +"OCTET_LENGTH" +"OFFSET" +"ON DELETE" +"ON UPDATE" +"ON VOLUME" +"ON" +"ONLY" +"OPTIMIZE TABLE" +"OR REPLACE" +"OR" +"ORDER BY" +"OUTER" +"OVER" +"OVERRIDABLE" +"Object" +"PART" +"PARTIAL" +"PARTITION BY" +"PARTITION" +"PARTITIONS" +"PART_MOVE_TO_SHARD" +"PASTE" +"PERIODIC REFRESH" +"PERMANENTLY" +"PERMISSIVE" +"PERSISTENT SEQUENTIAL" +"PERSISTENT" +"PIPELINE" +"PLAINTEXT_PASSWORD" +"PLAN" +"POPULATE" +"PRECEDING" +"PRECISION" +"PREWHERE" +"PRIMARY KEY" +"PRIMARY" +"PROFILE" +"PROJECTION" +"PULL" +"Point" +"Polygon" +"Protobuf" +"Q" +"QQ" +"QUALIFY" +"QUARTER" +"QUARTERS" +"QUERY TREE" +"QUERY" +"QUOTA" +"RANDOMIZE FOR" +"RANDOMIZED" +"RANGE" +"READONLY" +"REAL" +"REALM" +"RECOMPRESS" +"RECURSIVE" +"REFERENCES" +"REFRESH" +"REGEXP" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"REMOVE SAMPLE BY" +"REMOVE TTL" +"REMOVE" +"RENAME COLUMN" +"RENAME DATABASE" +"RENAME DICTIONARY" +"RENAME TABLE" +"RENAME TO" +"RENAME" +"REPLACE PARTITION" +"REPLACE" +"RESET SETTING" +"RESPECT NULLS" +"RESTORE" +"RESTRICT" +"RESTRICTIVE" +"RESUME" +"REVOKE" +"RIGHT" +"ROLLBACK" +"ROLLUP" +"ROW" +"ROWS" +"RTRIM" +"Resample" +"Ring" +"S" +"S3" +"SALT" +"SAMPLE BY" +"SAMPLE" +"SAN" +"SCHEMA" +"SCHEME" +"SECOND" +"SECONDS" +"SELECT" +"SEMI" +"SERVER" +"SET DEFAULT ROLE" +"SET DEFAULT" +"SET FAKE TIME" +"SET NULL" +"SET ROLE DEFAULT" +"SET ROLE" +"SET TRANSACTION SNAPSHOT" +"SET" +"SETTINGS" +"SHA1" +"SHA224" +"SHA256" +"SHA256_HASH" +"SHA256_PASSWORD" +"SHA384" +"SHA512" +"SHA512_256" +"SHOW ACCESS" +"SHOW CREATE" +"SHOW ENGINES" +"SHOW FUNCTIONS" +"SHOW GRANTS" +"SHOW PRIVILEGES" +"SHOW PROCESSLIST" +"SHOW SETTING" +"SHOW" +"SIGNED" +"SIMPLE" +"SINGLE" +"SMALLINT SIGNED" +"SMALLINT UNSIGNED" +"SMALLINT" +"SOURCE" +"SPATIAL" +"SQL SECURITY" +"SQL_TSI_DAY" +"SQL_TSI_HOUR" +"SQL_TSI_MICROSECOND" +"SQL_TSI_MILLISECOND" +"SQL_TSI_MINUTE" +"SQL_TSI_MONTH" +"SQL_TSI_NANOSECOND" +"SQL_TSI_QUARTER" +"SQL_TSI_SECOND" +"SQL_TSI_WEEK" +"SQL_TSI_YEAR" +"SS" +"SSH_KEY" +"SSL_CERTIFICATE" +"START TRANSACTION" +"STATISTICS" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"STEP" +"STORAGE" +"STRICT" +"STRICTLY_ASCENDING" +"SUBPARTITION BY" +"SUBPARTITION" +"SUBPARTITIONS" +"SUBSTRING" +"SUBSTRING_INDEX" +"SUSPEND" +"SVG" +"SYNC" +"SYNTAX" +"SYSTEM" +"SimpleAggregateFunction" +"State" +"String" +"TABLE OVERRIDE" +"TABLE" +"TABLES" +"TAGS INNER UUID" +"TAGS" +"TEMPORARY TABLE" +"TEMPORARY" +"TEST" +"TEXT" +"THEN" +"TIME" +"TIMESTAMP" +"TIMESTAMPADD" +"TIMESTAMPDIFF" +"TIMESTAMPSUB" +"TIMESTAMP_ADD" +"TIMESTAMP_DIFF" +"TIMESTAMP_SUB" +"TINYBLOB" +"TINYINT SIGNED" +"TINYINT UNSIGNED" +"TINYINT" +"TINYTEXT" +"TO DISK" +"TO INNER UUID" +"TO SHARD" +"TO TABLE" +"TO VOLUME" +"TO" +"TOP" +"TOTALS" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"TRACKING ONLY" +"TRAILING" +"TRANSACTION" +"TRIGGER" +"TRIM" +"TRUE" +"TRUNCATE" +"TTL" +"TYPE" +"TYPEOF" +"Tuple" +"UInt128" +"UInt16" +"UInt256" +"UInt32" +"UInt64" +"UInt8" +"ULIDStringToDateTime" +"UNBOUNDED" +"UNDROP" +"UNFREEZE" +"UNION" +"UNIQUE" +"UNSET FAKE TIME" +"UNSIGNED" +"UPDATE" +"URL" +"URLHash" +"URLHierarchy" +"URLPathHierarchy" +"USE" +"USING" +"UTCTimestamp" +"UTC_timestamp" +"UUID" +"UUIDNumToString" +"UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" +"VALID UNTIL" +"VALUES" +"VARBINARY" +"VARCHAR" +"VARCHAR2" +"VARYING" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" +"VIEW" +"VISIBLE" +"Variant" +"WATCH" +"WATERMARK" +"WEEK" +"WEEKS" +"WHEN" +"WHERE" +"WINDOW" +"WITH ADMIN OPTION" +"WITH CHECK" +"WITH FILL" +"WITH GRANT OPTION" +"WITH NAME" +"WITH REPLACE OPTION" +"WITH TIES" +"WITH" +"WITH_ITEMINDEX" +"WK" +"WRITABLE" +"WW" +"YEAR" +"YEARS" +"YY" +"YYYY" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"ZKPATH" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" "addDate" "addDays" "addHours" @@ -20,19 +976,16 @@ "addMonths" "addNanoseconds" "addQuarters" -"addressToLine" -"addressToLineWithInlines" -"addressToSymbol" "addSeconds" "addTupleOfIntervals" "addWeeks" "addYears" -"ADMIN OPTION FOR" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" "aes_decrypt_mysql" "aes_encrypt_mysql" -"AFTER" "age" -"AggregateFunction" "aggThrow" "aggThrowArgMax" "aggThrowArgMin" @@ -48,24 +1001,7 @@ "aggThrowResample" "aggThrowSimpleState" "aggThrowState" -"ALGORITHM" -"ALIAS" -"ALL" -"ALLOWED_LATENESS" "alphaTokens" -"ALTER COLUMN" -"ALTER DATABASE" -"ALTER LIVE VIEW" -"ALTER POLICY" -"ALTER PROFILE" -"ALTER QUOTA" -"ALTER ROLE" -"ALTER ROW POLICY" -"ALTER SETTINGS PROFILE" -"ALTER TABLE" -"ALTER TEMPORARY TABLE" -"ALTER USER" -"ALTER" "analysisOfVariance" "analysisOfVarianceArgMax" "analysisOfVarianceArgMin" @@ -81,8 +1017,6 @@ "analysisOfVarianceResample" "analysisOfVarianceSimpleState" "analysisOfVarianceState" -"AND STDOUT" -"AND" "and" "anova" "anovaArgMax" @@ -99,8 +1033,6 @@ "anovaResample" "anovaSimpleState" "anovaState" -"ANTI" -"ANY" "any" "anyArgMax" "anyArgMin" @@ -206,10 +1138,7 @@ "any_value_respect_nullsResample" "any_value_respect_nullsSimpleState" "any_value_respect_nullsState" -"APPEND" "appendTrailingCharIfAbsent" -"APPLY DELETED MASK" -"APPLY" "approx_top_count" "approx_top_countArgMax" "approx_top_countArgMin" @@ -285,11 +1214,9 @@ "argMinResample" "argMinSimpleState" "argMinState" -"ARRAY JOIN" -"Array" "array" -"arrayAll" "arrayAUC" +"arrayAll" "arrayAvg" "arrayCompact" "arrayConcat" @@ -382,31 +1309,13 @@ "array_concat_aggResample" "array_concat_aggSimpleState" "array_concat_aggState" -"AS" -"ASC" -"ASCENDING" "ascii" "asin" "asinh" -"ASOF" -"ASSUME" "assumeNotNull" -"AST" -"ASYNC" "atan" "atan2" "atanh" -"ATTACH PART" -"ATTACH PARTITION" -"ATTACH POLICY" -"ATTACH PROFILE" -"ATTACH QUOTA" -"ATTACH ROLE" -"ATTACH ROW POLICY" -"ATTACH SETTINGS PROFILE" -"ATTACH USER" -"ATTACH" -"AUTO_INCREMENT" "avg" "avgArgMax" "avgArgMin" @@ -437,8 +1346,6 @@ "avgWeightedResample" "avgWeightedSimpleState" "avgWeightedState" -"AZURE" -"BACKUP" "bagexpansion" "bar" "base58Decode" @@ -447,17 +1354,24 @@ "base64Encode" "base64URLDecode" "base64URLEncode" -"basename" "base_backup" -"BCRYPT_HASH" -"BCRYPT_PASSWORD" -"BEGIN TRANSACTION" -"BETWEEN" -"BIDIRECTIONAL" +"basename" "bin" "bitAnd" "bitCount" "bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" "bitmapAnd" "bitmapAndCardinality" "bitmapAndnot" @@ -479,68 +1393,11 @@ "bitmapXorCardinality" "bitmaskToArray" "bitmaskToList" -"bitNot" -"bitOr" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"bitTest" -"bitTestAll" -"bitTestAny" -"bitXor" -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"BIT_OR" -"BIT_ORArgMax" -"BIT_ORArgMin" -"BIT_ORArray" -"BIT_ORDistinct" -"BIT_ORForEach" -"BIT_ORIf" -"BIT_ORMap" -"BIT_ORMerge" -"BIT_ORNull" -"BIT_OROrDefault" -"BIT_OROrNull" -"BIT_ORResample" -"BIT_ORSimpleState" -"BIT_ORState" -"BIT_XOR" -"BIT_XORArgMax" -"BIT_XORArgMin" -"BIT_XORArray" -"BIT_XORDistinct" -"BIT_XORForEach" -"BIT_XORIf" -"BIT_XORMap" -"BIT_XORMerge" -"BIT_XORNull" -"BIT_XOROrDefault" -"BIT_XOROrNull" -"BIT_XORResample" -"BIT_XORSimpleState" -"BIT_XORState" -"BLAKE3" "blockNumber" "blockSerializedSize" "blockSize" -"BOTH" +"bool" +"boolean" "boundingRatio" "boundingRatioArgMax" "boundingRatioArgMin" @@ -557,18 +1414,14 @@ "boundingRatioSimpleState" "boundingRatioState" "buildId" -"BY" "byteHammingDistance" "byteSize" "byteSlice" "byteSwap" -"CASCADE" -"CASE" "caseWithExpr" "caseWithExpression" "caseWithoutExpr" "caseWithoutExpression" -"CAST" "catboostEvaluate" "categoricalInformationValue" "categoricalInformationValueArgMax" @@ -588,46 +1441,17 @@ "cbrt" "ceil" "ceiling" -"CHANGE" -"CHANGEABLE_IN_READONLY" -"CHANGED" "changeDay" "changeHour" "changeMinute" "changeMonth" "changeSecond" "changeYear" -"CHAR VARYING" -"CHAR" "char" -"CHARACTER LARGE OBJECT" -"CHARACTER VARYING" -"CHARACTER" -"CHARACTER_LENGTH" -"CHAR_LENGTH" -"CHECK ALL TABLES" -"CHECK TABLE" -"CHECK" "cityHash64" "clamp" -"CLEANUP" -"CLEAR COLUMN" -"CLEAR INDEX" -"CLEAR PROJECTION" -"CLEAR STATISTICS" -"CLUSTER" -"CLUSTERS" "cluster_host_ids" -"CN" "coalesce" -"CODEC" -"COLLATE" -"COLUMN" -"COLUMNS" -"COMMENT COLUMN" -"COMMENT" -"COMMIT" -"COMPRESSION" "concat" "concatAssumeInjective" "concatWithSeparator" @@ -635,8 +1459,6 @@ "concat_ws" "connectionId" "connection_id" -"CONST" -"CONSTRAINT" "contingency" "contingencyArgMax" "contingencyArgMin" @@ -813,36 +1635,6 @@ "covarSampStableSimpleState" "covarSampStableState" "covarSampState" -"COVAR_POP" -"COVAR_POPArgMax" -"COVAR_POPArgMin" -"COVAR_POPArray" -"COVAR_POPDistinct" -"COVAR_POPForEach" -"COVAR_POPIf" -"COVAR_POPMap" -"COVAR_POPMerge" -"COVAR_POPNull" -"COVAR_POPOrDefault" -"COVAR_POPOrNull" -"COVAR_POPResample" -"COVAR_POPSimpleState" -"COVAR_POPState" -"COVAR_SAMP" -"COVAR_SAMPArgMax" -"COVAR_SAMPArgMin" -"COVAR_SAMPArray" -"COVAR_SAMPDistinct" -"COVAR_SAMPForEach" -"COVAR_SAMPIf" -"COVAR_SAMPMap" -"COVAR_SAMPMerge" -"COVAR_SAMPNull" -"COVAR_SAMPOrDefault" -"COVAR_SAMPOrNull" -"COVAR_SAMPResample" -"COVAR_SAMPSimpleState" -"COVAR_SAMPState" "cramersV" "cramersVArgMax" "cramersVArgMin" @@ -873,38 +1665,16 @@ "cramersVResample" "cramersVSimpleState" "cramersVState" -"CRC32" -"CRC32IEEE" -"CRC64" -"CREATE POLICY" -"CREATE PROFILE" -"CREATE QUOTA" -"CREATE ROLE" -"CREATE ROW POLICY" -"CREATE SETTINGS PROFILE" -"CREATE TABLE" -"CREATE TEMPORARY TABLE" -"CREATE USER" -"CREATE" -"CROSS" -"CUBE" "curdate" -"CURRENT GRANTS" -"CURRENT QUOTA" -"CURRENT ROLES" -"CURRENT ROW" -"CURRENT TRANSACTION" "currentDatabase" "currentProfiles" "currentRoles" "currentSchemas" -"CURRENTUSER" "currentUser" "current_database" "current_date" "current_schemas" "current_timestamp" -"CURRENT_USER" "current_user" "cutFragment" "cutIPv6" @@ -920,59 +1690,25 @@ "cutToFirstSignificantSubdomainWithWWWRFC" "cutURLParameter" "cutWWW" -"D" "damerauLevenshteinDistance" -"DATA INNER UUID" -"DATA" -"DATABASE" -"DATABASES" -"DATE" -"Date" -"DATEADD" -"DATEDIFF" "dateDiff" "dateName" -"DATESUB" -"DateTime" -"DateTime64" "dateTime64ToSnowflake" "dateTime64ToSnowflakeID" "dateTimeToSnowflake" "dateTimeToSnowflakeID" "dateTrunc" -"DATE_ADD" -"DATE_DIFF" "date_diff" -"DATE_FORMAT" -"DATE_SUB" -"DATE_TRUNC" -"DAY" -"DAYOFMONTH" -"DAYOFWEEK" -"DAYOFYEAR" -"DAYS" -"DD" -"Decimal" -"Decimal128" -"Decimal32" -"Decimal64" "decodeHTMLComponent" "decodeURLComponent" "decodeURLFormComponent" "decodeXMLComponent" "decrypt" -"DEDUPLICATE" -"DEFAULT DATABASE" -"DEFAULT ROLE" -"DEFAULT" "defaultProfiles" "defaultRoles" "defaultValueOfArgumentType" "defaultValueOfTypeName" -"DEFINER" "degrees" -"DELETE WHERE" -"DELETE" "deltaSum" "deltaSumArgMax" "deltaSumArgMin" @@ -1034,13 +1770,6 @@ "dense_rankResample" "dense_rankSimpleState" "dense_rankState" -"DEPENDS ON" -"DESC" -"DESCENDING" -"DESCRIBE" -"DETACH PART" -"DETACH PARTITION" -"DETACH" "detectCharset" "detectLanguage" "detectLanguageMixed" @@ -1060,6 +1789,10 @@ "dictGetFloat64" "dictGetFloat64OrDefault" "dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" "dictGetInt16" "dictGetInt16OrDefault" "dictGetInt32" @@ -1068,10 +1801,6 @@ "dictGetInt64OrDefault" "dictGetInt8" "dictGetInt8OrDefault" -"dictGetIPv4" -"dictGetIPv4OrDefault" -"dictGetIPv6" -"dictGetIPv6OrDefault" "dictGetOrDefault" "dictGetOrNull" "dictGetString" @@ -1087,19 +1816,13 @@ "dictGetUUID" "dictGetUUIDOrDefault" "dictHas" -"DICTIONARIES" -"DICTIONARY" "dictIsIn" -"DISK" "displayName" "distanceL1" "distanceL2" "distanceL2Squared" "distanceLinf" "distanceLp" -"DISTINCT ON" -"DISTINCT" -"DIV" "divide" "divideDecimal" "domain" @@ -1107,30 +1830,12 @@ "domainWithoutWWW" "domainWithoutWWWRFC" "dotProduct" -"DOUBLE_SHA1_HASH" -"DOUBLE_SHA1_PASSWORD" -"DROP COLUMN" -"DROP CONSTRAINT" -"DROP DEFAULT" -"DROP DETACHED PART" -"DROP DETACHED PARTITION" -"DROP INDEX" -"DROP PART" -"DROP PARTITION" -"DROP PROJECTION" -"DROP STATISTICS" -"DROP TABLE" -"DROP TEMPORARY TABLE" -"DROP" "dumpColumnStructure" "dynamicElement" "dynamicType" "e" "editDistance" "editDistanceUTF8" -"ELSE" -"EMPTY AS" -"EMPTY" "empty" "emptyArrayDate" "emptyArrayDateTime" @@ -1146,18 +1851,14 @@ "emptyArrayUInt32" "emptyArrayUInt64" "emptyArrayUInt8" -"ENABLED ROLES" "enabledProfiles" "enabledRoles" "encodeURLComponent" "encodeURLFormComponent" "encodeXMLComponent" "encrypt" -"END" "endsWith" "endsWithUTF8" -"ENFORCED" -"ENGINE" "entropy" "entropyArgMax" "entropyArgMin" @@ -1173,32 +1874,14 @@ "entropyResample" "entropySimpleState" "entropyState" -"Enum" -"Enum16" -"Enum8" -"EPHEMERAL SEQUENTIAL" -"EPHEMERAL" "equals" "erf" "erfc" "errorCodeToName" -"ESTIMATE" "evalMLMethod" -"EVENT" -"EVENTS" -"EVERY" -"EXCEPT DATABASE" -"EXCEPT DATABASES" -"EXCEPT TABLE" -"EXCEPT TABLES" -"EXCEPT" -"EXCHANGE DICTIONARIES" -"EXCHANGE TABLES" -"EXISTS" "exp" "exp10" "exp2" -"EXPLAIN" "exponentialMovingAverage" "exponentialMovingAverageArgMax" "exponentialMovingAverageArgMin" @@ -1274,10 +1957,6 @@ "exponentialTimeDecayedSumResample" "exponentialTimeDecayedSumSimpleState" "exponentialTimeDecayedSumState" -"EXPRESSION" -"EXTENDED" -"EXTERNAL DDL FROM" -"EXTRACT" "extract" "extractAll" "extractAllGroups" @@ -1291,24 +1970,13 @@ "extractURLParameterNames" "extractURLParameters" "factorial" -"FALSE" "farmFingerprint64" "farmHash64" -"FETCH PART" -"FETCH PARTITION" -"FETCH" -"FIELDS" -"FILE" "file" -"FILESYSTEM CACHE" -"FILESYSTEM CACHES" "filesystemAvailable" "filesystemCapacity" "filesystemUnreserved" -"FILTER" -"FINAL" "finalizeAggregation" -"FIRST" "firstLine" "firstSignificantSubdomain" "firstSignificantSubdomainCustom" @@ -1344,7 +2012,6 @@ "first_value_respect_nullsResample" "first_value_respect_nullsSimpleState" "first_value_respect_nullsState" -"FixedString" "flameGraph" "flameGraphArgMax" "flameGraphArgMin" @@ -1362,16 +2029,7 @@ "flameGraphState" "flatten" "flattenTuple" -"Float32" -"Float64" "floor" -"FOLLOWING" -"FOR" -"ForEach" -"FOREIGN KEY" -"FOREIGN" -"FORGET PARTITION" -"FORMAT" "format" "formatDateTime" "formatDateTimeInJodaSyntax" @@ -1385,31 +2043,19 @@ "formatReadableTimeDelta" "formatRow" "formatRowNoNewline" -"FORMAT_BYTES" -"FQDN" "fragment" -"FREEZE" -"FROM INFILE" -"FROM SHARD" -"FROM" "fromDaysSinceYearZero" "fromDaysSinceYearZero32" "fromModifiedJulianDay" "fromModifiedJulianDayOrNull" +"fromUTCTimestamp" "fromUnixTimestamp" "fromUnixTimestamp64Micro" "fromUnixTimestamp64Milli" "fromUnixTimestamp64Nano" "fromUnixTimestampInJodaSyntax" -"fromUTCTimestamp" -"FROM_BASE64" -"FROM_DAYS" -"FROM_UNIXTIME" "from_utc_timestamp" -"FULL" "fullHostName" -"FULLTEXT" -"FUNCTION" "fuzzBits" "gccMurmurHash" "gcd" @@ -1419,11 +2065,11 @@ "generateUUIDv4" "generateUUIDv7" "geoDistance" +"geoToH3" +"geoToS2" "geohashDecode" "geohashEncode" "geohashesInBox" -"geoToH3" -"geoToS2" "getClientHTTPHeader" "getMacro" "getOSKernelVersion" @@ -1432,9 +2078,6 @@ "getSizeOfEnumType" "getSubcolumn" "getTypeSerializationStreams" -"GLOBAL IN" -"GLOBAL NOT IN" -"GLOBAL" "globalIn" "globalInIgnoreSet" "globalNotIn" @@ -1444,16 +2087,11 @@ "globalNullIn" "globalNullInIgnoreSet" "globalVariable" -"GRANT OPTION FOR" -"GRANT" -"GRANTEES" -"GRANULARITY" "greatCircleAngle" "greatCircleDistance" "greater" "greaterOrEquals" "greatest" -"GROUP BY" "groupArray" "groupArrayArgMax" "groupArrayArgMin" @@ -1589,6 +2227,36 @@ "groupBitAndResample" "groupBitAndSimpleState" "groupBitAndState" +"groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" +"groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" "groupBitmap" "groupBitmapAnd" "groupBitmapAndArgMax" @@ -1648,36 +2316,6 @@ "groupBitmapXorResample" "groupBitmapXorSimpleState" "groupBitmapXorState" -"groupBitOr" -"groupBitOrArgMax" -"groupBitOrArgMin" -"groupBitOrArray" -"groupBitOrDistinct" -"groupBitOrForEach" -"groupBitOrIf" -"groupBitOrMap" -"groupBitOrMerge" -"groupBitOrNull" -"groupBitOrOrDefault" -"groupBitOrOrNull" -"groupBitOrResample" -"groupBitOrSimpleState" -"groupBitOrState" -"groupBitXor" -"groupBitXorArgMax" -"groupBitXorArgMin" -"groupBitXorArray" -"groupBitXorDistinct" -"groupBitXorForEach" -"groupBitXorIf" -"groupBitXorMap" -"groupBitXorMerge" -"groupBitXorNull" -"groupBitXorOrDefault" -"groupBitXorOrNull" -"groupBitXorResample" -"groupBitXorSimpleState" -"groupBitXorState" "groupConcat" "groupConcatArgMax" "groupConcatArgMin" @@ -1693,8 +2331,6 @@ "groupConcatResample" "groupConcatSimpleState" "groupConcatState" -"GROUPING SETS" -"GROUPS" "groupUniqArray" "groupUniqArrayArgMax" "groupUniqArrayArgMin" @@ -1725,7 +2361,6 @@ "group_concatResample" "group_concatSimpleState" "group_concatState" -"H" "h3CellAreaM2" "h3CellAreaRads2" "h3Distance" @@ -1753,7 +2388,6 @@ "h3IsPentagon" "h3IsResClassIII" "h3IsValid" -"h3kRing" "h3Line" "h3NumHexagons" "h3PointDistKm" @@ -1766,12 +2400,12 @@ "h3ToParent" "h3ToString" "h3UnidirectionalEdgeIsValid" +"h3kRing" "halfMD5" "has" "hasAll" "hasAny" "hasColumnInTable" -"HASH" "hasSubsequence" "hasSubsequenceCaseInsensitive" "hasSubsequenceCaseInsensitiveUTF8" @@ -1782,11 +2416,7 @@ "hasTokenCaseInsensitive" "hasTokenCaseInsensitiveOrNull" "hasTokenOrNull" -"HAVING" -"HDFS" "hex" -"HH" -"HIERARCHICAL" "hilbertDecode" "hilbertEncode" "histogram" @@ -1808,62 +2438,33 @@ "hop" "hopEnd" "hopStart" -"HOST" "hostName" "hostname" -"HOUR" -"HOURS" -"HTTP" "hypot" -"ID" -"IDENTIFIED" "identity" "idnaDecode" "idnaEncode" -"IF EMPTY" -"IF EXISTS" -"IF NOT EXISTS" "if" "ifNotFinite" "ifNull" -"IGNORE NULLS" "ignore" -"ILIKE" "ilike" -"IN PARTITION" -"IN" "in" -"INDEX" -"INDEXES" +"inIgnoreSet" "indexHint" "indexOf" -"INDICES" -"INET6_ATON" -"INET6_NTOA" -"INET_ATON" -"INET_NTOA" -"INHERIT" -"inIgnoreSet" "initcap" "initcapUTF8" -"initializeAggregation" "initialQueryID" "initial_query_id" -"INJECTIVE" -"INNER" -"INSERT INTO" +"initializeAggregation" "instr" -"Int16" -"Int32" -"Int64" -"Int8" "intDiv" "intDivOrZero" -"INTERPOLATE" -"INTERSECT" -"INTERVAL" -"IntervalDay" -"IntervalHour" +"intExp10" +"intExp2" +"intHash32" +"intHash64" "intervalLengthSum" "intervalLengthSumArgMax" "intervalLengthSumArgMin" @@ -1879,42 +2480,13 @@ "intervalLengthSumResample" "intervalLengthSumSimpleState" "intervalLengthSumState" -"IntervalMinute" -"IntervalMonth" -"IntervalQuarter" -"IntervalSecond" -"IntervalWeek" -"IntervalYear" -"intExp10" -"intExp2" -"intHash32" -"intHash64" -"INTO OUTFILE" -"INVISIBLE" -"INVOKER" -"IP" -"IPv4CIDRToRange" -"IPv4NumToString" -"IPv4NumToStringClassC" -"IPv4StringToNum" -"IPv4StringToNumOrDefault" -"IPv4StringToNumOrNull" -"IPv4ToIPv6" -"IPv6CIDRToRange" -"IPv6NumToString" -"IPv6StringToNum" -"IPv6StringToNumOrDefault" -"IPv6StringToNumOrNull" -"IS NOT DISTINCT FROM" -"IS NOT NULL" -"IS NULL" "isConstant" "isDecimalOverflow" "isFinite" -"isInfinite" "isIPAddressInRange" "isIPv4String" "isIPv6String" +"isInfinite" "isNaN" "isNotDistinctFrom" "isNotNull" @@ -1923,46 +2495,15 @@ "isValidJSON" "isValidUTF8" "isZeroOrNull" -"IS_OBJECT_ID" "jaroSimilarity" "jaroWinklerSimilarity" "javaHash" "javaHashUTF16LE" -"JOIN" "joinGet" "joinGetOrNull" -"JSONArrayLength" -"JSONExtract" -"JSONExtractArrayRaw" -"JSONExtractBool" -"JSONExtractFloat" -"JSONExtractInt" -"JSONExtractKeys" -"JSONExtractKeysAndValues" -"JSONExtractKeysAndValuesRaw" -"JSONExtractRaw" -"JSONExtractString" -"JSONExtractUInt" -"JSONHas" -"JSONKey" -"JSONLength" -"JSONMergePatch" "jsonMergePatch" -"JSONType" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" -"JSON_QUERY" -"JSON_VALUE" "jumpConsistentHash" -"JWT" "kafkaMurmurHash" -"KERBEROS" -"KEY BY" -"KEY" -"KEYED BY" -"KEYS" -"KILL" -"KIND" "kolmogorovSmirnovTest" "kolmogorovSmirnovTestArgMax" "kolmogorovSmirnovTestArgMin" @@ -2011,14 +2552,6 @@ "kurtSampResample" "kurtSampSimpleState" "kurtSampState" -"L1Distance" -"L1Norm" -"L1Normalize" -"L2Distance" -"L2Norm" -"L2Normalize" -"L2SquaredDistance" -"L2SquaredNorm" "lagInFrame" "lagInFrameArgMax" "lagInFrameArgMin" @@ -2034,7 +2567,6 @@ "lagInFrameResample" "lagInFrameSimpleState" "lagInFrameState" -"LARGE OBJECT" "largestTriangleThreeBuckets" "largestTriangleThreeBucketsArgMax" "largestTriangleThreeBucketsArgMin" @@ -2050,8 +2582,6 @@ "largestTriangleThreeBucketsResample" "largestTriangleThreeBucketsSimpleState" "largestTriangleThreeBucketsState" -"LAST" -"LAST_DAY" "last_value" "last_valueArgMax" "last_valueArgMin" @@ -2082,10 +2612,8 @@ "last_value_respect_nullsResample" "last_value_respect_nullsSimpleState" "last_value_respect_nullsState" -"LAYOUT" "lcase" "lcm" -"LDAP" "leadInFrame" "leadInFrameArgMax" "leadInFrameArgMin" @@ -2101,10 +2629,7 @@ "leadInFrameResample" "leadInFrameSimpleState" "leadInFrameState" -"LEADING" "least" -"LEFT ARRAY JOIN" -"LEFT" "left" "leftPad" "leftPadUTF8" @@ -2112,42 +2637,24 @@ "lemmatize" "length" "lengthUTF8" -"LESS THAN" "less" "lessOrEquals" -"LEVEL" "levenshteinDistance" "levenshteinDistanceUTF8" "lgamma" -"LIFETIME" -"LIGHTWEIGHT" -"LIKE" "like" -"LIMIT" -"LINEAR" -"LinfDistance" -"LinfNorm" -"LinfNormalize" -"LIST" -"LIVE" "ln" -"LOCAL" "locate" "log" "log10" "log1p" "log2" "logTrace" -"LowCardinality" "lowCardinalityIndices" "lowCardinalityKeys" "lower" "lowerUTF8" "lpad" -"LpDistance" -"LpNorm" -"LpNormalize" -"LTRIM" "ltrim" "lttb" "lttbArgMax" @@ -2164,10 +2671,6 @@ "lttbResample" "lttbSimpleState" "lttbState" -"M" -"MACNumToString" -"MACStringToNum" -"MACStringToOUI" "makeDate" "makeDate32" "makeDateTime" @@ -2208,18 +2711,8 @@ "mapSubtract" "mapUpdate" "mapValues" -"MAP_FROM_ARRAYS" -"MATCH" "match" -"MATERIALIZE COLUMN" -"MATERIALIZE INDEX" -"MATERIALIZE PROJECTION" -"MATERIALIZE STATISTICS" -"MATERIALIZE TTL" -"MATERIALIZE" "materialize" -"MATERIALIZED" -"MAX" "max" "max2" "maxArgMax" @@ -2281,9 +2774,6 @@ "maxResample" "maxSimpleState" "maxState" -"MCS" -"MD4" -"MD5" "meanZTest" "meanZTestArgMax" "meanZTestArgMin" @@ -2524,19 +3014,8 @@ "medianTimingWeightedResample" "medianTimingWeightedSimpleState" "medianTimingWeightedState" -"MEMORY" -"Merge" -"MERGES" -"METRICS INNER UUID" -"METRICS" "metroHash64" -"MI" -"MICROSECOND" -"MICROSECONDS" "mid" -"MILLISECOND" -"MILLISECONDS" -"MIN" "min" "min2" "minArgMax" @@ -2572,37 +3051,15 @@ "minSimpleState" "minState" "minus" -"MINUTE" -"MINUTES" "mismatches" -"MM" -"MOD" "mod" -"MODIFY COLUMN" -"MODIFY COMMENT" -"MODIFY DEFINER" -"MODIFY ORDER BY" -"MODIFY QUERY" -"MODIFY REFRESH" -"MODIFY SAMPLE BY" -"MODIFY SETTING" -"MODIFY SQL SECURITY" -"MODIFY STATISTICS" -"MODIFY TTL" -"MODIFY" "modulo" "moduloLegacy" "moduloOrZero" -"MONTH" "monthName" -"MONTHS" "mortonDecode" "mortonEncode" -"MOVE PART" -"MOVE PARTITION" -"MOVE" "movingXXX" -"MS" "multiFuzzyMatchAllIndices" "multiFuzzyMatchAny" "multiFuzzyMatchAnyIndex" @@ -2610,8 +3067,6 @@ "multiMatchAllIndices" "multiMatchAny" "multiMatchAnyIndex" -"multiply" -"multiplyDecimal" "multiSearchAllPositions" "multiSearchAllPositionsCaseInsensitive" "multiSearchAllPositionsCaseInsensitiveUTF8" @@ -2628,23 +3083,17 @@ "multiSearchFirstPositionCaseInsensitive" "multiSearchFirstPositionCaseInsensitiveUTF8" "multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" "murmurHash2_32" "murmurHash2_64" "murmurHash3_128" "murmurHash3_32" "murmurHash3_64" -"MUTATION" -"N" -"NAME" -"NAMED COLLECTION" -"NANOSECOND" -"NANOSECONDS" "negate" "neighbor" -"Nested" "nested" "netloc" -"NEXT" "ngramDistance" "ngramDistanceCaseInsensitive" "ngramDistanceCaseInsensitiveUTF8" @@ -2657,7 +3106,6 @@ "ngramMinHashCaseInsensitive" "ngramMinHashCaseInsensitiveUTF8" "ngramMinHashUTF8" -"ngrams" "ngramSearch" "ngramSearchCaseInsensitive" "ngramSearchCaseInsensitiveUTF8" @@ -2666,10 +3114,7 @@ "ngramSimHashCaseInsensitive" "ngramSimHashCaseInsensitiveUTF8" "ngramSimHashUTF8" -"NO ACTION" -"NO DELAY" -"NO LIMITS" -"NONE" +"ngrams" "nonNegativeDerivative" "nonNegativeDerivativeArgMax" "nonNegativeDerivativeArgMin" @@ -2685,8 +3130,11 @@ "nonNegativeDerivativeResample" "nonNegativeDerivativeSimpleState" "nonNegativeDerivativeState" -"normalizedQueryHash" -"normalizedQueryHashKeepNames" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" "normalizeL1" "normalizeL2" "normalizeLinf" @@ -2697,23 +3145,17 @@ "normalizeUTF8NFD" "normalizeUTF8NFKC" "normalizeUTF8NFKD" -"normL1" -"normL2" -"normL2Squared" -"normLinf" -"normLp" -"NOT BETWEEN" -"NOT IDENTIFIED" -"NOT ILIKE" -"NOT IN" -"NOT KEYED" -"NOT LIKE" -"NOT OVERRIDABLE" -"NOT" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" "not" "notEmpty" "notEquals" -"Nothing" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" "nothing" "nothingArgMax" "nothingArgMin" @@ -2758,17 +3200,9 @@ "nothingUInt64Resample" "nothingUInt64SimpleState" "nothingUInt64State" -"notILike" -"notIn" -"notInIgnoreSet" -"notLike" -"notNullIn" -"notNullInIgnoreSet" "now" "now64" "nowInBlock" -"NO_PASSWORD" -"NS" "nth_value" "nth_valueArgMax" "nth_valueArgMin" @@ -2799,28 +3233,10 @@ "ntileResample" "ntileSimpleState" "ntileState" -"NULL" -"Null" -"Nullable" "nullIf" "nullIn" "nullInIgnoreSet" -"NULLS" -"OCTET_LENGTH" -"OFFSET" -"ON DELETE" -"ON UPDATE" -"ON VOLUME" -"ON" -"ONLY" -"OPTIMIZE TABLE" -"OR REPLACE" -"OR" "or" -"ORDER BY" -"OUTER" -"OVER" -"OVERRIDABLE" "parseDateTime" "parseDateTime32BestEffort" "parseDateTime32BestEffortOrNull" @@ -2846,15 +3262,8 @@ "parseReadableSizeOrNull" "parseReadableSizeOrZero" "parseTimeDelta" -"PART" -"PARTIAL" -"PARTITION BY" -"PARTITION" "partitionID" "partitionId" -"PARTITIONS" -"PART_MOVE_TO_SHARD" -"PASTE" "path" "pathFull" "percentRank" @@ -2887,15 +3296,7 @@ "percent_rankResample" "percent_rankSimpleState" "percent_rankState" -"PERIODIC REFRESH" -"PERMANENTLY" -"PERMISSIVE" -"PERSISTENT SEQUENTIAL" -"PERSISTENT" "pi" -"PIPELINE" -"PLAINTEXT_PASSWORD" -"PLAN" "plus" "pmod" "pointInEllipses" @@ -2916,7 +3317,6 @@ "polygonsUnionSpherical" "polygonsWithinCartesian" "polygonsWithinSpherical" -"POPULATE" "port" "portRFC" "position" @@ -2927,23 +3327,11 @@ "positive_modulo" "pow" "power" -"PRECEDING" -"PRECISION" -"PREWHERE" -"PRIMARY KEY" -"PRIMARY" "printf" -"PROFILE" -"PROJECTION" "proportionsZTest" -"Protobuf" "protocol" -"PULL" "punycodeDecode" "punycodeEncode" -"Q" -"QQ" -"QUALIFY" "quantile" "quantileArgMax" "quantileArgMin" @@ -3137,6 +3525,68 @@ "quantileOrDefault" "quantileOrNull" "quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" "quantiles" "quantilesArgMax" "quantilesArgMin" @@ -3309,7 +3759,6 @@ "quantilesGKSimpleState" "quantilesGKState" "quantilesIf" -"quantileSimpleState" "quantilesInterpolatedWeighted" "quantilesInterpolatedWeightedArgMax" "quantilesInterpolatedWeightedArgMin" @@ -3333,7 +3782,6 @@ "quantilesResample" "quantilesSimpleState" "quantilesState" -"quantileState" "quantilesTDigest" "quantilesTDigestArgMax" "quantilesTDigestArgMin" @@ -3394,75 +3842,10 @@ "quantilesTimingWeightedResample" "quantilesTimingWeightedSimpleState" "quantilesTimingWeightedState" -"quantileTDigest" -"quantileTDigestArgMax" -"quantileTDigestArgMin" -"quantileTDigestArray" -"quantileTDigestDistinct" -"quantileTDigestForEach" -"quantileTDigestIf" -"quantileTDigestMap" -"quantileTDigestMerge" -"quantileTDigestNull" -"quantileTDigestOrDefault" -"quantileTDigestOrNull" -"quantileTDigestResample" -"quantileTDigestSimpleState" -"quantileTDigestState" -"quantileTDigestWeighted" -"quantileTDigestWeightedArgMax" -"quantileTDigestWeightedArgMin" -"quantileTDigestWeightedArray" -"quantileTDigestWeightedDistinct" -"quantileTDigestWeightedForEach" -"quantileTDigestWeightedIf" -"quantileTDigestWeightedMap" -"quantileTDigestWeightedMerge" -"quantileTDigestWeightedNull" -"quantileTDigestWeightedOrDefault" -"quantileTDigestWeightedOrNull" -"quantileTDigestWeightedResample" -"quantileTDigestWeightedSimpleState" -"quantileTDigestWeightedState" -"quantileTiming" -"quantileTimingArgMax" -"quantileTimingArgMin" -"quantileTimingArray" -"quantileTimingDistinct" -"quantileTimingForEach" -"quantileTimingIf" -"quantileTimingMap" -"quantileTimingMerge" -"quantileTimingNull" -"quantileTimingOrDefault" -"quantileTimingOrNull" -"quantileTimingResample" -"quantileTimingSimpleState" -"quantileTimingState" -"quantileTimingWeighted" -"quantileTimingWeightedArgMax" -"quantileTimingWeightedArgMin" -"quantileTimingWeightedArray" -"quantileTimingWeightedDistinct" -"quantileTimingWeightedForEach" -"quantileTimingWeightedIf" -"quantileTimingWeightedMap" -"quantileTimingWeightedMerge" -"quantileTimingWeightedNull" -"quantileTimingWeightedOrDefault" -"quantileTimingWeightedOrNull" -"quantileTimingWeightedResample" -"quantileTimingWeightedSimpleState" -"quantileTimingWeightedState" -"QUARTER" -"QUARTERS" -"QUERY TREE" -"QUERY" "queryID" "queryString" "queryStringAndFragment" "query_id" -"QUOTA" "radians" "rand" "rand32" @@ -3477,16 +3860,13 @@ "randLogNormal" "randNegativeBinomial" "randNormal" -"randomFixedString" -"RANDOMIZE FOR" -"RANDOMIZED" -"randomPrintableASCII" -"randomString" -"randomStringUTF8" "randPoisson" "randStudentT" "randUniform" -"RANGE" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" "range" "rank" "rankArgMax" @@ -3518,24 +3898,14 @@ "rankResample" "rankSimpleState" "rankState" -"READONLY" "readWKTLineString" "readWKTMultiLineString" "readWKTMultiPolygon" "readWKTPoint" "readWKTPolygon" "readWKTRing" -"REALM" -"RECOMPRESS" -"RECURSIVE" -"REFERENCES" -"REFRESH" -"REGEXP" "regexpExtract" "regexpQuoteMeta" -"REGEXP_EXTRACT" -"REGEXP_MATCHES" -"REGEXP_REPLACE" "regionHierarchy" "regionIn" "regionToArea" @@ -3566,31 +3936,13 @@ "reinterpretAsUInt64" "reinterpretAsUInt8" "reinterpretAsUUID" -"REMOVE SAMPLE BY" -"REMOVE TTL" -"REMOVE" -"RENAME COLUMN" -"RENAME DATABASE" -"RENAME DICTIONARY" -"RENAME TABLE" -"RENAME TO" -"RENAME" "repeat" -"REPLACE PARTITION" -"REPLACE" "replace" "replaceAll" "replaceOne" "replaceRegexpAll" "replaceRegexpOne" "replicate" -"Resample" -"RESET SETTING" -"RESPECT NULLS" -"RESTORE" -"RESTRICT" -"RESTRICTIVE" -"RESUME" "retention" "retentionArgMax" "retentionArgMin" @@ -3609,24 +3961,18 @@ "reverse" "reverseUTF8" "revision" -"REVOKE" -"RIGHT" "right" "rightPad" "rightPadUTF8" "rightUTF8" -"ROLLBACK" -"ROLLUP" "round" "roundAge" "roundBankers" "roundDown" "roundDuration" "roundToExp2" -"ROW" "rowNumberInAllBlocks" "rowNumberInBlock" -"ROWS" "row_number" "row_numberArgMax" "row_numberArgMin" @@ -3643,13 +3989,11 @@ "row_numberSimpleState" "row_numberState" "rpad" -"RTRIM" "rtrim" "runningAccumulate" "runningConcurrency" "runningDifference" "runningDifferenceStartingWithFirstValue" -"S" "s2CapContains" "s2CapUnion" "s2CellsIntersect" @@ -3659,18 +4003,7 @@ "s2RectIntersection" "s2RectUnion" "s2ToGeo" -"S3" -"SALT" -"SAMPLE BY" -"SAMPLE" -"SAN" "scalarProduct" -"SCHEMA" -"SCHEME" -"SECOND" -"SECONDS" -"SELECT" -"SEMI" "sequenceCount" "sequenceCountArgMax" "sequenceCountArgMin" @@ -3719,43 +4052,14 @@ "seriesDecomposeSTL" "seriesOutliersDetectTukey" "seriesPeriodDetectFFT" -"SERVER" "serverTimeZone" "serverTimezone" "serverUUID" -"SET DEFAULT ROLE" -"SET DEFAULT" -"SET FAKE TIME" -"SET NULL" -"SET ROLE DEFAULT" -"SET ROLE" -"SET TRANSACTION SNAPSHOT" -"SET" -"SETTINGS" -"SHA1" -"SHA224" -"SHA256" -"SHA256_HASH" -"SHA256_PASSWORD" -"SHA384" -"SHA512" -"SHA512_256" "shardCount" "shardNum" -"SHOW ACCESS" -"SHOW CREATE" -"SHOW ENGINES" -"SHOW FUNCTIONS" -"SHOW GRANTS" -"SHOW PRIVILEGES" -"SHOW PROCESSLIST" -"SHOW SETTING" -"SHOW" "showCertificate" "sigmoid" "sign" -"SIGNED" -"SIMPLE" "simpleJSONExtractBool" "simpleJSONExtractFloat" "simpleJSONExtractInt" @@ -3838,39 +4142,37 @@ "snowflakeToDateTime" "snowflakeToDateTime64" "soundex" -"SOURCE" "space" "sparkBar" -"sparkbar" "sparkBarArgMax" -"sparkbarArgMax" "sparkBarArgMin" -"sparkbarArgMin" "sparkBarArray" -"sparkbarArray" "sparkBarDistinct" -"sparkbarDistinct" "sparkBarForEach" -"sparkbarForEach" "sparkBarIf" -"sparkbarIf" "sparkBarMap" -"sparkbarMap" "sparkBarMerge" -"sparkbarMerge" "sparkBarNull" -"sparkbarNull" "sparkBarOrDefault" -"sparkbarOrDefault" "sparkBarOrNull" -"sparkbarOrNull" "sparkBarResample" -"sparkbarResample" "sparkBarSimpleState" -"sparkbarSimpleState" "sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" "sparkbarState" -"SPATIAL" "splitByAlpha" "splitByChar" "splitByNonAlpha" @@ -3880,31 +4182,9 @@ "sqid" "sqidDecode" "sqidEncode" -"SQL SECURITY" -"SQL_TSI_DAY" -"SQL_TSI_HOUR" -"SQL_TSI_MICROSECOND" -"SQL_TSI_MILLISECOND" -"SQL_TSI_MINUTE" -"SQL_TSI_MONTH" -"SQL_TSI_NANOSECOND" -"SQL_TSI_QUARTER" -"SQL_TSI_SECOND" -"SQL_TSI_WEEK" -"SQL_TSI_YEAR" "sqrt" -"SS" -"SSH_KEY" -"SSL_CERTIFICATE" -"START TRANSACTION" "startsWith" "startsWithUTF8" -"State" -"STATISTICS" -"STD" -"STDArgMax" -"STDArgMin" -"STDArray" "stddevPop" "stddevPopArgMax" "stddevPopArgMin" @@ -3965,49 +4245,7 @@ "stddevSampStableSimpleState" "stddevSampStableState" "stddevSampState" -"STDDEV_POP" -"STDDEV_POPArgMax" -"STDDEV_POPArgMin" -"STDDEV_POPArray" -"STDDEV_POPDistinct" -"STDDEV_POPForEach" -"STDDEV_POPIf" -"STDDEV_POPMap" -"STDDEV_POPMerge" -"STDDEV_POPNull" -"STDDEV_POPOrDefault" -"STDDEV_POPOrNull" -"STDDEV_POPResample" -"STDDEV_POPSimpleState" -"STDDEV_POPState" -"STDDEV_SAMP" -"STDDEV_SAMPArgMax" -"STDDEV_SAMPArgMin" -"STDDEV_SAMPArray" -"STDDEV_SAMPDistinct" -"STDDEV_SAMPForEach" -"STDDEV_SAMPIf" -"STDDEV_SAMPMap" -"STDDEV_SAMPMerge" -"STDDEV_SAMPNull" -"STDDEV_SAMPOrDefault" -"STDDEV_SAMPOrNull" -"STDDEV_SAMPResample" -"STDDEV_SAMPSimpleState" -"STDDEV_SAMPState" -"STDDistinct" -"STDForEach" -"STDIf" -"STDMap" -"STDMerge" -"STDNull" -"STDOrDefault" -"STDOrNull" -"STDResample" -"STDSimpleState" -"STDState" "stem" -"STEP" "stochasticLinearRegression" "stochasticLinearRegressionArgMax" "stochasticLinearRegressionArgMin" @@ -4038,17 +4276,13 @@ "stochasticLogisticRegressionResample" "stochasticLogisticRegressionSimpleState" "stochasticLogisticRegressionState" -"STORAGE" -"STRICT" -"STRICTLY_ASCENDING" -"String" +"str_to_date" +"str_to_map" "stringJaccardIndex" "stringJaccardIndexUTF8" "stringToH3" "structureToCapnProtoSchema" "structureToProtobufSchema" -"str_to_date" -"str_to_map" "studentTTest" "studentTTestArgMax" "studentTTestArgMin" @@ -4066,16 +4300,11 @@ "studentTTestState" "subBitmap" "subDate" -"SUBPARTITION BY" -"SUBPARTITION" -"SUBPARTITIONS" "substr" -"SUBSTRING" "substring" "substringIndex" "substringIndexUTF8" "substringUTF8" -"SUBSTRING_INDEX" "subtractDays" "subtractHours" "subtractInterval" @@ -4157,21 +4386,6 @@ "sumMapFilteredWithOverflowResample" "sumMapFilteredWithOverflowSimpleState" "sumMapFilteredWithOverflowState" -"sumMappedArrays" -"sumMappedArraysArgMax" -"sumMappedArraysArgMin" -"sumMappedArraysArray" -"sumMappedArraysDistinct" -"sumMappedArraysForEach" -"sumMappedArraysIf" -"sumMappedArraysMap" -"sumMappedArraysMerge" -"sumMappedArraysNull" -"sumMappedArraysOrDefault" -"sumMappedArraysOrNull" -"sumMappedArraysResample" -"sumMappedArraysSimpleState" -"sumMappedArraysState" "sumMapWithOverflow" "sumMapWithOverflowArgMax" "sumMapWithOverflowArgMin" @@ -4187,6 +4401,21 @@ "sumMapWithOverflowResample" "sumMapWithOverflowSimpleState" "sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" "sumMerge" "sumNull" "sumOrDefault" @@ -4209,24 +4438,11 @@ "sumWithOverflowResample" "sumWithOverflowSimpleState" "sumWithOverflowState" -"SUSPEND" -"SVG" "svg" -"SYNC" "synonyms" -"SYNTAX" -"SYSTEM" -"TABLE OVERRIDE" -"TABLE" -"TABLES" -"TAGS INNER UUID" -"TAGS" "tan" "tanh" "tcpPort" -"TEMPORARY TABLE" -"TEMPORARY" -"TEST" "tgamma" "theilsU" "theilsUArgMax" @@ -4243,34 +4459,20 @@ "theilsUResample" "theilsUSimpleState" "theilsUState" -"THEN" "throwIf" "tid" "timeDiff" "timeSlot" "timeSlots" -"TIMESTAMP" -"timestamp" -"TIMESTAMPADD" -"TIMESTAMPDIFF" -"timestampDiff" -"TIMESTAMPSUB" -"TIMESTAMP_ADD" -"TIMESTAMP_DIFF" -"timestamp_diff" -"TIMESTAMP_SUB" "timeZone" -"timezone" "timeZoneOf" -"timezoneOf" "timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" "timezoneOffset" -"TO DISK" -"TO INNER UUID" -"TO SHARD" -"TO TABLE" -"TO VOLUME" -"TO" "toBool" "toColumnTypeName" "toDate" @@ -4290,7 +4492,6 @@ "toDateTimeOrDefault" "toDateTimeOrNull" "toDateTimeOrZero" -"today" "toDayOfMonth" "toDayOfWeek" "toDayOfYear" @@ -4322,6 +4523,16 @@ "toFloat64OrNull" "toFloat64OrZero" "toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" "toInt128" "toInt128OrDefault" "toInt128OrNull" @@ -4357,18 +4568,7 @@ "toIntervalSecond" "toIntervalWeek" "toIntervalYear" -"toIPv4" -"toIPv4OrDefault" -"toIPv4OrNull" -"toIPv4OrZero" -"toIPv6" -"toIPv6OrDefault" -"toIPv6OrNull" -"toIPv6OrZero" -"toISOWeek" -"toISOYear" "toJSONString" -"tokens" "toLastDayOfMonth" "toLastDayOfWeek" "toLowCardinality" @@ -4379,7 +4579,82 @@ "toMonday" "toMonth" "toNullable" -"TOP" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" "topK" "topKArgMax" "topKArgMin" @@ -4412,102 +4687,17 @@ "topKWeightedState" "topLevelDomain" "topLevelDomainRFC" -"toQuarter" -"toRelativeDayNum" -"toRelativeHourNum" -"toRelativeMinuteNum" -"toRelativeMonthNum" -"toRelativeQuarterNum" -"toRelativeSecondNum" -"toRelativeWeekNum" -"toRelativeYearNum" -"toSecond" -"toStartOfDay" -"toStartOfFifteenMinutes" -"toStartOfFiveMinute" -"toStartOfFiveMinutes" -"toStartOfHour" -"toStartOfInterval" -"toStartOfISOYear" -"toStartOfMicrosecond" -"toStartOfMillisecond" -"toStartOfMinute" -"toStartOfMonth" -"toStartOfNanosecond" -"toStartOfQuarter" -"toStartOfSecond" -"toStartOfTenMinutes" -"toStartOfWeek" -"toStartOfYear" -"toString" -"toStringCutToZero" -"TOTALS" -"toTime" -"toTimeZone" -"toTimezone" -"toTypeName" -"toUInt128" -"toUInt128OrDefault" -"toUInt128OrNull" -"toUInt128OrZero" -"toUInt16" -"toUInt16OrDefault" -"toUInt16OrNull" -"toUInt16OrZero" -"toUInt256" -"toUInt256OrDefault" -"toUInt256OrNull" -"toUInt256OrZero" -"toUInt32" -"toUInt32OrDefault" -"toUInt32OrNull" -"toUInt32OrZero" -"toUInt64" -"toUInt64OrDefault" -"toUInt64OrNull" -"toUInt64OrZero" -"toUInt8" -"toUInt8OrDefault" -"toUInt8OrNull" -"toUInt8OrZero" -"toUnixTimestamp" -"toUnixTimestamp64Micro" -"toUnixTimestamp64Milli" -"toUnixTimestamp64Nano" -"toUTCTimestamp" -"toUUID" -"toUUIDOrDefault" -"toUUIDOrNull" -"toUUIDOrZero" -"toValidUTF8" -"toWeek" -"toYear" -"toYearWeek" -"toYYYYMM" -"toYYYYMMDD" -"toYYYYMMDDhhmmss" -"TO_BASE64" -"TO_DAYS" -"TO_UNIXTIME" -"to_utc_timestamp" -"TRACKING ONLY" -"TRAILING" -"TRANSACTION" "transactionID" "transactionLatestSnapshot" "transactionOldestSnapshot" "transform" "translate" "translateUTF8" -"TRIGGER" -"TRIM" "trim" "trimBoth" "trimLeft" "trimRight" -"TRUE" "trunc" -"TRUNCATE" "truncate" "tryBase58Decode" "tryBase64Decode" @@ -4515,11 +4705,9 @@ "tryDecrypt" "tryIdnaEncode" "tryPunycodeDecode" -"TTL" "tumble" "tumbleEnd" "tumbleStart" -"Tuple" "tuple" "tupleConcat" "tupleDivide" @@ -4539,20 +4727,9 @@ "tupleNegate" "tuplePlus" "tupleToNameValuePairs" -"TYPE" -"TYPEOF" "ucase" -"UInt16" -"UInt32" -"UInt64" -"UInt8" -"ULIDStringToDateTime" "unbin" -"UNBOUNDED" -"UNDROP" -"UNFREEZE" "unhex" -"UNION" "uniq" "uniqArgMax" "uniqArgMin" @@ -4646,7 +4823,6 @@ "uniqThetaSimpleState" "uniqThetaState" "uniqThetaUnion" -"UNIQUE" "uniqUpTo" "uniqUpToArgMax" "uniqUpToArgMin" @@ -4662,31 +4838,11 @@ "uniqUpToResample" "uniqUpToSimpleState" "uniqUpToState" -"UNSET FAKE TIME" -"UNSIGNED" -"UPDATE" "upper" "upperUTF8" "uptime" -"URL" -"URLHash" -"URLHierarchy" -"URLPathHierarchy" -"USE" "user" -"USING" -"UTCTimestamp" -"UTC_timestamp" -"UUID" -"UUIDNumToString" -"UUIDStringToNum" -"UUIDToNum" -"UUIDv7ToDateTime" -"VALID UNTIL" "validateNestedArraySizes" -"VALUES" -"variantElement" -"variantType" "varPop" "varPopArgMax" "varPopArgMin" @@ -4747,42 +4903,11 @@ "varSampStableSimpleState" "varSampStableState" "varSampState" -"VARYING" -"VAR_POP" -"VAR_POPArgMax" -"VAR_POPArgMin" -"VAR_POPArray" -"VAR_POPDistinct" -"VAR_POPForEach" -"VAR_POPIf" -"VAR_POPMap" -"VAR_POPMerge" -"VAR_POPNull" -"VAR_POPOrDefault" -"VAR_POPOrNull" -"VAR_POPResample" -"VAR_POPSimpleState" -"VAR_POPState" -"VAR_SAMP" -"VAR_SAMPArgMax" -"VAR_SAMPArgMin" -"VAR_SAMPArray" -"VAR_SAMPDistinct" -"VAR_SAMPForEach" -"VAR_SAMPIf" -"VAR_SAMPMap" -"VAR_SAMPMerge" -"VAR_SAMPNull" -"VAR_SAMPOrDefault" -"VAR_SAMPOrNull" -"VAR_SAMPResample" -"VAR_SAMPSimpleState" -"VAR_SAMPState" +"variantElement" +"variantType" "vectorDifference" "vectorSum" "version" -"VIEW" -"VISIBLE" "visibleWidth" "visitParamExtractBool" "visitParamExtractFloat" @@ -4791,11 +4916,7 @@ "visitParamExtractString" "visitParamExtractUInt" "visitParamHas" -"WATCH" -"WATERMARK" -"WEEK" "week" -"WEEKS" "welchTTest" "welchTTestArgMax" "welchTTestArgMin" @@ -4811,11 +4932,8 @@ "welchTTestResample" "welchTTestSimpleState" "welchTTestState" -"WHEN" -"WHERE" "widthBucket" "width_bucket" -"WINDOW" "windowFunnel" "windowFunnelArgMax" "windowFunnelArgMin" @@ -4832,16 +4950,6 @@ "windowFunnelSimpleState" "windowFunnelState" "windowID" -"WITH ADMIN OPTION" -"WITH CHECK" -"WITH FILL" -"WITH GRANT OPTION" -"WITH NAME" -"WITH REPLACE OPTION" -"WITH TIES" -"WITH" -"WITH_ITEMINDEX" -"WK" "wkt" "wordShingleMinHash" "wordShingleMinHashArg" @@ -4855,31 +4963,12 @@ "wordShingleSimHashCaseInsensitive" "wordShingleSimHashCaseInsensitiveUTF8" "wordShingleSimHashUTF8" -"WRITABLE" -"WW" "wyHash64" "xor" -"xxh3" "xxHash32" "xxHash64" +"xxh3" "yandexConsistentHash" -"YEAR" -"YEARS" "yearweek" "yesterday" -"YY" -"YYYY" -"YYYYMMDDhhmmssToDateTime" -"YYYYMMDDhhmmssToDateTime64" -"YYYYMMDDToDate" -"YYYYMMDDToDate32" -"ZKPATH" "zookeeperSessionUptime" -"_CAST" -"__actionName" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" -"__bitSwapLastTwo" -"__bitWrapperFunc" -"__getScalar" -"__scalarSubqueryResult" diff --git a/tests/fuzz/dictionaries/datatypes.dict b/tests/fuzz/dictionaries/datatypes.dict index e562595fb67..797905203b2 100644 --- a/tests/fuzz/dictionaries/datatypes.dict +++ b/tests/fuzz/dictionaries/datatypes.dict @@ -1,4283 +1,137 @@ -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"BIT_OR" -"BIT_ORArgMax" -"BIT_ORArgMin" -"BIT_ORArray" -"BIT_ORDistinct" -"BIT_ORForEach" -"BIT_ORIf" -"BIT_ORMap" -"BIT_ORMerge" -"BIT_ORNull" -"BIT_OROrDefault" -"BIT_OROrNull" -"BIT_ORResample" -"BIT_ORSimpleState" -"BIT_ORState" -"BIT_XOR" -"BIT_XORArgMax" -"BIT_XORArgMin" -"BIT_XORArray" -"BIT_XORDistinct" -"BIT_XORForEach" -"BIT_XORIf" -"BIT_XORMap" -"BIT_XORMerge" -"BIT_XORNull" -"BIT_XOROrDefault" -"BIT_XOROrNull" -"BIT_XORResample" -"BIT_XORSimpleState" -"BIT_XORState" -"BLAKE3" -"CAST" -"CHARACTER_LENGTH" -"CHAR_LENGTH" -"COVAR_POP" -"COVAR_POPArgMax" -"COVAR_POPArgMin" -"COVAR_POPArray" -"COVAR_POPDistinct" -"COVAR_POPForEach" -"COVAR_POPIf" -"COVAR_POPMap" -"COVAR_POPMerge" -"COVAR_POPNull" -"COVAR_POPOrDefault" -"COVAR_POPOrNull" -"COVAR_POPResample" -"COVAR_POPSimpleState" -"COVAR_POPState" -"COVAR_SAMP" -"COVAR_SAMPArgMax" -"COVAR_SAMPArgMin" -"COVAR_SAMPArray" -"COVAR_SAMPDistinct" -"COVAR_SAMPForEach" -"COVAR_SAMPIf" -"COVAR_SAMPMap" -"COVAR_SAMPMerge" -"COVAR_SAMPNull" -"COVAR_SAMPOrDefault" -"COVAR_SAMPOrNull" -"COVAR_SAMPResample" -"COVAR_SAMPSimpleState" -"COVAR_SAMPState" -"CRC32" -"CRC32IEEE" -"CRC64" -"DATABASE" -"DATE" -"DATE_DIFF" -"DATE_FORMAT" -"DATE_TRUNC" -"DAY" -"DAYOFMONTH" -"DAYOFWEEK" -"DAYOFYEAR" -"FORMAT_BYTES" -"FQDN" -"FROM_BASE64" -"FROM_DAYS" -"FROM_UNIXTIME" -"HOUR" -"INET6_ATON" -"INET6_NTOA" -"INET_ATON" -"INET_NTOA" -"IPv4CIDRToRange" -"IPv4NumToString" -"IPv4NumToStringClassC" -"IPv4StringToNum" -"IPv4StringToNumOrDefault" -"IPv4StringToNumOrNull" -"IPv4ToIPv6" -"IPv6CIDRToRange" -"IPv6NumToString" -"IPv6StringToNum" -"IPv6StringToNumOrDefault" -"IPv6StringToNumOrNull" -"JSONArrayLength" -"JSONExtract" -"JSONExtractArrayRaw" -"JSONExtractBool" -"JSONExtractFloat" -"JSONExtractInt" -"JSONExtractKeys" -"JSONExtractKeysAndValues" -"JSONExtractKeysAndValuesRaw" -"JSONExtractRaw" -"JSONExtractString" -"JSONExtractUInt" -"JSONHas" -"JSONKey" -"JSONLength" -"JSONMergePatch" -"JSONType" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" -"JSON_QUERY" -"JSON_VALUE" -"L1Distance" -"L1Norm" -"L1Normalize" -"L2Distance" -"L2Norm" -"L2Normalize" -"L2SquaredDistance" -"L2SquaredNorm" -"LAST_DAY" -"LinfDistance" -"LinfNorm" -"LinfNormalize" -"LpDistance" -"LpNorm" -"LpNormalize" -"MACNumToString" -"MACStringToNum" -"MACStringToOUI" -"MAP_FROM_ARRAYS" -"MD4" -"MD5" -"MILLISECOND" -"MINUTE" -"MONTH" -"OCTET_LENGTH" -"QUARTER" -"REGEXP_EXTRACT" -"REGEXP_MATCHES" -"REGEXP_REPLACE" -"SCHEMA" -"SECOND" -"SHA1" -"SHA224" -"SHA256" -"SHA384" -"SHA512" -"SHA512_256" -"STD" -"STDArgMax" -"STDArgMin" -"STDArray" -"STDDEV_POP" -"STDDEV_POPArgMax" -"STDDEV_POPArgMin" -"STDDEV_POPArray" -"STDDEV_POPDistinct" -"STDDEV_POPForEach" -"STDDEV_POPIf" -"STDDEV_POPMap" -"STDDEV_POPMerge" -"STDDEV_POPNull" -"STDDEV_POPOrDefault" -"STDDEV_POPOrNull" -"STDDEV_POPResample" -"STDDEV_POPSimpleState" -"STDDEV_POPState" -"STDDEV_SAMP" -"STDDEV_SAMPArgMax" -"STDDEV_SAMPArgMin" -"STDDEV_SAMPArray" -"STDDEV_SAMPDistinct" -"STDDEV_SAMPForEach" -"STDDEV_SAMPIf" -"STDDEV_SAMPMap" -"STDDEV_SAMPMerge" -"STDDEV_SAMPNull" -"STDDEV_SAMPOrDefault" -"STDDEV_SAMPOrNull" -"STDDEV_SAMPResample" -"STDDEV_SAMPSimpleState" -"STDDEV_SAMPState" -"STDDistinct" -"STDForEach" -"STDIf" -"STDMap" -"STDMerge" -"STDNull" -"STDOrDefault" -"STDOrNull" -"STDResample" -"STDSimpleState" -"STDState" -"SUBSTRING_INDEX" -"SVG" -"TIMESTAMP_DIFF" -"TO_BASE64" -"TO_DAYS" -"TO_UNIXTIME" -"ULIDStringToDateTime" -"URLHash" -"URLHierarchy" -"URLPathHierarchy" -"UTCTimestamp" -"UTC_timestamp" -"UUIDNumToString" -"UUIDStringToNum" -"UUIDToNum" -"UUIDv7ToDateTime" -"VAR_POP" -"VAR_POPArgMax" -"VAR_POPArgMin" -"VAR_POPArray" -"VAR_POPDistinct" -"VAR_POPForEach" -"VAR_POPIf" -"VAR_POPMap" -"VAR_POPMerge" -"VAR_POPNull" -"VAR_POPOrDefault" -"VAR_POPOrNull" -"VAR_POPResample" -"VAR_POPSimpleState" -"VAR_POPState" -"VAR_SAMP" -"VAR_SAMPArgMax" -"VAR_SAMPArgMin" -"VAR_SAMPArray" -"VAR_SAMPDistinct" -"VAR_SAMPForEach" -"VAR_SAMPIf" -"VAR_SAMPMap" -"VAR_SAMPMerge" -"VAR_SAMPNull" -"VAR_SAMPOrDefault" -"VAR_SAMPOrNull" -"VAR_SAMPResample" -"VAR_SAMPSimpleState" -"VAR_SAMPState" +"AggregateFunction" +"Array" +"BIGINT" +"BIGINT SIGNED" +"BIGINT UNSIGNED" +"BINARY" +"BINARY LARGE OBJECT" +"BINARY VARYING" +"BIT" +"BLOB" +"BYTE" +"BYTEA" +"Bool" +"CHAR" +"CHAR LARGE OBJECT" +"CHAR VARYING" +"CHARACTER" +"CHARACTER LARGE OBJECT" +"CHARACTER VARYING" +"CLOB" +"DEC" +"DOUBLE" +"DOUBLE PRECISION" +"Date" +"Date32" +"DateTime" +"DateTime32" +"DateTime64" +"Decimal" +"Decimal128" +"Decimal256" +"Decimal32" +"Decimal64" +"Dynamic" +"ENUM" +"Enum" +"Enum16" +"Enum8" +"FIXED" +"FLOAT" +"FixedString" +"Float32" +"Float64" +"GEOMETRY" +"INET4" +"INET6" +"INT" +"INT SIGNED" +"INT UNSIGNED" +"INT1" +"INT1 SIGNED" +"INT1 UNSIGNED" +"INTEGER" +"INTEGER SIGNED" +"INTEGER UNSIGNED" +"IPv4" +"IPv6" +"Int128" +"Int16" +"Int256" +"Int32" +"Int64" +"Int8" +"IntervalDay" +"IntervalHour" +"IntervalMicrosecond" +"IntervalMillisecond" +"IntervalMinute" +"IntervalMonth" +"IntervalNanosecond" +"IntervalQuarter" +"IntervalSecond" +"IntervalWeek" +"IntervalYear" +"JSON" +"LONGBLOB" +"LONGTEXT" +"LineString" +"LowCardinality" +"MEDIUMBLOB" +"MEDIUMINT" +"MEDIUMINT SIGNED" +"MEDIUMINT UNSIGNED" +"MEDIUMTEXT" +"Map" +"MultiLineString" +"MultiPolygon" +"NATIONAL CHAR" +"NATIONAL CHAR VARYING" +"NATIONAL CHARACTER" +"NATIONAL CHARACTER LARGE OBJECT" +"NATIONAL CHARACTER VARYING" +"NCHAR" +"NCHAR LARGE OBJECT" +"NCHAR VARYING" +"NUMERIC" +"NVARCHAR" +"Nested" +"Nothing" +"Nullable" +"Object" +"Point" +"Polygon" +"REAL" +"Ring" +"SET" +"SIGNED" +"SINGLE" +"SMALLINT" +"SMALLINT SIGNED" +"SMALLINT UNSIGNED" +"SimpleAggregateFunction" +"String" +"TEXT" +"TIME" +"TIMESTAMP" +"TINYBLOB" +"TINYINT" +"TINYINT SIGNED" +"TINYINT UNSIGNED" +"TINYTEXT" +"Tuple" +"UInt128" +"UInt16" +"UInt256" +"UInt32" +"UInt64" +"UInt8" +"UNSIGNED" +"UUID" +"VARBINARY" +"VARCHAR" +"VARCHAR2" +"Variant" "YEAR" -"YYYYMMDDToDate" -"YYYYMMDDToDate32" -"YYYYMMDDhhmmssToDateTime" -"YYYYMMDDhhmmssToDateTime64" -"_CAST" -"__actionName" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" -"__bitSwapLastTwo" -"__bitWrapperFunc" -"__getScalar" -"__scalarSubqueryResult" -"abs" -"accurateCast" -"accurateCastOrDefault" -"accurateCastOrNull" -"acos" -"acosh" -"addDate" -"addDays" -"addHours" -"addInterval" -"addMicroseconds" -"addMilliseconds" -"addMinutes" -"addMonths" -"addNanoseconds" -"addQuarters" -"addSeconds" -"addTupleOfIntervals" -"addWeeks" -"addYears" -"addressToLine" -"addressToLineWithInlines" -"addressToSymbol" -"aes_decrypt_mysql" -"aes_encrypt_mysql" -"age" -"aggThrow" -"aggThrowArgMax" -"aggThrowArgMin" -"aggThrowArray" -"aggThrowDistinct" -"aggThrowForEach" -"aggThrowIf" -"aggThrowMap" -"aggThrowMerge" -"aggThrowNull" -"aggThrowOrDefault" -"aggThrowOrNull" -"aggThrowResample" -"aggThrowSimpleState" -"aggThrowState" -"alphaTokens" -"analysisOfVariance" -"analysisOfVarianceArgMax" -"analysisOfVarianceArgMin" -"analysisOfVarianceArray" -"analysisOfVarianceDistinct" -"analysisOfVarianceForEach" -"analysisOfVarianceIf" -"analysisOfVarianceMap" -"analysisOfVarianceMerge" -"analysisOfVarianceNull" -"analysisOfVarianceOrDefault" -"analysisOfVarianceOrNull" -"analysisOfVarianceResample" -"analysisOfVarianceSimpleState" -"analysisOfVarianceState" -"and" -"anova" -"anovaArgMax" -"anovaArgMin" -"anovaArray" -"anovaDistinct" -"anovaForEach" -"anovaIf" -"anovaMap" -"anovaMerge" -"anovaNull" -"anovaOrDefault" -"anovaOrNull" -"anovaResample" -"anovaSimpleState" -"anovaState" -"any" -"anyArgMax" -"anyArgMin" -"anyArray" -"anyDistinct" -"anyForEach" -"anyHeavy" -"anyHeavyArgMax" -"anyHeavyArgMin" -"anyHeavyArray" -"anyHeavyDistinct" -"anyHeavyForEach" -"anyHeavyIf" -"anyHeavyMap" -"anyHeavyMerge" -"anyHeavyNull" -"anyHeavyOrDefault" -"anyHeavyOrNull" -"anyHeavyResample" -"anyHeavySimpleState" -"anyHeavyState" -"anyIf" -"anyLast" -"anyLastArgMax" -"anyLastArgMin" -"anyLastArray" -"anyLastDistinct" -"anyLastForEach" -"anyLastIf" -"anyLastMap" -"anyLastMerge" -"anyLastNull" -"anyLastOrDefault" -"anyLastOrNull" -"anyLastResample" -"anyLastSimpleState" -"anyLastState" -"anyLast_respect_nulls" -"anyLast_respect_nullsArgMax" -"anyLast_respect_nullsArgMin" -"anyLast_respect_nullsArray" -"anyLast_respect_nullsDistinct" -"anyLast_respect_nullsForEach" -"anyLast_respect_nullsIf" -"anyLast_respect_nullsMap" -"anyLast_respect_nullsMerge" -"anyLast_respect_nullsNull" -"anyLast_respect_nullsOrDefault" -"anyLast_respect_nullsOrNull" -"anyLast_respect_nullsResample" -"anyLast_respect_nullsSimpleState" -"anyLast_respect_nullsState" -"anyMap" -"anyMerge" -"anyNull" -"anyOrDefault" -"anyOrNull" -"anyResample" -"anySimpleState" -"anyState" -"any_respect_nulls" -"any_respect_nullsArgMax" -"any_respect_nullsArgMin" -"any_respect_nullsArray" -"any_respect_nullsDistinct" -"any_respect_nullsForEach" -"any_respect_nullsIf" -"any_respect_nullsMap" -"any_respect_nullsMerge" -"any_respect_nullsNull" -"any_respect_nullsOrDefault" -"any_respect_nullsOrNull" -"any_respect_nullsResample" -"any_respect_nullsSimpleState" -"any_respect_nullsState" -"any_value" -"any_valueArgMax" -"any_valueArgMin" -"any_valueArray" -"any_valueDistinct" -"any_valueForEach" -"any_valueIf" -"any_valueMap" -"any_valueMerge" -"any_valueNull" -"any_valueOrDefault" -"any_valueOrNull" -"any_valueResample" -"any_valueSimpleState" -"any_valueState" -"any_value_respect_nulls" -"any_value_respect_nullsArgMax" -"any_value_respect_nullsArgMin" -"any_value_respect_nullsArray" -"any_value_respect_nullsDistinct" -"any_value_respect_nullsForEach" -"any_value_respect_nullsIf" -"any_value_respect_nullsMap" -"any_value_respect_nullsMerge" -"any_value_respect_nullsNull" -"any_value_respect_nullsOrDefault" -"any_value_respect_nullsOrNull" -"any_value_respect_nullsResample" -"any_value_respect_nullsSimpleState" -"any_value_respect_nullsState" -"appendTrailingCharIfAbsent" -"approx_top_count" -"approx_top_countArgMax" -"approx_top_countArgMin" -"approx_top_countArray" -"approx_top_countDistinct" -"approx_top_countForEach" -"approx_top_countIf" -"approx_top_countMap" -"approx_top_countMerge" -"approx_top_countNull" -"approx_top_countOrDefault" -"approx_top_countOrNull" -"approx_top_countResample" -"approx_top_countSimpleState" -"approx_top_countState" -"approx_top_k" -"approx_top_kArgMax" -"approx_top_kArgMin" -"approx_top_kArray" -"approx_top_kDistinct" -"approx_top_kForEach" -"approx_top_kIf" -"approx_top_kMap" -"approx_top_kMerge" -"approx_top_kNull" -"approx_top_kOrDefault" -"approx_top_kOrNull" -"approx_top_kResample" -"approx_top_kSimpleState" -"approx_top_kState" -"approx_top_sum" -"approx_top_sumArgMax" -"approx_top_sumArgMin" -"approx_top_sumArray" -"approx_top_sumDistinct" -"approx_top_sumForEach" -"approx_top_sumIf" -"approx_top_sumMap" -"approx_top_sumMerge" -"approx_top_sumNull" -"approx_top_sumOrDefault" -"approx_top_sumOrNull" -"approx_top_sumResample" -"approx_top_sumSimpleState" -"approx_top_sumState" -"argMax" -"argMaxArgMax" -"argMaxArgMin" -"argMaxArray" -"argMaxDistinct" -"argMaxForEach" -"argMaxIf" -"argMaxMap" -"argMaxMerge" -"argMaxNull" -"argMaxOrDefault" -"argMaxOrNull" -"argMaxResample" -"argMaxSimpleState" -"argMaxState" -"argMin" -"argMinArgMax" -"argMinArgMin" -"argMinArray" -"argMinDistinct" -"argMinForEach" -"argMinIf" -"argMinMap" -"argMinMerge" -"argMinNull" -"argMinOrDefault" -"argMinOrNull" -"argMinResample" -"argMinSimpleState" -"argMinState" -"array" -"arrayAUC" -"arrayAll" -"arrayAvg" -"arrayCompact" -"arrayConcat" -"arrayCount" -"arrayCumSum" -"arrayCumSumNonNegative" -"arrayDifference" -"arrayDistinct" -"arrayDotProduct" -"arrayElement" -"arrayEnumerate" -"arrayEnumerateDense" -"arrayEnumerateDenseRanked" -"arrayEnumerateUniq" -"arrayEnumerateUniqRanked" -"arrayExists" -"arrayFill" -"arrayFilter" -"arrayFirst" -"arrayFirstIndex" -"arrayFirstOrNull" -"arrayFlatten" -"arrayFold" -"arrayIntersect" -"arrayJaccardIndex" -"arrayJoin" -"arrayLast" -"arrayLastIndex" -"arrayLastOrNull" -"arrayMap" -"arrayMax" -"arrayMin" -"arrayPartialReverseSort" -"arrayPartialShuffle" -"arrayPartialSort" -"arrayPopBack" -"arrayPopFront" -"arrayProduct" -"arrayPushBack" -"arrayPushFront" -"arrayRandomSample" -"arrayReduce" -"arrayReduceInRanges" -"arrayResize" -"arrayReverse" -"arrayReverseFill" -"arrayReverseSort" -"arrayReverseSplit" -"arrayRotateLeft" -"arrayRotateRight" -"arrayShiftLeft" -"arrayShiftRight" -"arrayShingles" -"arrayShuffle" -"arraySlice" -"arraySort" -"arraySplit" -"arrayStringConcat" -"arraySum" -"arrayUniq" -"arrayWithConstant" -"arrayZip" -"array_agg" -"array_aggArgMax" -"array_aggArgMin" -"array_aggArray" -"array_aggDistinct" -"array_aggForEach" -"array_aggIf" -"array_aggMap" -"array_aggMerge" -"array_aggNull" -"array_aggOrDefault" -"array_aggOrNull" -"array_aggResample" -"array_aggSimpleState" -"array_aggState" -"array_concat_agg" -"array_concat_aggArgMax" -"array_concat_aggArgMin" -"array_concat_aggArray" -"array_concat_aggDistinct" -"array_concat_aggForEach" -"array_concat_aggIf" -"array_concat_aggMap" -"array_concat_aggMerge" -"array_concat_aggNull" -"array_concat_aggOrDefault" -"array_concat_aggOrNull" -"array_concat_aggResample" -"array_concat_aggSimpleState" -"array_concat_aggState" -"ascii" -"asin" -"asinh" -"assumeNotNull" -"atan" -"atan2" -"atanh" -"avg" -"avgArgMax" -"avgArgMin" -"avgArray" -"avgDistinct" -"avgForEach" -"avgIf" -"avgMap" -"avgMerge" -"avgNull" -"avgOrDefault" -"avgOrNull" -"avgResample" -"avgSimpleState" -"avgState" -"avgWeighted" -"avgWeightedArgMax" -"avgWeightedArgMin" -"avgWeightedArray" -"avgWeightedDistinct" -"avgWeightedForEach" -"avgWeightedIf" -"avgWeightedMap" -"avgWeightedMerge" -"avgWeightedNull" -"avgWeightedOrDefault" -"avgWeightedOrNull" -"avgWeightedResample" -"avgWeightedSimpleState" -"avgWeightedState" -"bar" -"base58Decode" -"base58Encode" -"base64Decode" -"base64Encode" -"base64URLDecode" -"base64URLEncode" -"basename" -"bin" -"bitAnd" -"bitCount" -"bitHammingDistance" -"bitNot" -"bitOr" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"bitTest" -"bitTestAll" -"bitTestAny" -"bitXor" -"bitmapAnd" -"bitmapAndCardinality" -"bitmapAndnot" -"bitmapAndnotCardinality" -"bitmapBuild" -"bitmapCardinality" -"bitmapContains" -"bitmapHasAll" -"bitmapHasAny" -"bitmapMax" -"bitmapMin" -"bitmapOr" -"bitmapOrCardinality" -"bitmapSubsetInRange" -"bitmapSubsetLimit" -"bitmapToArray" -"bitmapTransform" -"bitmapXor" -"bitmapXorCardinality" -"bitmaskToArray" -"bitmaskToList" -"blockNumber" -"blockSerializedSize" -"blockSize" -"boundingRatio" -"boundingRatioArgMax" -"boundingRatioArgMin" -"boundingRatioArray" -"boundingRatioDistinct" -"boundingRatioForEach" -"boundingRatioIf" -"boundingRatioMap" -"boundingRatioMerge" -"boundingRatioNull" -"boundingRatioOrDefault" -"boundingRatioOrNull" -"boundingRatioResample" -"boundingRatioSimpleState" -"boundingRatioState" -"buildId" -"byteHammingDistance" -"byteSize" -"byteSlice" -"byteSwap" -"caseWithExpr" -"caseWithExpression" -"caseWithoutExpr" -"caseWithoutExpression" -"catboostEvaluate" -"categoricalInformationValue" -"categoricalInformationValueArgMax" -"categoricalInformationValueArgMin" -"categoricalInformationValueArray" -"categoricalInformationValueDistinct" -"categoricalInformationValueForEach" -"categoricalInformationValueIf" -"categoricalInformationValueMap" -"categoricalInformationValueMerge" -"categoricalInformationValueNull" -"categoricalInformationValueOrDefault" -"categoricalInformationValueOrNull" -"categoricalInformationValueResample" -"categoricalInformationValueSimpleState" -"categoricalInformationValueState" -"cbrt" -"ceil" -"ceiling" -"changeDay" -"changeHour" -"changeMinute" -"changeMonth" -"changeSecond" -"changeYear" -"char" -"cityHash64" -"clamp" -"coalesce" -"concat" -"concatAssumeInjective" -"concatWithSeparator" -"concatWithSeparatorAssumeInjective" -"concat_ws" -"connectionId" -"connection_id" -"contingency" -"contingencyArgMax" -"contingencyArgMin" -"contingencyArray" -"contingencyDistinct" -"contingencyForEach" -"contingencyIf" -"contingencyMap" -"contingencyMerge" -"contingencyNull" -"contingencyOrDefault" -"contingencyOrNull" -"contingencyResample" -"contingencySimpleState" -"contingencyState" -"convertCharset" -"corr" -"corrArgMax" -"corrArgMin" -"corrArray" -"corrDistinct" -"corrForEach" -"corrIf" -"corrMap" -"corrMatrix" -"corrMatrixArgMax" -"corrMatrixArgMin" -"corrMatrixArray" -"corrMatrixDistinct" -"corrMatrixForEach" -"corrMatrixIf" -"corrMatrixMap" -"corrMatrixMerge" -"corrMatrixNull" -"corrMatrixOrDefault" -"corrMatrixOrNull" -"corrMatrixResample" -"corrMatrixSimpleState" -"corrMatrixState" -"corrMerge" -"corrNull" -"corrOrDefault" -"corrOrNull" -"corrResample" -"corrSimpleState" -"corrStable" -"corrStableArgMax" -"corrStableArgMin" -"corrStableArray" -"corrStableDistinct" -"corrStableForEach" -"corrStableIf" -"corrStableMap" -"corrStableMerge" -"corrStableNull" -"corrStableOrDefault" -"corrStableOrNull" -"corrStableResample" -"corrStableSimpleState" -"corrStableState" -"corrState" -"cos" -"cosh" -"cosineDistance" -"count" -"countArgMax" -"countArgMin" -"countArray" -"countDigits" -"countDistinct" -"countEqual" -"countForEach" -"countIf" -"countMap" -"countMatches" -"countMatchesCaseInsensitive" -"countMerge" -"countNull" -"countOrDefault" -"countOrNull" -"countResample" -"countSimpleState" -"countState" -"countSubstrings" -"countSubstringsCaseInsensitive" -"countSubstringsCaseInsensitiveUTF8" -"covarPop" -"covarPopArgMax" -"covarPopArgMin" -"covarPopArray" -"covarPopDistinct" -"covarPopForEach" -"covarPopIf" -"covarPopMap" -"covarPopMatrix" -"covarPopMatrixArgMax" -"covarPopMatrixArgMin" -"covarPopMatrixArray" -"covarPopMatrixDistinct" -"covarPopMatrixForEach" -"covarPopMatrixIf" -"covarPopMatrixMap" -"covarPopMatrixMerge" -"covarPopMatrixNull" -"covarPopMatrixOrDefault" -"covarPopMatrixOrNull" -"covarPopMatrixResample" -"covarPopMatrixSimpleState" -"covarPopMatrixState" -"covarPopMerge" -"covarPopNull" -"covarPopOrDefault" -"covarPopOrNull" -"covarPopResample" -"covarPopSimpleState" -"covarPopStable" -"covarPopStableArgMax" -"covarPopStableArgMin" -"covarPopStableArray" -"covarPopStableDistinct" -"covarPopStableForEach" -"covarPopStableIf" -"covarPopStableMap" -"covarPopStableMerge" -"covarPopStableNull" -"covarPopStableOrDefault" -"covarPopStableOrNull" -"covarPopStableResample" -"covarPopStableSimpleState" -"covarPopStableState" -"covarPopState" -"covarSamp" -"covarSampArgMax" -"covarSampArgMin" -"covarSampArray" -"covarSampDistinct" -"covarSampForEach" -"covarSampIf" -"covarSampMap" -"covarSampMatrix" -"covarSampMatrixArgMax" -"covarSampMatrixArgMin" -"covarSampMatrixArray" -"covarSampMatrixDistinct" -"covarSampMatrixForEach" -"covarSampMatrixIf" -"covarSampMatrixMap" -"covarSampMatrixMerge" -"covarSampMatrixNull" -"covarSampMatrixOrDefault" -"covarSampMatrixOrNull" -"covarSampMatrixResample" -"covarSampMatrixSimpleState" -"covarSampMatrixState" -"covarSampMerge" -"covarSampNull" -"covarSampOrDefault" -"covarSampOrNull" -"covarSampResample" -"covarSampSimpleState" -"covarSampStable" -"covarSampStableArgMax" -"covarSampStableArgMin" -"covarSampStableArray" -"covarSampStableDistinct" -"covarSampStableForEach" -"covarSampStableIf" -"covarSampStableMap" -"covarSampStableMerge" -"covarSampStableNull" -"covarSampStableOrDefault" -"covarSampStableOrNull" -"covarSampStableResample" -"covarSampStableSimpleState" -"covarSampStableState" -"covarSampState" -"cramersV" -"cramersVArgMax" -"cramersVArgMin" -"cramersVArray" -"cramersVBiasCorrected" -"cramersVBiasCorrectedArgMax" -"cramersVBiasCorrectedArgMin" -"cramersVBiasCorrectedArray" -"cramersVBiasCorrectedDistinct" -"cramersVBiasCorrectedForEach" -"cramersVBiasCorrectedIf" -"cramersVBiasCorrectedMap" -"cramersVBiasCorrectedMerge" -"cramersVBiasCorrectedNull" -"cramersVBiasCorrectedOrDefault" -"cramersVBiasCorrectedOrNull" -"cramersVBiasCorrectedResample" -"cramersVBiasCorrectedSimpleState" -"cramersVBiasCorrectedState" -"cramersVDistinct" -"cramersVForEach" -"cramersVIf" -"cramersVMap" -"cramersVMerge" -"cramersVNull" -"cramersVOrDefault" -"cramersVOrNull" -"cramersVResample" -"cramersVSimpleState" -"cramersVState" -"curdate" -"currentDatabase" -"currentProfiles" -"currentRoles" -"currentSchemas" -"currentUser" -"current_database" -"current_date" -"current_schemas" -"current_timestamp" -"current_user" -"cutFragment" -"cutIPv6" -"cutQueryString" -"cutQueryStringAndFragment" -"cutToFirstSignificantSubdomain" -"cutToFirstSignificantSubdomainCustom" -"cutToFirstSignificantSubdomainCustomRFC" -"cutToFirstSignificantSubdomainCustomWithWWW" -"cutToFirstSignificantSubdomainCustomWithWWWRFC" -"cutToFirstSignificantSubdomainRFC" -"cutToFirstSignificantSubdomainWithWWW" -"cutToFirstSignificantSubdomainWithWWWRFC" -"cutURLParameter" -"cutWWW" -"damerauLevenshteinDistance" -"dateDiff" -"dateName" -"dateTime64ToSnowflake" -"dateTime64ToSnowflakeID" -"dateTimeToSnowflake" -"dateTimeToSnowflakeID" -"dateTrunc" -"date_diff" -"decodeHTMLComponent" -"decodeURLComponent" -"decodeURLFormComponent" -"decodeXMLComponent" -"decrypt" -"defaultProfiles" -"defaultRoles" -"defaultValueOfArgumentType" -"defaultValueOfTypeName" -"degrees" -"deltaSum" -"deltaSumArgMax" -"deltaSumArgMin" -"deltaSumArray" -"deltaSumDistinct" -"deltaSumForEach" -"deltaSumIf" -"deltaSumMap" -"deltaSumMerge" -"deltaSumNull" -"deltaSumOrDefault" -"deltaSumOrNull" -"deltaSumResample" -"deltaSumSimpleState" -"deltaSumState" -"deltaSumTimestamp" -"deltaSumTimestampArgMax" -"deltaSumTimestampArgMin" -"deltaSumTimestampArray" -"deltaSumTimestampDistinct" -"deltaSumTimestampForEach" -"deltaSumTimestampIf" -"deltaSumTimestampMap" -"deltaSumTimestampMerge" -"deltaSumTimestampNull" -"deltaSumTimestampOrDefault" -"deltaSumTimestampOrNull" -"deltaSumTimestampResample" -"deltaSumTimestampSimpleState" -"deltaSumTimestampState" -"demangle" -"denseRank" -"denseRankArgMax" -"denseRankArgMin" -"denseRankArray" -"denseRankDistinct" -"denseRankForEach" -"denseRankIf" -"denseRankMap" -"denseRankMerge" -"denseRankNull" -"denseRankOrDefault" -"denseRankOrNull" -"denseRankResample" -"denseRankSimpleState" -"denseRankState" -"dense_rank" -"dense_rankArgMax" -"dense_rankArgMin" -"dense_rankArray" -"dense_rankDistinct" -"dense_rankForEach" -"dense_rankIf" -"dense_rankMap" -"dense_rankMerge" -"dense_rankNull" -"dense_rankOrDefault" -"dense_rankOrNull" -"dense_rankResample" -"dense_rankSimpleState" -"dense_rankState" -"detectCharset" -"detectLanguage" -"detectLanguageMixed" -"detectLanguageUnknown" -"detectProgrammingLanguage" -"detectTonality" -"dictGet" -"dictGetAll" -"dictGetChildren" -"dictGetDate" -"dictGetDateOrDefault" -"dictGetDateTime" -"dictGetDateTimeOrDefault" -"dictGetDescendants" -"dictGetFloat32" -"dictGetFloat32OrDefault" -"dictGetFloat64" -"dictGetFloat64OrDefault" -"dictGetHierarchy" -"dictGetIPv4" -"dictGetIPv4OrDefault" -"dictGetIPv6" -"dictGetIPv6OrDefault" -"dictGetInt16" -"dictGetInt16OrDefault" -"dictGetInt32" -"dictGetInt32OrDefault" -"dictGetInt64" -"dictGetInt64OrDefault" -"dictGetInt8" -"dictGetInt8OrDefault" -"dictGetOrDefault" -"dictGetOrNull" -"dictGetString" -"dictGetStringOrDefault" -"dictGetUInt16" -"dictGetUInt16OrDefault" -"dictGetUInt32" -"dictGetUInt32OrDefault" -"dictGetUInt64" -"dictGetUInt64OrDefault" -"dictGetUInt8" -"dictGetUInt8OrDefault" -"dictGetUUID" -"dictGetUUIDOrDefault" -"dictHas" -"dictIsIn" -"displayName" -"distanceL1" -"distanceL2" -"distanceL2Squared" -"distanceLinf" -"distanceLp" -"divide" -"divideDecimal" -"domain" -"domainRFC" -"domainWithoutWWW" -"domainWithoutWWWRFC" -"dotProduct" -"dumpColumnStructure" -"dynamicElement" -"dynamicType" -"e" -"editDistance" -"editDistanceUTF8" -"empty" -"emptyArrayDate" -"emptyArrayDateTime" -"emptyArrayFloat32" -"emptyArrayFloat64" -"emptyArrayInt16" -"emptyArrayInt32" -"emptyArrayInt64" -"emptyArrayInt8" -"emptyArrayString" -"emptyArrayToSingle" -"emptyArrayUInt16" -"emptyArrayUInt32" -"emptyArrayUInt64" -"emptyArrayUInt8" -"enabledProfiles" -"enabledRoles" -"encodeURLComponent" -"encodeURLFormComponent" -"encodeXMLComponent" -"encrypt" -"endsWith" -"endsWithUTF8" -"entropy" -"entropyArgMax" -"entropyArgMin" -"entropyArray" -"entropyDistinct" -"entropyForEach" -"entropyIf" -"entropyMap" -"entropyMerge" -"entropyNull" -"entropyOrDefault" -"entropyOrNull" -"entropyResample" -"entropySimpleState" -"entropyState" -"equals" -"erf" -"erfc" -"errorCodeToName" -"evalMLMethod" -"exp" -"exp10" -"exp2" -"exponentialMovingAverage" -"exponentialMovingAverageArgMax" -"exponentialMovingAverageArgMin" -"exponentialMovingAverageArray" -"exponentialMovingAverageDistinct" -"exponentialMovingAverageForEach" -"exponentialMovingAverageIf" -"exponentialMovingAverageMap" -"exponentialMovingAverageMerge" -"exponentialMovingAverageNull" -"exponentialMovingAverageOrDefault" -"exponentialMovingAverageOrNull" -"exponentialMovingAverageResample" -"exponentialMovingAverageSimpleState" -"exponentialMovingAverageState" -"exponentialTimeDecayedAvg" -"exponentialTimeDecayedAvgArgMax" -"exponentialTimeDecayedAvgArgMin" -"exponentialTimeDecayedAvgArray" -"exponentialTimeDecayedAvgDistinct" -"exponentialTimeDecayedAvgForEach" -"exponentialTimeDecayedAvgIf" -"exponentialTimeDecayedAvgMap" -"exponentialTimeDecayedAvgMerge" -"exponentialTimeDecayedAvgNull" -"exponentialTimeDecayedAvgOrDefault" -"exponentialTimeDecayedAvgOrNull" -"exponentialTimeDecayedAvgResample" -"exponentialTimeDecayedAvgSimpleState" -"exponentialTimeDecayedAvgState" -"exponentialTimeDecayedCount" -"exponentialTimeDecayedCountArgMax" -"exponentialTimeDecayedCountArgMin" -"exponentialTimeDecayedCountArray" -"exponentialTimeDecayedCountDistinct" -"exponentialTimeDecayedCountForEach" -"exponentialTimeDecayedCountIf" -"exponentialTimeDecayedCountMap" -"exponentialTimeDecayedCountMerge" -"exponentialTimeDecayedCountNull" -"exponentialTimeDecayedCountOrDefault" -"exponentialTimeDecayedCountOrNull" -"exponentialTimeDecayedCountResample" -"exponentialTimeDecayedCountSimpleState" -"exponentialTimeDecayedCountState" -"exponentialTimeDecayedMax" -"exponentialTimeDecayedMaxArgMax" -"exponentialTimeDecayedMaxArgMin" -"exponentialTimeDecayedMaxArray" -"exponentialTimeDecayedMaxDistinct" -"exponentialTimeDecayedMaxForEach" -"exponentialTimeDecayedMaxIf" -"exponentialTimeDecayedMaxMap" -"exponentialTimeDecayedMaxMerge" -"exponentialTimeDecayedMaxNull" -"exponentialTimeDecayedMaxOrDefault" -"exponentialTimeDecayedMaxOrNull" -"exponentialTimeDecayedMaxResample" -"exponentialTimeDecayedMaxSimpleState" -"exponentialTimeDecayedMaxState" -"exponentialTimeDecayedSum" -"exponentialTimeDecayedSumArgMax" -"exponentialTimeDecayedSumArgMin" -"exponentialTimeDecayedSumArray" -"exponentialTimeDecayedSumDistinct" -"exponentialTimeDecayedSumForEach" -"exponentialTimeDecayedSumIf" -"exponentialTimeDecayedSumMap" -"exponentialTimeDecayedSumMerge" -"exponentialTimeDecayedSumNull" -"exponentialTimeDecayedSumOrDefault" -"exponentialTimeDecayedSumOrNull" -"exponentialTimeDecayedSumResample" -"exponentialTimeDecayedSumSimpleState" -"exponentialTimeDecayedSumState" -"extract" -"extractAll" -"extractAllGroups" -"extractAllGroupsHorizontal" -"extractAllGroupsVertical" -"extractGroups" -"extractKeyValuePairs" -"extractKeyValuePairsWithEscaping" -"extractTextFromHTML" -"extractURLParameter" -"extractURLParameterNames" -"extractURLParameters" -"factorial" -"farmFingerprint64" -"farmHash64" -"file" -"filesystemAvailable" -"filesystemCapacity" -"filesystemUnreserved" -"finalizeAggregation" -"firstLine" -"firstSignificantSubdomain" -"firstSignificantSubdomainCustom" -"firstSignificantSubdomainCustomRFC" -"firstSignificantSubdomainRFC" -"first_value" -"first_valueArgMax" -"first_valueArgMin" -"first_valueArray" -"first_valueDistinct" -"first_valueForEach" -"first_valueIf" -"first_valueMap" -"first_valueMerge" -"first_valueNull" -"first_valueOrDefault" -"first_valueOrNull" -"first_valueResample" -"first_valueSimpleState" -"first_valueState" -"first_value_respect_nulls" -"first_value_respect_nullsArgMax" -"first_value_respect_nullsArgMin" -"first_value_respect_nullsArray" -"first_value_respect_nullsDistinct" -"first_value_respect_nullsForEach" -"first_value_respect_nullsIf" -"first_value_respect_nullsMap" -"first_value_respect_nullsMerge" -"first_value_respect_nullsNull" -"first_value_respect_nullsOrDefault" -"first_value_respect_nullsOrNull" -"first_value_respect_nullsResample" -"first_value_respect_nullsSimpleState" -"first_value_respect_nullsState" -"flameGraph" -"flameGraphArgMax" -"flameGraphArgMin" -"flameGraphArray" -"flameGraphDistinct" -"flameGraphForEach" -"flameGraphIf" -"flameGraphMap" -"flameGraphMerge" -"flameGraphNull" -"flameGraphOrDefault" -"flameGraphOrNull" -"flameGraphResample" -"flameGraphSimpleState" -"flameGraphState" -"flatten" -"flattenTuple" -"floor" -"format" -"formatDateTime" -"formatDateTimeInJodaSyntax" -"formatQuery" -"formatQueryOrNull" -"formatQuerySingleLine" -"formatQuerySingleLineOrNull" -"formatReadableDecimalSize" -"formatReadableQuantity" -"formatReadableSize" -"formatReadableTimeDelta" -"formatRow" -"formatRowNoNewline" -"fragment" -"fromDaysSinceYearZero" -"fromDaysSinceYearZero32" -"fromModifiedJulianDay" -"fromModifiedJulianDayOrNull" -"fromUTCTimestamp" -"fromUnixTimestamp" -"fromUnixTimestamp64Micro" -"fromUnixTimestamp64Milli" -"fromUnixTimestamp64Nano" -"fromUnixTimestampInJodaSyntax" -"from_utc_timestamp" -"fullHostName" -"fuzzBits" -"gccMurmurHash" -"gcd" -"generateRandomStructure" -"generateSnowflakeID" -"generateULID" -"generateUUIDv4" -"generateUUIDv7" -"geoDistance" -"geoToH3" -"geoToS2" -"geohashDecode" -"geohashEncode" -"geohashesInBox" -"getClientHTTPHeader" -"getMacro" -"getOSKernelVersion" -"getServerPort" -"getSetting" -"getSizeOfEnumType" -"getSubcolumn" -"getTypeSerializationStreams" -"globalIn" -"globalInIgnoreSet" -"globalNotIn" -"globalNotInIgnoreSet" -"globalNotNullIn" -"globalNotNullInIgnoreSet" -"globalNullIn" -"globalNullInIgnoreSet" -"globalVariable" -"greatCircleAngle" -"greatCircleDistance" -"greater" -"greaterOrEquals" -"greatest" -"groupArray" -"groupArrayArgMax" -"groupArrayArgMin" -"groupArrayArray" -"groupArrayDistinct" -"groupArrayForEach" -"groupArrayIf" -"groupArrayInsertAt" -"groupArrayInsertAtArgMax" -"groupArrayInsertAtArgMin" -"groupArrayInsertAtArray" -"groupArrayInsertAtDistinct" -"groupArrayInsertAtForEach" -"groupArrayInsertAtIf" -"groupArrayInsertAtMap" -"groupArrayInsertAtMerge" -"groupArrayInsertAtNull" -"groupArrayInsertAtOrDefault" -"groupArrayInsertAtOrNull" -"groupArrayInsertAtResample" -"groupArrayInsertAtSimpleState" -"groupArrayInsertAtState" -"groupArrayIntersect" -"groupArrayIntersectArgMax" -"groupArrayIntersectArgMin" -"groupArrayIntersectArray" -"groupArrayIntersectDistinct" -"groupArrayIntersectForEach" -"groupArrayIntersectIf" -"groupArrayIntersectMap" -"groupArrayIntersectMerge" -"groupArrayIntersectNull" -"groupArrayIntersectOrDefault" -"groupArrayIntersectOrNull" -"groupArrayIntersectResample" -"groupArrayIntersectSimpleState" -"groupArrayIntersectState" -"groupArrayLast" -"groupArrayLastArgMax" -"groupArrayLastArgMin" -"groupArrayLastArray" -"groupArrayLastDistinct" -"groupArrayLastForEach" -"groupArrayLastIf" -"groupArrayLastMap" -"groupArrayLastMerge" -"groupArrayLastNull" -"groupArrayLastOrDefault" -"groupArrayLastOrNull" -"groupArrayLastResample" -"groupArrayLastSimpleState" -"groupArrayLastState" -"groupArrayMap" -"groupArrayMerge" -"groupArrayMovingAvg" -"groupArrayMovingAvgArgMax" -"groupArrayMovingAvgArgMin" -"groupArrayMovingAvgArray" -"groupArrayMovingAvgDistinct" -"groupArrayMovingAvgForEach" -"groupArrayMovingAvgIf" -"groupArrayMovingAvgMap" -"groupArrayMovingAvgMerge" -"groupArrayMovingAvgNull" -"groupArrayMovingAvgOrDefault" -"groupArrayMovingAvgOrNull" -"groupArrayMovingAvgResample" -"groupArrayMovingAvgSimpleState" -"groupArrayMovingAvgState" -"groupArrayMovingSum" -"groupArrayMovingSumArgMax" -"groupArrayMovingSumArgMin" -"groupArrayMovingSumArray" -"groupArrayMovingSumDistinct" -"groupArrayMovingSumForEach" -"groupArrayMovingSumIf" -"groupArrayMovingSumMap" -"groupArrayMovingSumMerge" -"groupArrayMovingSumNull" -"groupArrayMovingSumOrDefault" -"groupArrayMovingSumOrNull" -"groupArrayMovingSumResample" -"groupArrayMovingSumSimpleState" -"groupArrayMovingSumState" -"groupArrayNull" -"groupArrayOrDefault" -"groupArrayOrNull" -"groupArrayResample" -"groupArraySample" -"groupArraySampleArgMax" -"groupArraySampleArgMin" -"groupArraySampleArray" -"groupArraySampleDistinct" -"groupArraySampleForEach" -"groupArraySampleIf" -"groupArraySampleMap" -"groupArraySampleMerge" -"groupArraySampleNull" -"groupArraySampleOrDefault" -"groupArraySampleOrNull" -"groupArraySampleResample" -"groupArraySampleSimpleState" -"groupArraySampleState" -"groupArraySimpleState" -"groupArraySorted" -"groupArraySortedArgMax" -"groupArraySortedArgMin" -"groupArraySortedArray" -"groupArraySortedDistinct" -"groupArraySortedForEach" -"groupArraySortedIf" -"groupArraySortedMap" -"groupArraySortedMerge" -"groupArraySortedNull" -"groupArraySortedOrDefault" -"groupArraySortedOrNull" -"groupArraySortedResample" -"groupArraySortedSimpleState" -"groupArraySortedState" -"groupArrayState" -"groupBitAnd" -"groupBitAndArgMax" -"groupBitAndArgMin" -"groupBitAndArray" -"groupBitAndDistinct" -"groupBitAndForEach" -"groupBitAndIf" -"groupBitAndMap" -"groupBitAndMerge" -"groupBitAndNull" -"groupBitAndOrDefault" -"groupBitAndOrNull" -"groupBitAndResample" -"groupBitAndSimpleState" -"groupBitAndState" -"groupBitOr" -"groupBitOrArgMax" -"groupBitOrArgMin" -"groupBitOrArray" -"groupBitOrDistinct" -"groupBitOrForEach" -"groupBitOrIf" -"groupBitOrMap" -"groupBitOrMerge" -"groupBitOrNull" -"groupBitOrOrDefault" -"groupBitOrOrNull" -"groupBitOrResample" -"groupBitOrSimpleState" -"groupBitOrState" -"groupBitXor" -"groupBitXorArgMax" -"groupBitXorArgMin" -"groupBitXorArray" -"groupBitXorDistinct" -"groupBitXorForEach" -"groupBitXorIf" -"groupBitXorMap" -"groupBitXorMerge" -"groupBitXorNull" -"groupBitXorOrDefault" -"groupBitXorOrNull" -"groupBitXorResample" -"groupBitXorSimpleState" -"groupBitXorState" -"groupBitmap" -"groupBitmapAnd" -"groupBitmapAndArgMax" -"groupBitmapAndArgMin" -"groupBitmapAndArray" -"groupBitmapAndDistinct" -"groupBitmapAndForEach" -"groupBitmapAndIf" -"groupBitmapAndMap" -"groupBitmapAndMerge" -"groupBitmapAndNull" -"groupBitmapAndOrDefault" -"groupBitmapAndOrNull" -"groupBitmapAndResample" -"groupBitmapAndSimpleState" -"groupBitmapAndState" -"groupBitmapArgMax" -"groupBitmapArgMin" -"groupBitmapArray" -"groupBitmapDistinct" -"groupBitmapForEach" -"groupBitmapIf" -"groupBitmapMap" -"groupBitmapMerge" -"groupBitmapNull" -"groupBitmapOr" -"groupBitmapOrArgMax" -"groupBitmapOrArgMin" -"groupBitmapOrArray" -"groupBitmapOrDefault" -"groupBitmapOrDistinct" -"groupBitmapOrForEach" -"groupBitmapOrIf" -"groupBitmapOrMap" -"groupBitmapOrMerge" -"groupBitmapOrNull" -"groupBitmapOrNull" -"groupBitmapOrOrDefault" -"groupBitmapOrOrNull" -"groupBitmapOrResample" -"groupBitmapOrSimpleState" -"groupBitmapOrState" -"groupBitmapResample" -"groupBitmapSimpleState" -"groupBitmapState" -"groupBitmapXor" -"groupBitmapXorArgMax" -"groupBitmapXorArgMin" -"groupBitmapXorArray" -"groupBitmapXorDistinct" -"groupBitmapXorForEach" -"groupBitmapXorIf" -"groupBitmapXorMap" -"groupBitmapXorMerge" -"groupBitmapXorNull" -"groupBitmapXorOrDefault" -"groupBitmapXorOrNull" -"groupBitmapXorResample" -"groupBitmapXorSimpleState" -"groupBitmapXorState" -"groupConcat" -"groupConcatArgMax" -"groupConcatArgMin" -"groupConcatArray" -"groupConcatDistinct" -"groupConcatForEach" -"groupConcatIf" -"groupConcatMap" -"groupConcatMerge" -"groupConcatNull" -"groupConcatOrDefault" -"groupConcatOrNull" -"groupConcatResample" -"groupConcatSimpleState" -"groupConcatState" -"groupUniqArray" -"groupUniqArrayArgMax" -"groupUniqArrayArgMin" -"groupUniqArrayArray" -"groupUniqArrayDistinct" -"groupUniqArrayForEach" -"groupUniqArrayIf" -"groupUniqArrayMap" -"groupUniqArrayMerge" -"groupUniqArrayNull" -"groupUniqArrayOrDefault" -"groupUniqArrayOrNull" -"groupUniqArrayResample" -"groupUniqArraySimpleState" -"groupUniqArrayState" -"group_concat" -"group_concatArgMax" -"group_concatArgMin" -"group_concatArray" -"group_concatDistinct" -"group_concatForEach" -"group_concatIf" -"group_concatMap" -"group_concatMerge" -"group_concatNull" -"group_concatOrDefault" -"group_concatOrNull" -"group_concatResample" -"group_concatSimpleState" -"group_concatState" -"h3CellAreaM2" -"h3CellAreaRads2" -"h3Distance" -"h3EdgeAngle" -"h3EdgeLengthKm" -"h3EdgeLengthM" -"h3ExactEdgeLengthKm" -"h3ExactEdgeLengthM" -"h3ExactEdgeLengthRads" -"h3GetBaseCell" -"h3GetDestinationIndexFromUnidirectionalEdge" -"h3GetFaces" -"h3GetIndexesFromUnidirectionalEdge" -"h3GetOriginIndexFromUnidirectionalEdge" -"h3GetPentagonIndexes" -"h3GetRes0Indexes" -"h3GetResolution" -"h3GetUnidirectionalEdge" -"h3GetUnidirectionalEdgeBoundary" -"h3GetUnidirectionalEdgesFromHexagon" -"h3HexAreaKm2" -"h3HexAreaM2" -"h3HexRing" -"h3IndexesAreNeighbors" -"h3IsPentagon" -"h3IsResClassIII" -"h3IsValid" -"h3Line" -"h3NumHexagons" -"h3PointDistKm" -"h3PointDistM" -"h3PointDistRads" -"h3ToCenterChild" -"h3ToChildren" -"h3ToGeo" -"h3ToGeoBoundary" -"h3ToParent" -"h3ToString" -"h3UnidirectionalEdgeIsValid" -"h3kRing" -"halfMD5" -"has" -"hasAll" -"hasAny" -"hasColumnInTable" -"hasSubsequence" -"hasSubsequenceCaseInsensitive" -"hasSubsequenceCaseInsensitiveUTF8" -"hasSubsequenceUTF8" -"hasSubstr" -"hasThreadFuzzer" -"hasToken" -"hasTokenCaseInsensitive" -"hasTokenCaseInsensitiveOrNull" -"hasTokenOrNull" -"hex" -"hilbertDecode" -"hilbertEncode" -"histogram" -"histogramArgMax" -"histogramArgMin" -"histogramArray" -"histogramDistinct" -"histogramForEach" -"histogramIf" -"histogramMap" -"histogramMerge" -"histogramNull" -"histogramOrDefault" -"histogramOrNull" -"histogramResample" -"histogramSimpleState" -"histogramState" -"hiveHash" -"hop" -"hopEnd" -"hopStart" -"hostName" -"hostname" -"hypot" -"identity" -"idnaDecode" -"idnaEncode" -"if" -"ifNotFinite" -"ifNull" -"ignore" -"ilike" -"in" -"inIgnoreSet" -"indexHint" -"indexOf" -"initcap" -"initcapUTF8" -"initialQueryID" -"initial_query_id" -"initializeAggregation" -"instr" -"intDiv" -"intDivOrZero" -"intExp10" -"intExp2" -"intHash32" -"intHash64" -"intervalLengthSum" -"intervalLengthSumArgMax" -"intervalLengthSumArgMin" -"intervalLengthSumArray" -"intervalLengthSumDistinct" -"intervalLengthSumForEach" -"intervalLengthSumIf" -"intervalLengthSumMap" -"intervalLengthSumMerge" -"intervalLengthSumNull" -"intervalLengthSumOrDefault" -"intervalLengthSumOrNull" -"intervalLengthSumResample" -"intervalLengthSumSimpleState" -"intervalLengthSumState" -"isConstant" -"isDecimalOverflow" -"isFinite" -"isIPAddressInRange" -"isIPv4String" -"isIPv6String" -"isInfinite" -"isNaN" -"isNotDistinctFrom" -"isNotNull" -"isNull" -"isNullable" -"isValidJSON" -"isValidUTF8" -"isZeroOrNull" -"jaroSimilarity" -"jaroWinklerSimilarity" -"javaHash" -"javaHashUTF16LE" -"joinGet" -"joinGetOrNull" -"jsonMergePatch" -"jumpConsistentHash" -"kafkaMurmurHash" -"kolmogorovSmirnovTest" -"kolmogorovSmirnovTestArgMax" -"kolmogorovSmirnovTestArgMin" -"kolmogorovSmirnovTestArray" -"kolmogorovSmirnovTestDistinct" -"kolmogorovSmirnovTestForEach" -"kolmogorovSmirnovTestIf" -"kolmogorovSmirnovTestMap" -"kolmogorovSmirnovTestMerge" -"kolmogorovSmirnovTestNull" -"kolmogorovSmirnovTestOrDefault" -"kolmogorovSmirnovTestOrNull" -"kolmogorovSmirnovTestResample" -"kolmogorovSmirnovTestSimpleState" -"kolmogorovSmirnovTestState" -"kostikConsistentHash" -"kql_array_sort_asc" -"kql_array_sort_desc" -"kurtPop" -"kurtPopArgMax" -"kurtPopArgMin" -"kurtPopArray" -"kurtPopDistinct" -"kurtPopForEach" -"kurtPopIf" -"kurtPopMap" -"kurtPopMerge" -"kurtPopNull" -"kurtPopOrDefault" -"kurtPopOrNull" -"kurtPopResample" -"kurtPopSimpleState" -"kurtPopState" -"kurtSamp" -"kurtSampArgMax" -"kurtSampArgMin" -"kurtSampArray" -"kurtSampDistinct" -"kurtSampForEach" -"kurtSampIf" -"kurtSampMap" -"kurtSampMerge" -"kurtSampNull" -"kurtSampOrDefault" -"kurtSampOrNull" -"kurtSampResample" -"kurtSampSimpleState" -"kurtSampState" -"lagInFrame" -"lagInFrameArgMax" -"lagInFrameArgMin" -"lagInFrameArray" -"lagInFrameDistinct" -"lagInFrameForEach" -"lagInFrameIf" -"lagInFrameMap" -"lagInFrameMerge" -"lagInFrameNull" -"lagInFrameOrDefault" -"lagInFrameOrNull" -"lagInFrameResample" -"lagInFrameSimpleState" -"lagInFrameState" -"largestTriangleThreeBuckets" -"largestTriangleThreeBucketsArgMax" -"largestTriangleThreeBucketsArgMin" -"largestTriangleThreeBucketsArray" -"largestTriangleThreeBucketsDistinct" -"largestTriangleThreeBucketsForEach" -"largestTriangleThreeBucketsIf" -"largestTriangleThreeBucketsMap" -"largestTriangleThreeBucketsMerge" -"largestTriangleThreeBucketsNull" -"largestTriangleThreeBucketsOrDefault" -"largestTriangleThreeBucketsOrNull" -"largestTriangleThreeBucketsResample" -"largestTriangleThreeBucketsSimpleState" -"largestTriangleThreeBucketsState" -"last_value" -"last_valueArgMax" -"last_valueArgMin" -"last_valueArray" -"last_valueDistinct" -"last_valueForEach" -"last_valueIf" -"last_valueMap" -"last_valueMerge" -"last_valueNull" -"last_valueOrDefault" -"last_valueOrNull" -"last_valueResample" -"last_valueSimpleState" -"last_valueState" -"last_value_respect_nulls" -"last_value_respect_nullsArgMax" -"last_value_respect_nullsArgMin" -"last_value_respect_nullsArray" -"last_value_respect_nullsDistinct" -"last_value_respect_nullsForEach" -"last_value_respect_nullsIf" -"last_value_respect_nullsMap" -"last_value_respect_nullsMerge" -"last_value_respect_nullsNull" -"last_value_respect_nullsOrDefault" -"last_value_respect_nullsOrNull" -"last_value_respect_nullsResample" -"last_value_respect_nullsSimpleState" -"last_value_respect_nullsState" -"lcase" -"lcm" -"leadInFrame" -"leadInFrameArgMax" -"leadInFrameArgMin" -"leadInFrameArray" -"leadInFrameDistinct" -"leadInFrameForEach" -"leadInFrameIf" -"leadInFrameMap" -"leadInFrameMerge" -"leadInFrameNull" -"leadInFrameOrDefault" -"leadInFrameOrNull" -"leadInFrameResample" -"leadInFrameSimpleState" -"leadInFrameState" -"least" -"left" -"leftPad" -"leftPadUTF8" -"leftUTF8" -"lemmatize" -"length" -"lengthUTF8" -"less" -"lessOrEquals" -"levenshteinDistance" -"levenshteinDistanceUTF8" -"lgamma" -"like" -"ln" -"locate" -"log" -"log10" -"log1p" -"log2" -"logTrace" -"lowCardinalityIndices" -"lowCardinalityKeys" -"lower" -"lowerUTF8" -"lpad" -"ltrim" -"lttb" -"lttbArgMax" -"lttbArgMin" -"lttbArray" -"lttbDistinct" -"lttbForEach" -"lttbIf" -"lttbMap" -"lttbMerge" -"lttbNull" -"lttbOrDefault" -"lttbOrNull" -"lttbResample" -"lttbSimpleState" -"lttbState" -"makeDate" -"makeDate32" -"makeDateTime" -"makeDateTime64" -"mannWhitneyUTest" -"mannWhitneyUTestArgMax" -"mannWhitneyUTestArgMin" -"mannWhitneyUTestArray" -"mannWhitneyUTestDistinct" -"mannWhitneyUTestForEach" -"mannWhitneyUTestIf" -"mannWhitneyUTestMap" -"mannWhitneyUTestMerge" -"mannWhitneyUTestNull" -"mannWhitneyUTestOrDefault" -"mannWhitneyUTestOrNull" -"mannWhitneyUTestResample" -"mannWhitneyUTestSimpleState" -"mannWhitneyUTestState" -"map" -"mapAdd" -"mapAll" -"mapApply" -"mapConcat" -"mapContains" -"mapContainsKeyLike" -"mapExists" -"mapExtractKeyLike" -"mapFilter" -"mapFromArrays" -"mapFromString" -"mapKeys" -"mapPartialReverseSort" -"mapPartialSort" -"mapPopulateSeries" -"mapReverseSort" -"mapSort" -"mapSubtract" -"mapUpdate" -"mapValues" -"match" -"materialize" -"max" -"max2" -"maxArgMax" -"maxArgMin" -"maxArray" -"maxDistinct" -"maxForEach" -"maxIf" -"maxIntersections" -"maxIntersectionsArgMax" -"maxIntersectionsArgMin" -"maxIntersectionsArray" -"maxIntersectionsDistinct" -"maxIntersectionsForEach" -"maxIntersectionsIf" -"maxIntersectionsMap" -"maxIntersectionsMerge" -"maxIntersectionsNull" -"maxIntersectionsOrDefault" -"maxIntersectionsOrNull" -"maxIntersectionsPosition" -"maxIntersectionsPositionArgMax" -"maxIntersectionsPositionArgMin" -"maxIntersectionsPositionArray" -"maxIntersectionsPositionDistinct" -"maxIntersectionsPositionForEach" -"maxIntersectionsPositionIf" -"maxIntersectionsPositionMap" -"maxIntersectionsPositionMerge" -"maxIntersectionsPositionNull" -"maxIntersectionsPositionOrDefault" -"maxIntersectionsPositionOrNull" -"maxIntersectionsPositionResample" -"maxIntersectionsPositionSimpleState" -"maxIntersectionsPositionState" -"maxIntersectionsResample" -"maxIntersectionsSimpleState" -"maxIntersectionsState" -"maxMap" -"maxMappedArrays" -"maxMappedArraysArgMax" -"maxMappedArraysArgMin" -"maxMappedArraysArray" -"maxMappedArraysDistinct" -"maxMappedArraysForEach" -"maxMappedArraysIf" -"maxMappedArraysMap" -"maxMappedArraysMerge" -"maxMappedArraysNull" -"maxMappedArraysOrDefault" -"maxMappedArraysOrNull" -"maxMappedArraysResample" -"maxMappedArraysSimpleState" -"maxMappedArraysState" -"maxMerge" -"maxNull" -"maxOrDefault" -"maxOrNull" -"maxResample" -"maxSimpleState" -"maxState" -"meanZTest" -"meanZTestArgMax" -"meanZTestArgMin" -"meanZTestArray" -"meanZTestDistinct" -"meanZTestForEach" -"meanZTestIf" -"meanZTestMap" -"meanZTestMerge" -"meanZTestNull" -"meanZTestOrDefault" -"meanZTestOrNull" -"meanZTestResample" -"meanZTestSimpleState" -"meanZTestState" -"median" -"medianArgMax" -"medianArgMin" -"medianArray" -"medianBFloat16" -"medianBFloat16ArgMax" -"medianBFloat16ArgMin" -"medianBFloat16Array" -"medianBFloat16Distinct" -"medianBFloat16ForEach" -"medianBFloat16If" -"medianBFloat16Map" -"medianBFloat16Merge" -"medianBFloat16Null" -"medianBFloat16OrDefault" -"medianBFloat16OrNull" -"medianBFloat16Resample" -"medianBFloat16SimpleState" -"medianBFloat16State" -"medianBFloat16Weighted" -"medianBFloat16WeightedArgMax" -"medianBFloat16WeightedArgMin" -"medianBFloat16WeightedArray" -"medianBFloat16WeightedDistinct" -"medianBFloat16WeightedForEach" -"medianBFloat16WeightedIf" -"medianBFloat16WeightedMap" -"medianBFloat16WeightedMerge" -"medianBFloat16WeightedNull" -"medianBFloat16WeightedOrDefault" -"medianBFloat16WeightedOrNull" -"medianBFloat16WeightedResample" -"medianBFloat16WeightedSimpleState" -"medianBFloat16WeightedState" -"medianDD" -"medianDDArgMax" -"medianDDArgMin" -"medianDDArray" -"medianDDDistinct" -"medianDDForEach" -"medianDDIf" -"medianDDMap" -"medianDDMerge" -"medianDDNull" -"medianDDOrDefault" -"medianDDOrNull" -"medianDDResample" -"medianDDSimpleState" -"medianDDState" -"medianDeterministic" -"medianDeterministicArgMax" -"medianDeterministicArgMin" -"medianDeterministicArray" -"medianDeterministicDistinct" -"medianDeterministicForEach" -"medianDeterministicIf" -"medianDeterministicMap" -"medianDeterministicMerge" -"medianDeterministicNull" -"medianDeterministicOrDefault" -"medianDeterministicOrNull" -"medianDeterministicResample" -"medianDeterministicSimpleState" -"medianDeterministicState" -"medianDistinct" -"medianExact" -"medianExactArgMax" -"medianExactArgMin" -"medianExactArray" -"medianExactDistinct" -"medianExactForEach" -"medianExactHigh" -"medianExactHighArgMax" -"medianExactHighArgMin" -"medianExactHighArray" -"medianExactHighDistinct" -"medianExactHighForEach" -"medianExactHighIf" -"medianExactHighMap" -"medianExactHighMerge" -"medianExactHighNull" -"medianExactHighOrDefault" -"medianExactHighOrNull" -"medianExactHighResample" -"medianExactHighSimpleState" -"medianExactHighState" -"medianExactIf" -"medianExactLow" -"medianExactLowArgMax" -"medianExactLowArgMin" -"medianExactLowArray" -"medianExactLowDistinct" -"medianExactLowForEach" -"medianExactLowIf" -"medianExactLowMap" -"medianExactLowMerge" -"medianExactLowNull" -"medianExactLowOrDefault" -"medianExactLowOrNull" -"medianExactLowResample" -"medianExactLowSimpleState" -"medianExactLowState" -"medianExactMap" -"medianExactMerge" -"medianExactNull" -"medianExactOrDefault" -"medianExactOrNull" -"medianExactResample" -"medianExactSimpleState" -"medianExactState" -"medianExactWeighted" -"medianExactWeightedArgMax" -"medianExactWeightedArgMin" -"medianExactWeightedArray" -"medianExactWeightedDistinct" -"medianExactWeightedForEach" -"medianExactWeightedIf" -"medianExactWeightedMap" -"medianExactWeightedMerge" -"medianExactWeightedNull" -"medianExactWeightedOrDefault" -"medianExactWeightedOrNull" -"medianExactWeightedResample" -"medianExactWeightedSimpleState" -"medianExactWeightedState" -"medianForEach" -"medianGK" -"medianGKArgMax" -"medianGKArgMin" -"medianGKArray" -"medianGKDistinct" -"medianGKForEach" -"medianGKIf" -"medianGKMap" -"medianGKMerge" -"medianGKNull" -"medianGKOrDefault" -"medianGKOrNull" -"medianGKResample" -"medianGKSimpleState" -"medianGKState" -"medianIf" -"medianInterpolatedWeighted" -"medianInterpolatedWeightedArgMax" -"medianInterpolatedWeightedArgMin" -"medianInterpolatedWeightedArray" -"medianInterpolatedWeightedDistinct" -"medianInterpolatedWeightedForEach" -"medianInterpolatedWeightedIf" -"medianInterpolatedWeightedMap" -"medianInterpolatedWeightedMerge" -"medianInterpolatedWeightedNull" -"medianInterpolatedWeightedOrDefault" -"medianInterpolatedWeightedOrNull" -"medianInterpolatedWeightedResample" -"medianInterpolatedWeightedSimpleState" -"medianInterpolatedWeightedState" -"medianMap" -"medianMerge" -"medianNull" -"medianOrDefault" -"medianOrNull" -"medianResample" -"medianSimpleState" -"medianState" -"medianTDigest" -"medianTDigestArgMax" -"medianTDigestArgMin" -"medianTDigestArray" -"medianTDigestDistinct" -"medianTDigestForEach" -"medianTDigestIf" -"medianTDigestMap" -"medianTDigestMerge" -"medianTDigestNull" -"medianTDigestOrDefault" -"medianTDigestOrNull" -"medianTDigestResample" -"medianTDigestSimpleState" -"medianTDigestState" -"medianTDigestWeighted" -"medianTDigestWeightedArgMax" -"medianTDigestWeightedArgMin" -"medianTDigestWeightedArray" -"medianTDigestWeightedDistinct" -"medianTDigestWeightedForEach" -"medianTDigestWeightedIf" -"medianTDigestWeightedMap" -"medianTDigestWeightedMerge" -"medianTDigestWeightedNull" -"medianTDigestWeightedOrDefault" -"medianTDigestWeightedOrNull" -"medianTDigestWeightedResample" -"medianTDigestWeightedSimpleState" -"medianTDigestWeightedState" -"medianTiming" -"medianTimingArgMax" -"medianTimingArgMin" -"medianTimingArray" -"medianTimingDistinct" -"medianTimingForEach" -"medianTimingIf" -"medianTimingMap" -"medianTimingMerge" -"medianTimingNull" -"medianTimingOrDefault" -"medianTimingOrNull" -"medianTimingResample" -"medianTimingSimpleState" -"medianTimingState" -"medianTimingWeighted" -"medianTimingWeightedArgMax" -"medianTimingWeightedArgMin" -"medianTimingWeightedArray" -"medianTimingWeightedDistinct" -"medianTimingWeightedForEach" -"medianTimingWeightedIf" -"medianTimingWeightedMap" -"medianTimingWeightedMerge" -"medianTimingWeightedNull" -"medianTimingWeightedOrDefault" -"medianTimingWeightedOrNull" -"medianTimingWeightedResample" -"medianTimingWeightedSimpleState" -"medianTimingWeightedState" -"metroHash64" -"mid" -"min" -"min2" -"minArgMax" -"minArgMin" -"minArray" -"minDistinct" -"minForEach" -"minIf" -"minMap" -"minMappedArrays" -"minMappedArraysArgMax" -"minMappedArraysArgMin" -"minMappedArraysArray" -"minMappedArraysDistinct" -"minMappedArraysForEach" -"minMappedArraysIf" -"minMappedArraysMap" -"minMappedArraysMerge" -"minMappedArraysNull" -"minMappedArraysOrDefault" -"minMappedArraysOrNull" -"minMappedArraysResample" -"minMappedArraysSimpleState" -"minMappedArraysState" -"minMerge" -"minNull" -"minOrDefault" -"minOrNull" -"minResample" -"minSampleSizeContinous" -"minSampleSizeContinuous" -"minSampleSizeConversion" -"minSimpleState" -"minState" -"minus" -"mismatches" -"mod" -"modulo" -"moduloLegacy" -"moduloOrZero" -"monthName" -"mortonDecode" -"mortonEncode" -"multiFuzzyMatchAllIndices" -"multiFuzzyMatchAny" -"multiFuzzyMatchAnyIndex" -"multiIf" -"multiMatchAllIndices" -"multiMatchAny" -"multiMatchAnyIndex" -"multiSearchAllPositions" -"multiSearchAllPositionsCaseInsensitive" -"multiSearchAllPositionsCaseInsensitiveUTF8" -"multiSearchAllPositionsUTF8" -"multiSearchAny" -"multiSearchAnyCaseInsensitive" -"multiSearchAnyCaseInsensitiveUTF8" -"multiSearchAnyUTF8" -"multiSearchFirstIndex" -"multiSearchFirstIndexCaseInsensitive" -"multiSearchFirstIndexCaseInsensitiveUTF8" -"multiSearchFirstIndexUTF8" -"multiSearchFirstPosition" -"multiSearchFirstPositionCaseInsensitive" -"multiSearchFirstPositionCaseInsensitiveUTF8" -"multiSearchFirstPositionUTF8" -"multiply" -"multiplyDecimal" -"murmurHash2_32" -"murmurHash2_64" -"murmurHash3_128" -"murmurHash3_32" -"murmurHash3_64" -"negate" -"neighbor" -"nested" -"netloc" -"ngramDistance" -"ngramDistanceCaseInsensitive" -"ngramDistanceCaseInsensitiveUTF8" -"ngramDistanceUTF8" -"ngramMinHash" -"ngramMinHashArg" -"ngramMinHashArgCaseInsensitive" -"ngramMinHashArgCaseInsensitiveUTF8" -"ngramMinHashArgUTF8" -"ngramMinHashCaseInsensitive" -"ngramMinHashCaseInsensitiveUTF8" -"ngramMinHashUTF8" -"ngramSearch" -"ngramSearchCaseInsensitive" -"ngramSearchCaseInsensitiveUTF8" -"ngramSearchUTF8" -"ngramSimHash" -"ngramSimHashCaseInsensitive" -"ngramSimHashCaseInsensitiveUTF8" -"ngramSimHashUTF8" -"ngrams" -"nonNegativeDerivative" -"nonNegativeDerivativeArgMax" -"nonNegativeDerivativeArgMin" -"nonNegativeDerivativeArray" -"nonNegativeDerivativeDistinct" -"nonNegativeDerivativeForEach" -"nonNegativeDerivativeIf" -"nonNegativeDerivativeMap" -"nonNegativeDerivativeMerge" -"nonNegativeDerivativeNull" -"nonNegativeDerivativeOrDefault" -"nonNegativeDerivativeOrNull" -"nonNegativeDerivativeResample" -"nonNegativeDerivativeSimpleState" -"nonNegativeDerivativeState" -"normL1" -"normL2" -"normL2Squared" -"normLinf" -"normLp" -"normalizeL1" -"normalizeL2" -"normalizeLinf" -"normalizeLp" -"normalizeQuery" -"normalizeQueryKeepNames" -"normalizeUTF8NFC" -"normalizeUTF8NFD" -"normalizeUTF8NFKC" -"normalizeUTF8NFKD" -"normalizedQueryHash" -"normalizedQueryHashKeepNames" -"not" -"notEmpty" -"notEquals" -"notILike" -"notIn" -"notInIgnoreSet" -"notLike" -"notNullIn" -"notNullInIgnoreSet" -"nothing" -"nothingArgMax" -"nothingArgMin" -"nothingArray" -"nothingDistinct" -"nothingForEach" -"nothingIf" -"nothingMap" -"nothingMerge" -"nothingNull" -"nothingNull" -"nothingNullArgMax" -"nothingNullArgMin" -"nothingNullArray" -"nothingNullDistinct" -"nothingNullForEach" -"nothingNullIf" -"nothingNullMap" -"nothingNullMerge" -"nothingNullNull" -"nothingNullOrDefault" -"nothingNullOrNull" -"nothingNullResample" -"nothingNullSimpleState" -"nothingNullState" -"nothingOrDefault" -"nothingOrNull" -"nothingResample" -"nothingSimpleState" -"nothingState" -"nothingUInt64" -"nothingUInt64ArgMax" -"nothingUInt64ArgMin" -"nothingUInt64Array" -"nothingUInt64Distinct" -"nothingUInt64ForEach" -"nothingUInt64If" -"nothingUInt64Map" -"nothingUInt64Merge" -"nothingUInt64Null" -"nothingUInt64OrDefault" -"nothingUInt64OrNull" -"nothingUInt64Resample" -"nothingUInt64SimpleState" -"nothingUInt64State" -"now" -"now64" -"nowInBlock" -"nth_value" -"nth_valueArgMax" -"nth_valueArgMin" -"nth_valueArray" -"nth_valueDistinct" -"nth_valueForEach" -"nth_valueIf" -"nth_valueMap" -"nth_valueMerge" -"nth_valueNull" -"nth_valueOrDefault" -"nth_valueOrNull" -"nth_valueResample" -"nth_valueSimpleState" -"nth_valueState" -"ntile" -"ntileArgMax" -"ntileArgMin" -"ntileArray" -"ntileDistinct" -"ntileForEach" -"ntileIf" -"ntileMap" -"ntileMerge" -"ntileNull" -"ntileOrDefault" -"ntileOrNull" -"ntileResample" -"ntileSimpleState" -"ntileState" -"nullIf" -"nullIn" -"nullInIgnoreSet" -"or" -"parseDateTime" -"parseDateTime32BestEffort" -"parseDateTime32BestEffortOrNull" -"parseDateTime32BestEffortOrZero" -"parseDateTime64BestEffort" -"parseDateTime64BestEffortOrNull" -"parseDateTime64BestEffortOrZero" -"parseDateTime64BestEffortUS" -"parseDateTime64BestEffortUSOrNull" -"parseDateTime64BestEffortUSOrZero" -"parseDateTimeBestEffort" -"parseDateTimeBestEffortOrNull" -"parseDateTimeBestEffortOrZero" -"parseDateTimeBestEffortUS" -"parseDateTimeBestEffortUSOrNull" -"parseDateTimeBestEffortUSOrZero" -"parseDateTimeInJodaSyntax" -"parseDateTimeInJodaSyntaxOrNull" -"parseDateTimeInJodaSyntaxOrZero" -"parseDateTimeOrNull" -"parseDateTimeOrZero" -"parseReadableSize" -"parseReadableSizeOrNull" -"parseReadableSizeOrZero" -"parseTimeDelta" -"partitionID" -"partitionId" -"path" -"pathFull" -"percentRank" -"percentRankArgMax" -"percentRankArgMin" -"percentRankArray" -"percentRankDistinct" -"percentRankForEach" -"percentRankIf" -"percentRankMap" -"percentRankMerge" -"percentRankNull" -"percentRankOrDefault" -"percentRankOrNull" -"percentRankResample" -"percentRankSimpleState" -"percentRankState" -"percent_rank" -"percent_rankArgMax" -"percent_rankArgMin" -"percent_rankArray" -"percent_rankDistinct" -"percent_rankForEach" -"percent_rankIf" -"percent_rankMap" -"percent_rankMerge" -"percent_rankNull" -"percent_rankOrDefault" -"percent_rankOrNull" -"percent_rankResample" -"percent_rankSimpleState" -"percent_rankState" -"pi" -"plus" -"pmod" -"pointInEllipses" -"pointInPolygon" -"polygonAreaCartesian" -"polygonAreaSpherical" -"polygonConvexHullCartesian" -"polygonPerimeterCartesian" -"polygonPerimeterSpherical" -"polygonsDistanceCartesian" -"polygonsDistanceSpherical" -"polygonsEqualsCartesian" -"polygonsIntersectionCartesian" -"polygonsIntersectionSpherical" -"polygonsSymDifferenceCartesian" -"polygonsSymDifferenceSpherical" -"polygonsUnionCartesian" -"polygonsUnionSpherical" -"polygonsWithinCartesian" -"polygonsWithinSpherical" -"port" -"portRFC" -"position" -"positionCaseInsensitive" -"positionCaseInsensitiveUTF8" -"positionUTF8" -"positiveModulo" -"positive_modulo" -"pow" -"power" -"printf" -"proportionsZTest" -"protocol" -"punycodeDecode" -"punycodeEncode" -"quantile" -"quantileArgMax" -"quantileArgMin" -"quantileArray" -"quantileBFloat16" -"quantileBFloat16ArgMax" -"quantileBFloat16ArgMin" -"quantileBFloat16Array" -"quantileBFloat16Distinct" -"quantileBFloat16ForEach" -"quantileBFloat16If" -"quantileBFloat16Map" -"quantileBFloat16Merge" -"quantileBFloat16Null" -"quantileBFloat16OrDefault" -"quantileBFloat16OrNull" -"quantileBFloat16Resample" -"quantileBFloat16SimpleState" -"quantileBFloat16State" -"quantileBFloat16Weighted" -"quantileBFloat16WeightedArgMax" -"quantileBFloat16WeightedArgMin" -"quantileBFloat16WeightedArray" -"quantileBFloat16WeightedDistinct" -"quantileBFloat16WeightedForEach" -"quantileBFloat16WeightedIf" -"quantileBFloat16WeightedMap" -"quantileBFloat16WeightedMerge" -"quantileBFloat16WeightedNull" -"quantileBFloat16WeightedOrDefault" -"quantileBFloat16WeightedOrNull" -"quantileBFloat16WeightedResample" -"quantileBFloat16WeightedSimpleState" -"quantileBFloat16WeightedState" -"quantileDD" -"quantileDDArgMax" -"quantileDDArgMin" -"quantileDDArray" -"quantileDDDistinct" -"quantileDDForEach" -"quantileDDIf" -"quantileDDMap" -"quantileDDMerge" -"quantileDDNull" -"quantileDDOrDefault" -"quantileDDOrNull" -"quantileDDResample" -"quantileDDSimpleState" -"quantileDDState" -"quantileDeterministic" -"quantileDeterministicArgMax" -"quantileDeterministicArgMin" -"quantileDeterministicArray" -"quantileDeterministicDistinct" -"quantileDeterministicForEach" -"quantileDeterministicIf" -"quantileDeterministicMap" -"quantileDeterministicMerge" -"quantileDeterministicNull" -"quantileDeterministicOrDefault" -"quantileDeterministicOrNull" -"quantileDeterministicResample" -"quantileDeterministicSimpleState" -"quantileDeterministicState" -"quantileDistinct" -"quantileExact" -"quantileExactArgMax" -"quantileExactArgMin" -"quantileExactArray" -"quantileExactDistinct" -"quantileExactExclusive" -"quantileExactExclusiveArgMax" -"quantileExactExclusiveArgMin" -"quantileExactExclusiveArray" -"quantileExactExclusiveDistinct" -"quantileExactExclusiveForEach" -"quantileExactExclusiveIf" -"quantileExactExclusiveMap" -"quantileExactExclusiveMerge" -"quantileExactExclusiveNull" -"quantileExactExclusiveOrDefault" -"quantileExactExclusiveOrNull" -"quantileExactExclusiveResample" -"quantileExactExclusiveSimpleState" -"quantileExactExclusiveState" -"quantileExactForEach" -"quantileExactHigh" -"quantileExactHighArgMax" -"quantileExactHighArgMin" -"quantileExactHighArray" -"quantileExactHighDistinct" -"quantileExactHighForEach" -"quantileExactHighIf" -"quantileExactHighMap" -"quantileExactHighMerge" -"quantileExactHighNull" -"quantileExactHighOrDefault" -"quantileExactHighOrNull" -"quantileExactHighResample" -"quantileExactHighSimpleState" -"quantileExactHighState" -"quantileExactIf" -"quantileExactInclusive" -"quantileExactInclusiveArgMax" -"quantileExactInclusiveArgMin" -"quantileExactInclusiveArray" -"quantileExactInclusiveDistinct" -"quantileExactInclusiveForEach" -"quantileExactInclusiveIf" -"quantileExactInclusiveMap" -"quantileExactInclusiveMerge" -"quantileExactInclusiveNull" -"quantileExactInclusiveOrDefault" -"quantileExactInclusiveOrNull" -"quantileExactInclusiveResample" -"quantileExactInclusiveSimpleState" -"quantileExactInclusiveState" -"quantileExactLow" -"quantileExactLowArgMax" -"quantileExactLowArgMin" -"quantileExactLowArray" -"quantileExactLowDistinct" -"quantileExactLowForEach" -"quantileExactLowIf" -"quantileExactLowMap" -"quantileExactLowMerge" -"quantileExactLowNull" -"quantileExactLowOrDefault" -"quantileExactLowOrNull" -"quantileExactLowResample" -"quantileExactLowSimpleState" -"quantileExactLowState" -"quantileExactMap" -"quantileExactMerge" -"quantileExactNull" -"quantileExactOrDefault" -"quantileExactOrNull" -"quantileExactResample" -"quantileExactSimpleState" -"quantileExactState" -"quantileExactWeighted" -"quantileExactWeightedArgMax" -"quantileExactWeightedArgMin" -"quantileExactWeightedArray" -"quantileExactWeightedDistinct" -"quantileExactWeightedForEach" -"quantileExactWeightedIf" -"quantileExactWeightedMap" -"quantileExactWeightedMerge" -"quantileExactWeightedNull" -"quantileExactWeightedOrDefault" -"quantileExactWeightedOrNull" -"quantileExactWeightedResample" -"quantileExactWeightedSimpleState" -"quantileExactWeightedState" -"quantileForEach" -"quantileGK" -"quantileGKArgMax" -"quantileGKArgMin" -"quantileGKArray" -"quantileGKDistinct" -"quantileGKForEach" -"quantileGKIf" -"quantileGKMap" -"quantileGKMerge" -"quantileGKNull" -"quantileGKOrDefault" -"quantileGKOrNull" -"quantileGKResample" -"quantileGKSimpleState" -"quantileGKState" -"quantileIf" -"quantileInterpolatedWeighted" -"quantileInterpolatedWeightedArgMax" -"quantileInterpolatedWeightedArgMin" -"quantileInterpolatedWeightedArray" -"quantileInterpolatedWeightedDistinct" -"quantileInterpolatedWeightedForEach" -"quantileInterpolatedWeightedIf" -"quantileInterpolatedWeightedMap" -"quantileInterpolatedWeightedMerge" -"quantileInterpolatedWeightedNull" -"quantileInterpolatedWeightedOrDefault" -"quantileInterpolatedWeightedOrNull" -"quantileInterpolatedWeightedResample" -"quantileInterpolatedWeightedSimpleState" -"quantileInterpolatedWeightedState" -"quantileMap" -"quantileMerge" -"quantileNull" -"quantileOrDefault" -"quantileOrNull" -"quantileResample" -"quantileSimpleState" -"quantileState" -"quantileTDigest" -"quantileTDigestArgMax" -"quantileTDigestArgMin" -"quantileTDigestArray" -"quantileTDigestDistinct" -"quantileTDigestForEach" -"quantileTDigestIf" -"quantileTDigestMap" -"quantileTDigestMerge" -"quantileTDigestNull" -"quantileTDigestOrDefault" -"quantileTDigestOrNull" -"quantileTDigestResample" -"quantileTDigestSimpleState" -"quantileTDigestState" -"quantileTDigestWeighted" -"quantileTDigestWeightedArgMax" -"quantileTDigestWeightedArgMin" -"quantileTDigestWeightedArray" -"quantileTDigestWeightedDistinct" -"quantileTDigestWeightedForEach" -"quantileTDigestWeightedIf" -"quantileTDigestWeightedMap" -"quantileTDigestWeightedMerge" -"quantileTDigestWeightedNull" -"quantileTDigestWeightedOrDefault" -"quantileTDigestWeightedOrNull" -"quantileTDigestWeightedResample" -"quantileTDigestWeightedSimpleState" -"quantileTDigestWeightedState" -"quantileTiming" -"quantileTimingArgMax" -"quantileTimingArgMin" -"quantileTimingArray" -"quantileTimingDistinct" -"quantileTimingForEach" -"quantileTimingIf" -"quantileTimingMap" -"quantileTimingMerge" -"quantileTimingNull" -"quantileTimingOrDefault" -"quantileTimingOrNull" -"quantileTimingResample" -"quantileTimingSimpleState" -"quantileTimingState" -"quantileTimingWeighted" -"quantileTimingWeightedArgMax" -"quantileTimingWeightedArgMin" -"quantileTimingWeightedArray" -"quantileTimingWeightedDistinct" -"quantileTimingWeightedForEach" -"quantileTimingWeightedIf" -"quantileTimingWeightedMap" -"quantileTimingWeightedMerge" -"quantileTimingWeightedNull" -"quantileTimingWeightedOrDefault" -"quantileTimingWeightedOrNull" -"quantileTimingWeightedResample" -"quantileTimingWeightedSimpleState" -"quantileTimingWeightedState" -"quantiles" -"quantilesArgMax" -"quantilesArgMin" -"quantilesArray" -"quantilesBFloat16" -"quantilesBFloat16ArgMax" -"quantilesBFloat16ArgMin" -"quantilesBFloat16Array" -"quantilesBFloat16Distinct" -"quantilesBFloat16ForEach" -"quantilesBFloat16If" -"quantilesBFloat16Map" -"quantilesBFloat16Merge" -"quantilesBFloat16Null" -"quantilesBFloat16OrDefault" -"quantilesBFloat16OrNull" -"quantilesBFloat16Resample" -"quantilesBFloat16SimpleState" -"quantilesBFloat16State" -"quantilesBFloat16Weighted" -"quantilesBFloat16WeightedArgMax" -"quantilesBFloat16WeightedArgMin" -"quantilesBFloat16WeightedArray" -"quantilesBFloat16WeightedDistinct" -"quantilesBFloat16WeightedForEach" -"quantilesBFloat16WeightedIf" -"quantilesBFloat16WeightedMap" -"quantilesBFloat16WeightedMerge" -"quantilesBFloat16WeightedNull" -"quantilesBFloat16WeightedOrDefault" -"quantilesBFloat16WeightedOrNull" -"quantilesBFloat16WeightedResample" -"quantilesBFloat16WeightedSimpleState" -"quantilesBFloat16WeightedState" -"quantilesDD" -"quantilesDDArgMax" -"quantilesDDArgMin" -"quantilesDDArray" -"quantilesDDDistinct" -"quantilesDDForEach" -"quantilesDDIf" -"quantilesDDMap" -"quantilesDDMerge" -"quantilesDDNull" -"quantilesDDOrDefault" -"quantilesDDOrNull" -"quantilesDDResample" -"quantilesDDSimpleState" -"quantilesDDState" -"quantilesDeterministic" -"quantilesDeterministicArgMax" -"quantilesDeterministicArgMin" -"quantilesDeterministicArray" -"quantilesDeterministicDistinct" -"quantilesDeterministicForEach" -"quantilesDeterministicIf" -"quantilesDeterministicMap" -"quantilesDeterministicMerge" -"quantilesDeterministicNull" -"quantilesDeterministicOrDefault" -"quantilesDeterministicOrNull" -"quantilesDeterministicResample" -"quantilesDeterministicSimpleState" -"quantilesDeterministicState" -"quantilesDistinct" -"quantilesExact" -"quantilesExactArgMax" -"quantilesExactArgMin" -"quantilesExactArray" -"quantilesExactDistinct" -"quantilesExactExclusive" -"quantilesExactExclusiveArgMax" -"quantilesExactExclusiveArgMin" -"quantilesExactExclusiveArray" -"quantilesExactExclusiveDistinct" -"quantilesExactExclusiveForEach" -"quantilesExactExclusiveIf" -"quantilesExactExclusiveMap" -"quantilesExactExclusiveMerge" -"quantilesExactExclusiveNull" -"quantilesExactExclusiveOrDefault" -"quantilesExactExclusiveOrNull" -"quantilesExactExclusiveResample" -"quantilesExactExclusiveSimpleState" -"quantilesExactExclusiveState" -"quantilesExactForEach" -"quantilesExactHigh" -"quantilesExactHighArgMax" -"quantilesExactHighArgMin" -"quantilesExactHighArray" -"quantilesExactHighDistinct" -"quantilesExactHighForEach" -"quantilesExactHighIf" -"quantilesExactHighMap" -"quantilesExactHighMerge" -"quantilesExactHighNull" -"quantilesExactHighOrDefault" -"quantilesExactHighOrNull" -"quantilesExactHighResample" -"quantilesExactHighSimpleState" -"quantilesExactHighState" -"quantilesExactIf" -"quantilesExactInclusive" -"quantilesExactInclusiveArgMax" -"quantilesExactInclusiveArgMin" -"quantilesExactInclusiveArray" -"quantilesExactInclusiveDistinct" -"quantilesExactInclusiveForEach" -"quantilesExactInclusiveIf" -"quantilesExactInclusiveMap" -"quantilesExactInclusiveMerge" -"quantilesExactInclusiveNull" -"quantilesExactInclusiveOrDefault" -"quantilesExactInclusiveOrNull" -"quantilesExactInclusiveResample" -"quantilesExactInclusiveSimpleState" -"quantilesExactInclusiveState" -"quantilesExactLow" -"quantilesExactLowArgMax" -"quantilesExactLowArgMin" -"quantilesExactLowArray" -"quantilesExactLowDistinct" -"quantilesExactLowForEach" -"quantilesExactLowIf" -"quantilesExactLowMap" -"quantilesExactLowMerge" -"quantilesExactLowNull" -"quantilesExactLowOrDefault" -"quantilesExactLowOrNull" -"quantilesExactLowResample" -"quantilesExactLowSimpleState" -"quantilesExactLowState" -"quantilesExactMap" -"quantilesExactMerge" -"quantilesExactNull" -"quantilesExactOrDefault" -"quantilesExactOrNull" -"quantilesExactResample" -"quantilesExactSimpleState" -"quantilesExactState" -"quantilesExactWeighted" -"quantilesExactWeightedArgMax" -"quantilesExactWeightedArgMin" -"quantilesExactWeightedArray" -"quantilesExactWeightedDistinct" -"quantilesExactWeightedForEach" -"quantilesExactWeightedIf" -"quantilesExactWeightedMap" -"quantilesExactWeightedMerge" -"quantilesExactWeightedNull" -"quantilesExactWeightedOrDefault" -"quantilesExactWeightedOrNull" -"quantilesExactWeightedResample" -"quantilesExactWeightedSimpleState" -"quantilesExactWeightedState" -"quantilesForEach" -"quantilesGK" -"quantilesGKArgMax" -"quantilesGKArgMin" -"quantilesGKArray" -"quantilesGKDistinct" -"quantilesGKForEach" -"quantilesGKIf" -"quantilesGKMap" -"quantilesGKMerge" -"quantilesGKNull" -"quantilesGKOrDefault" -"quantilesGKOrNull" -"quantilesGKResample" -"quantilesGKSimpleState" -"quantilesGKState" -"quantilesIf" -"quantilesInterpolatedWeighted" -"quantilesInterpolatedWeightedArgMax" -"quantilesInterpolatedWeightedArgMin" -"quantilesInterpolatedWeightedArray" -"quantilesInterpolatedWeightedDistinct" -"quantilesInterpolatedWeightedForEach" -"quantilesInterpolatedWeightedIf" -"quantilesInterpolatedWeightedMap" -"quantilesInterpolatedWeightedMerge" -"quantilesInterpolatedWeightedNull" -"quantilesInterpolatedWeightedOrDefault" -"quantilesInterpolatedWeightedOrNull" -"quantilesInterpolatedWeightedResample" -"quantilesInterpolatedWeightedSimpleState" -"quantilesInterpolatedWeightedState" -"quantilesMap" -"quantilesMerge" -"quantilesNull" -"quantilesOrDefault" -"quantilesOrNull" -"quantilesResample" -"quantilesSimpleState" -"quantilesState" -"quantilesTDigest" -"quantilesTDigestArgMax" -"quantilesTDigestArgMin" -"quantilesTDigestArray" -"quantilesTDigestDistinct" -"quantilesTDigestForEach" -"quantilesTDigestIf" -"quantilesTDigestMap" -"quantilesTDigestMerge" -"quantilesTDigestNull" -"quantilesTDigestOrDefault" -"quantilesTDigestOrNull" -"quantilesTDigestResample" -"quantilesTDigestSimpleState" -"quantilesTDigestState" -"quantilesTDigestWeighted" -"quantilesTDigestWeightedArgMax" -"quantilesTDigestWeightedArgMin" -"quantilesTDigestWeightedArray" -"quantilesTDigestWeightedDistinct" -"quantilesTDigestWeightedForEach" -"quantilesTDigestWeightedIf" -"quantilesTDigestWeightedMap" -"quantilesTDigestWeightedMerge" -"quantilesTDigestWeightedNull" -"quantilesTDigestWeightedOrDefault" -"quantilesTDigestWeightedOrNull" -"quantilesTDigestWeightedResample" -"quantilesTDigestWeightedSimpleState" -"quantilesTDigestWeightedState" -"quantilesTiming" -"quantilesTimingArgMax" -"quantilesTimingArgMin" -"quantilesTimingArray" -"quantilesTimingDistinct" -"quantilesTimingForEach" -"quantilesTimingIf" -"quantilesTimingMap" -"quantilesTimingMerge" -"quantilesTimingNull" -"quantilesTimingOrDefault" -"quantilesTimingOrNull" -"quantilesTimingResample" -"quantilesTimingSimpleState" -"quantilesTimingState" -"quantilesTimingWeighted" -"quantilesTimingWeightedArgMax" -"quantilesTimingWeightedArgMin" -"quantilesTimingWeightedArray" -"quantilesTimingWeightedDistinct" -"quantilesTimingWeightedForEach" -"quantilesTimingWeightedIf" -"quantilesTimingWeightedMap" -"quantilesTimingWeightedMerge" -"quantilesTimingWeightedNull" -"quantilesTimingWeightedOrDefault" -"quantilesTimingWeightedOrNull" -"quantilesTimingWeightedResample" -"quantilesTimingWeightedSimpleState" -"quantilesTimingWeightedState" -"queryID" -"queryString" -"queryStringAndFragment" -"query_id" -"radians" -"rand" -"rand32" -"rand64" -"randBernoulli" -"randBinomial" -"randCanonical" -"randChiSquared" -"randConstant" -"randExponential" -"randFisherF" -"randLogNormal" -"randNegativeBinomial" -"randNormal" -"randPoisson" -"randStudentT" -"randUniform" -"randomFixedString" -"randomPrintableASCII" -"randomString" -"randomStringUTF8" -"range" -"rank" -"rankArgMax" -"rankArgMin" -"rankArray" -"rankCorr" -"rankCorrArgMax" -"rankCorrArgMin" -"rankCorrArray" -"rankCorrDistinct" -"rankCorrForEach" -"rankCorrIf" -"rankCorrMap" -"rankCorrMerge" -"rankCorrNull" -"rankCorrOrDefault" -"rankCorrOrNull" -"rankCorrResample" -"rankCorrSimpleState" -"rankCorrState" -"rankDistinct" -"rankForEach" -"rankIf" -"rankMap" -"rankMerge" -"rankNull" -"rankOrDefault" -"rankOrNull" -"rankResample" -"rankSimpleState" -"rankState" -"readWKTLineString" -"readWKTMultiLineString" -"readWKTMultiPolygon" -"readWKTPoint" -"readWKTPolygon" -"readWKTRing" -"regexpExtract" -"regexpQuoteMeta" -"regionHierarchy" -"regionIn" -"regionToArea" -"regionToCity" -"regionToContinent" -"regionToCountry" -"regionToDistrict" -"regionToName" -"regionToPopulation" -"regionToTopContinent" -"reinterpret" -"reinterpretAsDate" -"reinterpretAsDateTime" -"reinterpretAsFixedString" -"reinterpretAsFloat32" -"reinterpretAsFloat64" -"reinterpretAsInt128" -"reinterpretAsInt16" -"reinterpretAsInt256" -"reinterpretAsInt32" -"reinterpretAsInt64" -"reinterpretAsInt8" -"reinterpretAsString" -"reinterpretAsUInt128" -"reinterpretAsUInt16" -"reinterpretAsUInt256" -"reinterpretAsUInt32" -"reinterpretAsUInt64" -"reinterpretAsUInt8" -"reinterpretAsUUID" -"repeat" -"replace" -"replaceAll" -"replaceOne" -"replaceRegexpAll" -"replaceRegexpOne" -"replicate" -"retention" -"retentionArgMax" -"retentionArgMin" -"retentionArray" -"retentionDistinct" -"retentionForEach" -"retentionIf" -"retentionMap" -"retentionMerge" -"retentionNull" -"retentionOrDefault" -"retentionOrNull" -"retentionResample" -"retentionSimpleState" -"retentionState" -"reverse" -"reverseUTF8" -"revision" -"right" -"rightPad" -"rightPadUTF8" -"rightUTF8" -"round" -"roundAge" -"roundBankers" -"roundDown" -"roundDuration" -"roundToExp2" -"rowNumberInAllBlocks" -"rowNumberInBlock" -"row_number" -"row_numberArgMax" -"row_numberArgMin" -"row_numberArray" -"row_numberDistinct" -"row_numberForEach" -"row_numberIf" -"row_numberMap" -"row_numberMerge" -"row_numberNull" -"row_numberOrDefault" -"row_numberOrNull" -"row_numberResample" -"row_numberSimpleState" -"row_numberState" -"rpad" -"rtrim" -"runningAccumulate" -"runningConcurrency" -"runningDifference" -"runningDifferenceStartingWithFirstValue" -"s2CapContains" -"s2CapUnion" -"s2CellsIntersect" -"s2GetNeighbors" -"s2RectAdd" -"s2RectContains" -"s2RectIntersection" -"s2RectUnion" -"s2ToGeo" -"scalarProduct" -"sequenceCount" -"sequenceCountArgMax" -"sequenceCountArgMin" -"sequenceCountArray" -"sequenceCountDistinct" -"sequenceCountForEach" -"sequenceCountIf" -"sequenceCountMap" -"sequenceCountMerge" -"sequenceCountNull" -"sequenceCountOrDefault" -"sequenceCountOrNull" -"sequenceCountResample" -"sequenceCountSimpleState" -"sequenceCountState" -"sequenceMatch" -"sequenceMatchArgMax" -"sequenceMatchArgMin" -"sequenceMatchArray" -"sequenceMatchDistinct" -"sequenceMatchForEach" -"sequenceMatchIf" -"sequenceMatchMap" -"sequenceMatchMerge" -"sequenceMatchNull" -"sequenceMatchOrDefault" -"sequenceMatchOrNull" -"sequenceMatchResample" -"sequenceMatchSimpleState" -"sequenceMatchState" -"sequenceNextNode" -"sequenceNextNodeArgMax" -"sequenceNextNodeArgMin" -"sequenceNextNodeArray" -"sequenceNextNodeDistinct" -"sequenceNextNodeForEach" -"sequenceNextNodeIf" -"sequenceNextNodeMap" -"sequenceNextNodeMerge" -"sequenceNextNodeNull" -"sequenceNextNodeOrDefault" -"sequenceNextNodeOrNull" -"sequenceNextNodeResample" -"sequenceNextNodeSimpleState" -"sequenceNextNodeState" -"seriesDecomposeSTL" -"seriesOutliersDetectTukey" -"seriesPeriodDetectFFT" -"serverTimeZone" -"serverTimezone" -"serverUUID" -"shardCount" -"shardNum" -"showCertificate" -"sigmoid" -"sign" -"simpleJSONExtractBool" -"simpleJSONExtractFloat" -"simpleJSONExtractInt" -"simpleJSONExtractRaw" -"simpleJSONExtractString" -"simpleJSONExtractUInt" -"simpleJSONHas" -"simpleLinearRegression" -"simpleLinearRegressionArgMax" -"simpleLinearRegressionArgMin" -"simpleLinearRegressionArray" -"simpleLinearRegressionDistinct" -"simpleLinearRegressionForEach" -"simpleLinearRegressionIf" -"simpleLinearRegressionMap" -"simpleLinearRegressionMerge" -"simpleLinearRegressionNull" -"simpleLinearRegressionOrDefault" -"simpleLinearRegressionOrNull" -"simpleLinearRegressionResample" -"simpleLinearRegressionSimpleState" -"simpleLinearRegressionState" -"sin" -"singleValueOrNull" -"singleValueOrNullArgMax" -"singleValueOrNullArgMin" -"singleValueOrNullArray" -"singleValueOrNullDistinct" -"singleValueOrNullForEach" -"singleValueOrNullIf" -"singleValueOrNullMap" -"singleValueOrNullMerge" -"singleValueOrNullNull" -"singleValueOrNullOrDefault" -"singleValueOrNullOrNull" -"singleValueOrNullResample" -"singleValueOrNullSimpleState" -"singleValueOrNullState" -"sinh" -"sipHash128" -"sipHash128Keyed" -"sipHash128Reference" -"sipHash128ReferenceKeyed" -"sipHash64" -"sipHash64Keyed" -"skewPop" -"skewPopArgMax" -"skewPopArgMin" -"skewPopArray" -"skewPopDistinct" -"skewPopForEach" -"skewPopIf" -"skewPopMap" -"skewPopMerge" -"skewPopNull" -"skewPopOrDefault" -"skewPopOrNull" -"skewPopResample" -"skewPopSimpleState" -"skewPopState" -"skewSamp" -"skewSampArgMax" -"skewSampArgMin" -"skewSampArray" -"skewSampDistinct" -"skewSampForEach" -"skewSampIf" -"skewSampMap" -"skewSampMerge" -"skewSampNull" -"skewSampOrDefault" -"skewSampOrNull" -"skewSampResample" -"skewSampSimpleState" -"skewSampState" -"sleep" -"sleepEachRow" -"snowflakeIDToDateTime" -"snowflakeIDToDateTime64" -"snowflakeToDateTime" -"snowflakeToDateTime64" -"soundex" -"space" -"sparkBar" -"sparkBarArgMax" -"sparkBarArgMin" -"sparkBarArray" -"sparkBarDistinct" -"sparkBarForEach" -"sparkBarIf" -"sparkBarMap" -"sparkBarMerge" -"sparkBarNull" -"sparkBarOrDefault" -"sparkBarOrNull" -"sparkBarResample" -"sparkBarSimpleState" -"sparkBarState" -"sparkbar" -"sparkbarArgMax" -"sparkbarArgMin" -"sparkbarArray" -"sparkbarDistinct" -"sparkbarForEach" -"sparkbarIf" -"sparkbarMap" -"sparkbarMerge" -"sparkbarNull" -"sparkbarOrDefault" -"sparkbarOrNull" -"sparkbarResample" -"sparkbarSimpleState" -"sparkbarState" -"splitByAlpha" -"splitByChar" -"splitByNonAlpha" -"splitByRegexp" -"splitByString" -"splitByWhitespace" -"sqid" -"sqidDecode" -"sqidEncode" -"sqrt" -"startsWith" -"startsWithUTF8" -"stddevPop" -"stddevPopArgMax" -"stddevPopArgMin" -"stddevPopArray" -"stddevPopDistinct" -"stddevPopForEach" -"stddevPopIf" -"stddevPopMap" -"stddevPopMerge" -"stddevPopNull" -"stddevPopOrDefault" -"stddevPopOrNull" -"stddevPopResample" -"stddevPopSimpleState" -"stddevPopStable" -"stddevPopStableArgMax" -"stddevPopStableArgMin" -"stddevPopStableArray" -"stddevPopStableDistinct" -"stddevPopStableForEach" -"stddevPopStableIf" -"stddevPopStableMap" -"stddevPopStableMerge" -"stddevPopStableNull" -"stddevPopStableOrDefault" -"stddevPopStableOrNull" -"stddevPopStableResample" -"stddevPopStableSimpleState" -"stddevPopStableState" -"stddevPopState" -"stddevSamp" -"stddevSampArgMax" -"stddevSampArgMin" -"stddevSampArray" -"stddevSampDistinct" -"stddevSampForEach" -"stddevSampIf" -"stddevSampMap" -"stddevSampMerge" -"stddevSampNull" -"stddevSampOrDefault" -"stddevSampOrNull" -"stddevSampResample" -"stddevSampSimpleState" -"stddevSampStable" -"stddevSampStableArgMax" -"stddevSampStableArgMin" -"stddevSampStableArray" -"stddevSampStableDistinct" -"stddevSampStableForEach" -"stddevSampStableIf" -"stddevSampStableMap" -"stddevSampStableMerge" -"stddevSampStableNull" -"stddevSampStableOrDefault" -"stddevSampStableOrNull" -"stddevSampStableResample" -"stddevSampStableSimpleState" -"stddevSampStableState" -"stddevSampState" -"stem" -"stochasticLinearRegression" -"stochasticLinearRegressionArgMax" -"stochasticLinearRegressionArgMin" -"stochasticLinearRegressionArray" -"stochasticLinearRegressionDistinct" -"stochasticLinearRegressionForEach" -"stochasticLinearRegressionIf" -"stochasticLinearRegressionMap" -"stochasticLinearRegressionMerge" -"stochasticLinearRegressionNull" -"stochasticLinearRegressionOrDefault" -"stochasticLinearRegressionOrNull" -"stochasticLinearRegressionResample" -"stochasticLinearRegressionSimpleState" -"stochasticLinearRegressionState" -"stochasticLogisticRegression" -"stochasticLogisticRegressionArgMax" -"stochasticLogisticRegressionArgMin" -"stochasticLogisticRegressionArray" -"stochasticLogisticRegressionDistinct" -"stochasticLogisticRegressionForEach" -"stochasticLogisticRegressionIf" -"stochasticLogisticRegressionMap" -"stochasticLogisticRegressionMerge" -"stochasticLogisticRegressionNull" -"stochasticLogisticRegressionOrDefault" -"stochasticLogisticRegressionOrNull" -"stochasticLogisticRegressionResample" -"stochasticLogisticRegressionSimpleState" -"stochasticLogisticRegressionState" -"str_to_date" -"str_to_map" -"stringJaccardIndex" -"stringJaccardIndexUTF8" -"stringToH3" -"structureToCapnProtoSchema" -"structureToProtobufSchema" -"studentTTest" -"studentTTestArgMax" -"studentTTestArgMin" -"studentTTestArray" -"studentTTestDistinct" -"studentTTestForEach" -"studentTTestIf" -"studentTTestMap" -"studentTTestMerge" -"studentTTestNull" -"studentTTestOrDefault" -"studentTTestOrNull" -"studentTTestResample" -"studentTTestSimpleState" -"studentTTestState" -"subBitmap" -"subDate" -"substr" -"substring" -"substringIndex" -"substringIndexUTF8" -"substringUTF8" -"subtractDays" -"subtractHours" -"subtractInterval" -"subtractMicroseconds" -"subtractMilliseconds" -"subtractMinutes" -"subtractMonths" -"subtractNanoseconds" -"subtractQuarters" -"subtractSeconds" -"subtractTupleOfIntervals" -"subtractWeeks" -"subtractYears" -"sum" -"sumArgMax" -"sumArgMin" -"sumArray" -"sumCount" -"sumCountArgMax" -"sumCountArgMin" -"sumCountArray" -"sumCountDistinct" -"sumCountForEach" -"sumCountIf" -"sumCountMap" -"sumCountMerge" -"sumCountNull" -"sumCountOrDefault" -"sumCountOrNull" -"sumCountResample" -"sumCountSimpleState" -"sumCountState" -"sumDistinct" -"sumForEach" -"sumIf" -"sumKahan" -"sumKahanArgMax" -"sumKahanArgMin" -"sumKahanArray" -"sumKahanDistinct" -"sumKahanForEach" -"sumKahanIf" -"sumKahanMap" -"sumKahanMerge" -"sumKahanNull" -"sumKahanOrDefault" -"sumKahanOrNull" -"sumKahanResample" -"sumKahanSimpleState" -"sumKahanState" -"sumMap" -"sumMapFiltered" -"sumMapFilteredArgMax" -"sumMapFilteredArgMin" -"sumMapFilteredArray" -"sumMapFilteredDistinct" -"sumMapFilteredForEach" -"sumMapFilteredIf" -"sumMapFilteredMap" -"sumMapFilteredMerge" -"sumMapFilteredNull" -"sumMapFilteredOrDefault" -"sumMapFilteredOrNull" -"sumMapFilteredResample" -"sumMapFilteredSimpleState" -"sumMapFilteredState" -"sumMapFilteredWithOverflow" -"sumMapFilteredWithOverflowArgMax" -"sumMapFilteredWithOverflowArgMin" -"sumMapFilteredWithOverflowArray" -"sumMapFilteredWithOverflowDistinct" -"sumMapFilteredWithOverflowForEach" -"sumMapFilteredWithOverflowIf" -"sumMapFilteredWithOverflowMap" -"sumMapFilteredWithOverflowMerge" -"sumMapFilteredWithOverflowNull" -"sumMapFilteredWithOverflowOrDefault" -"sumMapFilteredWithOverflowOrNull" -"sumMapFilteredWithOverflowResample" -"sumMapFilteredWithOverflowSimpleState" -"sumMapFilteredWithOverflowState" -"sumMapWithOverflow" -"sumMapWithOverflowArgMax" -"sumMapWithOverflowArgMin" -"sumMapWithOverflowArray" -"sumMapWithOverflowDistinct" -"sumMapWithOverflowForEach" -"sumMapWithOverflowIf" -"sumMapWithOverflowMap" -"sumMapWithOverflowMerge" -"sumMapWithOverflowNull" -"sumMapWithOverflowOrDefault" -"sumMapWithOverflowOrNull" -"sumMapWithOverflowResample" -"sumMapWithOverflowSimpleState" -"sumMapWithOverflowState" -"sumMappedArrays" -"sumMappedArraysArgMax" -"sumMappedArraysArgMin" -"sumMappedArraysArray" -"sumMappedArraysDistinct" -"sumMappedArraysForEach" -"sumMappedArraysIf" -"sumMappedArraysMap" -"sumMappedArraysMerge" -"sumMappedArraysNull" -"sumMappedArraysOrDefault" -"sumMappedArraysOrNull" -"sumMappedArraysResample" -"sumMappedArraysSimpleState" -"sumMappedArraysState" -"sumMerge" -"sumNull" -"sumOrDefault" -"sumOrNull" -"sumResample" -"sumSimpleState" -"sumState" -"sumWithOverflow" -"sumWithOverflowArgMax" -"sumWithOverflowArgMin" -"sumWithOverflowArray" -"sumWithOverflowDistinct" -"sumWithOverflowForEach" -"sumWithOverflowIf" -"sumWithOverflowMap" -"sumWithOverflowMerge" -"sumWithOverflowNull" -"sumWithOverflowOrDefault" -"sumWithOverflowOrNull" -"sumWithOverflowResample" -"sumWithOverflowSimpleState" -"sumWithOverflowState" -"svg" -"synonyms" -"tan" -"tanh" -"tcpPort" -"tgamma" -"theilsU" -"theilsUArgMax" -"theilsUArgMin" -"theilsUArray" -"theilsUDistinct" -"theilsUForEach" -"theilsUIf" -"theilsUMap" -"theilsUMerge" -"theilsUNull" -"theilsUOrDefault" -"theilsUOrNull" -"theilsUResample" -"theilsUSimpleState" -"theilsUState" -"throwIf" -"tid" -"timeDiff" -"timeSlot" -"timeSlots" -"timeZone" -"timeZoneOf" -"timeZoneOffset" -"timestamp" -"timestampDiff" -"timestamp_diff" -"timezone" -"timezoneOf" -"timezoneOffset" -"toBool" -"toColumnTypeName" -"toDate" -"toDate32" -"toDate32OrDefault" -"toDate32OrNull" -"toDate32OrZero" -"toDateOrDefault" -"toDateOrNull" -"toDateOrZero" -"toDateTime" -"toDateTime32" -"toDateTime64" -"toDateTime64OrDefault" -"toDateTime64OrNull" -"toDateTime64OrZero" -"toDateTimeOrDefault" -"toDateTimeOrNull" -"toDateTimeOrZero" -"toDayOfMonth" -"toDayOfWeek" -"toDayOfYear" -"toDaysSinceYearZero" -"toDecimal128" -"toDecimal128OrDefault" -"toDecimal128OrNull" -"toDecimal128OrZero" -"toDecimal256" -"toDecimal256OrDefault" -"toDecimal256OrNull" -"toDecimal256OrZero" -"toDecimal32" -"toDecimal32OrDefault" -"toDecimal32OrNull" -"toDecimal32OrZero" -"toDecimal64" -"toDecimal64OrDefault" -"toDecimal64OrNull" -"toDecimal64OrZero" -"toDecimalString" -"toFixedString" -"toFloat32" -"toFloat32OrDefault" -"toFloat32OrNull" -"toFloat32OrZero" -"toFloat64" -"toFloat64OrDefault" -"toFloat64OrNull" -"toFloat64OrZero" -"toHour" -"toIPv4" -"toIPv4OrDefault" -"toIPv4OrNull" -"toIPv4OrZero" -"toIPv6" -"toIPv6OrDefault" -"toIPv6OrNull" -"toIPv6OrZero" -"toISOWeek" -"toISOYear" -"toInt128" -"toInt128OrDefault" -"toInt128OrNull" -"toInt128OrZero" -"toInt16" -"toInt16OrDefault" -"toInt16OrNull" -"toInt16OrZero" -"toInt256" -"toInt256OrDefault" -"toInt256OrNull" -"toInt256OrZero" -"toInt32" -"toInt32OrDefault" -"toInt32OrNull" -"toInt32OrZero" -"toInt64" -"toInt64OrDefault" -"toInt64OrNull" -"toInt64OrZero" -"toInt8" -"toInt8OrDefault" -"toInt8OrNull" -"toInt8OrZero" -"toIntervalDay" -"toIntervalHour" -"toIntervalMicrosecond" -"toIntervalMillisecond" -"toIntervalMinute" -"toIntervalMonth" -"toIntervalNanosecond" -"toIntervalQuarter" -"toIntervalSecond" -"toIntervalWeek" -"toIntervalYear" -"toJSONString" -"toLastDayOfMonth" -"toLastDayOfWeek" -"toLowCardinality" -"toMillisecond" -"toMinute" -"toModifiedJulianDay" -"toModifiedJulianDayOrNull" -"toMonday" -"toMonth" -"toNullable" -"toQuarter" -"toRelativeDayNum" -"toRelativeHourNum" -"toRelativeMinuteNum" -"toRelativeMonthNum" -"toRelativeQuarterNum" -"toRelativeSecondNum" -"toRelativeWeekNum" -"toRelativeYearNum" -"toSecond" -"toStartOfDay" -"toStartOfFifteenMinutes" -"toStartOfFiveMinute" -"toStartOfFiveMinutes" -"toStartOfHour" -"toStartOfISOYear" -"toStartOfInterval" -"toStartOfMicrosecond" -"toStartOfMillisecond" -"toStartOfMinute" -"toStartOfMonth" -"toStartOfNanosecond" -"toStartOfQuarter" -"toStartOfSecond" -"toStartOfTenMinutes" -"toStartOfWeek" -"toStartOfYear" -"toString" -"toStringCutToZero" -"toTime" -"toTimeZone" -"toTimezone" -"toTypeName" -"toUInt128" -"toUInt128OrDefault" -"toUInt128OrNull" -"toUInt128OrZero" -"toUInt16" -"toUInt16OrDefault" -"toUInt16OrNull" -"toUInt16OrZero" -"toUInt256" -"toUInt256OrDefault" -"toUInt256OrNull" -"toUInt256OrZero" -"toUInt32" -"toUInt32OrDefault" -"toUInt32OrNull" -"toUInt32OrZero" -"toUInt64" -"toUInt64OrDefault" -"toUInt64OrNull" -"toUInt64OrZero" -"toUInt8" -"toUInt8OrDefault" -"toUInt8OrNull" -"toUInt8OrZero" -"toUTCTimestamp" -"toUUID" -"toUUIDOrDefault" -"toUUIDOrNull" -"toUUIDOrZero" -"toUnixTimestamp" -"toUnixTimestamp64Micro" -"toUnixTimestamp64Milli" -"toUnixTimestamp64Nano" -"toValidUTF8" -"toWeek" -"toYYYYMM" -"toYYYYMMDD" -"toYYYYMMDDhhmmss" -"toYear" -"toYearWeek" -"to_utc_timestamp" -"today" -"tokens" -"topK" -"topKArgMax" -"topKArgMin" -"topKArray" -"topKDistinct" -"topKForEach" -"topKIf" -"topKMap" -"topKMerge" -"topKNull" -"topKOrDefault" -"topKOrNull" -"topKResample" -"topKSimpleState" -"topKState" -"topKWeighted" -"topKWeightedArgMax" -"topKWeightedArgMin" -"topKWeightedArray" -"topKWeightedDistinct" -"topKWeightedForEach" -"topKWeightedIf" -"topKWeightedMap" -"topKWeightedMerge" -"topKWeightedNull" -"topKWeightedOrDefault" -"topKWeightedOrNull" -"topKWeightedResample" -"topKWeightedSimpleState" -"topKWeightedState" -"topLevelDomain" -"topLevelDomainRFC" -"transactionID" -"transactionLatestSnapshot" -"transactionOldestSnapshot" -"transform" -"translate" -"translateUTF8" -"trim" -"trimBoth" -"trimLeft" -"trimRight" -"trunc" -"truncate" -"tryBase58Decode" -"tryBase64Decode" -"tryBase64URLDecode" -"tryDecrypt" -"tryIdnaEncode" -"tryPunycodeDecode" -"tumble" -"tumbleEnd" -"tumbleStart" -"tuple" -"tupleConcat" -"tupleDivide" -"tupleDivideByNumber" -"tupleElement" -"tupleHammingDistance" -"tupleIntDiv" -"tupleIntDivByNumber" -"tupleIntDivOrZero" -"tupleIntDivOrZeroByNumber" -"tupleMinus" -"tupleModulo" -"tupleModuloByNumber" -"tupleMultiply" -"tupleMultiplyByNumber" -"tupleNames" -"tupleNegate" -"tuplePlus" -"tupleToNameValuePairs" -"ucase" -"unbin" -"unhex" -"uniq" -"uniqArgMax" -"uniqArgMin" -"uniqArray" -"uniqCombined" -"uniqCombined64" -"uniqCombined64ArgMax" -"uniqCombined64ArgMin" -"uniqCombined64Array" -"uniqCombined64Distinct" -"uniqCombined64ForEach" -"uniqCombined64If" -"uniqCombined64Map" -"uniqCombined64Merge" -"uniqCombined64Null" -"uniqCombined64OrDefault" -"uniqCombined64OrNull" -"uniqCombined64Resample" -"uniqCombined64SimpleState" -"uniqCombined64State" -"uniqCombinedArgMax" -"uniqCombinedArgMin" -"uniqCombinedArray" -"uniqCombinedDistinct" -"uniqCombinedForEach" -"uniqCombinedIf" -"uniqCombinedMap" -"uniqCombinedMerge" -"uniqCombinedNull" -"uniqCombinedOrDefault" -"uniqCombinedOrNull" -"uniqCombinedResample" -"uniqCombinedSimpleState" -"uniqCombinedState" -"uniqDistinct" -"uniqExact" -"uniqExactArgMax" -"uniqExactArgMin" -"uniqExactArray" -"uniqExactDistinct" -"uniqExactForEach" -"uniqExactIf" -"uniqExactMap" -"uniqExactMerge" -"uniqExactNull" -"uniqExactOrDefault" -"uniqExactOrNull" -"uniqExactResample" -"uniqExactSimpleState" -"uniqExactState" -"uniqForEach" -"uniqHLL12" -"uniqHLL12ArgMax" -"uniqHLL12ArgMin" -"uniqHLL12Array" -"uniqHLL12Distinct" -"uniqHLL12ForEach" -"uniqHLL12If" -"uniqHLL12Map" -"uniqHLL12Merge" -"uniqHLL12Null" -"uniqHLL12OrDefault" -"uniqHLL12OrNull" -"uniqHLL12Resample" -"uniqHLL12SimpleState" -"uniqHLL12State" -"uniqIf" -"uniqMap" -"uniqMerge" -"uniqNull" -"uniqOrDefault" -"uniqOrNull" -"uniqResample" -"uniqSimpleState" -"uniqState" -"uniqTheta" -"uniqThetaArgMax" -"uniqThetaArgMin" -"uniqThetaArray" -"uniqThetaDistinct" -"uniqThetaForEach" -"uniqThetaIf" -"uniqThetaIntersect" -"uniqThetaMap" -"uniqThetaMerge" -"uniqThetaNot" -"uniqThetaNull" -"uniqThetaOrDefault" -"uniqThetaOrNull" -"uniqThetaResample" -"uniqThetaSimpleState" -"uniqThetaState" -"uniqThetaUnion" -"uniqUpTo" -"uniqUpToArgMax" -"uniqUpToArgMin" -"uniqUpToArray" -"uniqUpToDistinct" -"uniqUpToForEach" -"uniqUpToIf" -"uniqUpToMap" -"uniqUpToMerge" -"uniqUpToNull" -"uniqUpToOrDefault" -"uniqUpToOrNull" -"uniqUpToResample" -"uniqUpToSimpleState" -"uniqUpToState" -"upper" -"upperUTF8" -"uptime" -"user" -"validateNestedArraySizes" -"varPop" -"varPopArgMax" -"varPopArgMin" -"varPopArray" -"varPopDistinct" -"varPopForEach" -"varPopIf" -"varPopMap" -"varPopMerge" -"varPopNull" -"varPopOrDefault" -"varPopOrNull" -"varPopResample" -"varPopSimpleState" -"varPopStable" -"varPopStableArgMax" -"varPopStableArgMin" -"varPopStableArray" -"varPopStableDistinct" -"varPopStableForEach" -"varPopStableIf" -"varPopStableMap" -"varPopStableMerge" -"varPopStableNull" -"varPopStableOrDefault" -"varPopStableOrNull" -"varPopStableResample" -"varPopStableSimpleState" -"varPopStableState" -"varPopState" -"varSamp" -"varSampArgMax" -"varSampArgMin" -"varSampArray" -"varSampDistinct" -"varSampForEach" -"varSampIf" -"varSampMap" -"varSampMerge" -"varSampNull" -"varSampOrDefault" -"varSampOrNull" -"varSampResample" -"varSampSimpleState" -"varSampStable" -"varSampStableArgMax" -"varSampStableArgMin" -"varSampStableArray" -"varSampStableDistinct" -"varSampStableForEach" -"varSampStableIf" -"varSampStableMap" -"varSampStableMerge" -"varSampStableNull" -"varSampStableOrDefault" -"varSampStableOrNull" -"varSampStableResample" -"varSampStableSimpleState" -"varSampStableState" -"varSampState" -"variantElement" -"variantType" -"vectorDifference" -"vectorSum" -"version" -"visibleWidth" -"visitParamExtractBool" -"visitParamExtractFloat" -"visitParamExtractInt" -"visitParamExtractRaw" -"visitParamExtractString" -"visitParamExtractUInt" -"visitParamHas" -"week" -"welchTTest" -"welchTTestArgMax" -"welchTTestArgMin" -"welchTTestArray" -"welchTTestDistinct" -"welchTTestForEach" -"welchTTestIf" -"welchTTestMap" -"welchTTestMerge" -"welchTTestNull" -"welchTTestOrDefault" -"welchTTestOrNull" -"welchTTestResample" -"welchTTestSimpleState" -"welchTTestState" -"widthBucket" -"width_bucket" -"windowFunnel" -"windowFunnelArgMax" -"windowFunnelArgMin" -"windowFunnelArray" -"windowFunnelDistinct" -"windowFunnelForEach" -"windowFunnelIf" -"windowFunnelMap" -"windowFunnelMerge" -"windowFunnelNull" -"windowFunnelOrDefault" -"windowFunnelOrNull" -"windowFunnelResample" -"windowFunnelSimpleState" -"windowFunnelState" -"windowID" -"wkt" -"wordShingleMinHash" -"wordShingleMinHashArg" -"wordShingleMinHashArgCaseInsensitive" -"wordShingleMinHashArgCaseInsensitiveUTF8" -"wordShingleMinHashArgUTF8" -"wordShingleMinHashCaseInsensitive" -"wordShingleMinHashCaseInsensitiveUTF8" -"wordShingleMinHashUTF8" -"wordShingleSimHash" -"wordShingleSimHashCaseInsensitive" -"wordShingleSimHashCaseInsensitiveUTF8" -"wordShingleSimHashUTF8" -"wyHash64" -"xor" -"xxHash32" -"xxHash64" -"xxh3" -"yandexConsistentHash" -"yearweek" -"yesterday" -"zookeeperSessionUptime" +"bool" +"boolean" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index e562595fb67..e69de29bb2d 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -1,4283 +0,0 @@ -"BIT_AND" -"BIT_ANDArgMax" -"BIT_ANDArgMin" -"BIT_ANDArray" -"BIT_ANDDistinct" -"BIT_ANDForEach" -"BIT_ANDIf" -"BIT_ANDMap" -"BIT_ANDMerge" -"BIT_ANDNull" -"BIT_ANDOrDefault" -"BIT_ANDOrNull" -"BIT_ANDResample" -"BIT_ANDSimpleState" -"BIT_ANDState" -"BIT_OR" -"BIT_ORArgMax" -"BIT_ORArgMin" -"BIT_ORArray" -"BIT_ORDistinct" -"BIT_ORForEach" -"BIT_ORIf" -"BIT_ORMap" -"BIT_ORMerge" -"BIT_ORNull" -"BIT_OROrDefault" -"BIT_OROrNull" -"BIT_ORResample" -"BIT_ORSimpleState" -"BIT_ORState" -"BIT_XOR" -"BIT_XORArgMax" -"BIT_XORArgMin" -"BIT_XORArray" -"BIT_XORDistinct" -"BIT_XORForEach" -"BIT_XORIf" -"BIT_XORMap" -"BIT_XORMerge" -"BIT_XORNull" -"BIT_XOROrDefault" -"BIT_XOROrNull" -"BIT_XORResample" -"BIT_XORSimpleState" -"BIT_XORState" -"BLAKE3" -"CAST" -"CHARACTER_LENGTH" -"CHAR_LENGTH" -"COVAR_POP" -"COVAR_POPArgMax" -"COVAR_POPArgMin" -"COVAR_POPArray" -"COVAR_POPDistinct" -"COVAR_POPForEach" -"COVAR_POPIf" -"COVAR_POPMap" -"COVAR_POPMerge" -"COVAR_POPNull" -"COVAR_POPOrDefault" -"COVAR_POPOrNull" -"COVAR_POPResample" -"COVAR_POPSimpleState" -"COVAR_POPState" -"COVAR_SAMP" -"COVAR_SAMPArgMax" -"COVAR_SAMPArgMin" -"COVAR_SAMPArray" -"COVAR_SAMPDistinct" -"COVAR_SAMPForEach" -"COVAR_SAMPIf" -"COVAR_SAMPMap" -"COVAR_SAMPMerge" -"COVAR_SAMPNull" -"COVAR_SAMPOrDefault" -"COVAR_SAMPOrNull" -"COVAR_SAMPResample" -"COVAR_SAMPSimpleState" -"COVAR_SAMPState" -"CRC32" -"CRC32IEEE" -"CRC64" -"DATABASE" -"DATE" -"DATE_DIFF" -"DATE_FORMAT" -"DATE_TRUNC" -"DAY" -"DAYOFMONTH" -"DAYOFWEEK" -"DAYOFYEAR" -"FORMAT_BYTES" -"FQDN" -"FROM_BASE64" -"FROM_DAYS" -"FROM_UNIXTIME" -"HOUR" -"INET6_ATON" -"INET6_NTOA" -"INET_ATON" -"INET_NTOA" -"IPv4CIDRToRange" -"IPv4NumToString" -"IPv4NumToStringClassC" -"IPv4StringToNum" -"IPv4StringToNumOrDefault" -"IPv4StringToNumOrNull" -"IPv4ToIPv6" -"IPv6CIDRToRange" -"IPv6NumToString" -"IPv6StringToNum" -"IPv6StringToNumOrDefault" -"IPv6StringToNumOrNull" -"JSONArrayLength" -"JSONExtract" -"JSONExtractArrayRaw" -"JSONExtractBool" -"JSONExtractFloat" -"JSONExtractInt" -"JSONExtractKeys" -"JSONExtractKeysAndValues" -"JSONExtractKeysAndValuesRaw" -"JSONExtractRaw" -"JSONExtractString" -"JSONExtractUInt" -"JSONHas" -"JSONKey" -"JSONLength" -"JSONMergePatch" -"JSONType" -"JSON_ARRAY_LENGTH" -"JSON_EXISTS" -"JSON_QUERY" -"JSON_VALUE" -"L1Distance" -"L1Norm" -"L1Normalize" -"L2Distance" -"L2Norm" -"L2Normalize" -"L2SquaredDistance" -"L2SquaredNorm" -"LAST_DAY" -"LinfDistance" -"LinfNorm" -"LinfNormalize" -"LpDistance" -"LpNorm" -"LpNormalize" -"MACNumToString" -"MACStringToNum" -"MACStringToOUI" -"MAP_FROM_ARRAYS" -"MD4" -"MD5" -"MILLISECOND" -"MINUTE" -"MONTH" -"OCTET_LENGTH" -"QUARTER" -"REGEXP_EXTRACT" -"REGEXP_MATCHES" -"REGEXP_REPLACE" -"SCHEMA" -"SECOND" -"SHA1" -"SHA224" -"SHA256" -"SHA384" -"SHA512" -"SHA512_256" -"STD" -"STDArgMax" -"STDArgMin" -"STDArray" -"STDDEV_POP" -"STDDEV_POPArgMax" -"STDDEV_POPArgMin" -"STDDEV_POPArray" -"STDDEV_POPDistinct" -"STDDEV_POPForEach" -"STDDEV_POPIf" -"STDDEV_POPMap" -"STDDEV_POPMerge" -"STDDEV_POPNull" -"STDDEV_POPOrDefault" -"STDDEV_POPOrNull" -"STDDEV_POPResample" -"STDDEV_POPSimpleState" -"STDDEV_POPState" -"STDDEV_SAMP" -"STDDEV_SAMPArgMax" -"STDDEV_SAMPArgMin" -"STDDEV_SAMPArray" -"STDDEV_SAMPDistinct" -"STDDEV_SAMPForEach" -"STDDEV_SAMPIf" -"STDDEV_SAMPMap" -"STDDEV_SAMPMerge" -"STDDEV_SAMPNull" -"STDDEV_SAMPOrDefault" -"STDDEV_SAMPOrNull" -"STDDEV_SAMPResample" -"STDDEV_SAMPSimpleState" -"STDDEV_SAMPState" -"STDDistinct" -"STDForEach" -"STDIf" -"STDMap" -"STDMerge" -"STDNull" -"STDOrDefault" -"STDOrNull" -"STDResample" -"STDSimpleState" -"STDState" -"SUBSTRING_INDEX" -"SVG" -"TIMESTAMP_DIFF" -"TO_BASE64" -"TO_DAYS" -"TO_UNIXTIME" -"ULIDStringToDateTime" -"URLHash" -"URLHierarchy" -"URLPathHierarchy" -"UTCTimestamp" -"UTC_timestamp" -"UUIDNumToString" -"UUIDStringToNum" -"UUIDToNum" -"UUIDv7ToDateTime" -"VAR_POP" -"VAR_POPArgMax" -"VAR_POPArgMin" -"VAR_POPArray" -"VAR_POPDistinct" -"VAR_POPForEach" -"VAR_POPIf" -"VAR_POPMap" -"VAR_POPMerge" -"VAR_POPNull" -"VAR_POPOrDefault" -"VAR_POPOrNull" -"VAR_POPResample" -"VAR_POPSimpleState" -"VAR_POPState" -"VAR_SAMP" -"VAR_SAMPArgMax" -"VAR_SAMPArgMin" -"VAR_SAMPArray" -"VAR_SAMPDistinct" -"VAR_SAMPForEach" -"VAR_SAMPIf" -"VAR_SAMPMap" -"VAR_SAMPMerge" -"VAR_SAMPNull" -"VAR_SAMPOrDefault" -"VAR_SAMPOrNull" -"VAR_SAMPResample" -"VAR_SAMPSimpleState" -"VAR_SAMPState" -"YEAR" -"YYYYMMDDToDate" -"YYYYMMDDToDate32" -"YYYYMMDDhhmmssToDateTime" -"YYYYMMDDhhmmssToDateTime64" -"_CAST" -"__actionName" -"__bitBoolMaskAnd" -"__bitBoolMaskOr" -"__bitSwapLastTwo" -"__bitWrapperFunc" -"__getScalar" -"__scalarSubqueryResult" -"abs" -"accurateCast" -"accurateCastOrDefault" -"accurateCastOrNull" -"acos" -"acosh" -"addDate" -"addDays" -"addHours" -"addInterval" -"addMicroseconds" -"addMilliseconds" -"addMinutes" -"addMonths" -"addNanoseconds" -"addQuarters" -"addSeconds" -"addTupleOfIntervals" -"addWeeks" -"addYears" -"addressToLine" -"addressToLineWithInlines" -"addressToSymbol" -"aes_decrypt_mysql" -"aes_encrypt_mysql" -"age" -"aggThrow" -"aggThrowArgMax" -"aggThrowArgMin" -"aggThrowArray" -"aggThrowDistinct" -"aggThrowForEach" -"aggThrowIf" -"aggThrowMap" -"aggThrowMerge" -"aggThrowNull" -"aggThrowOrDefault" -"aggThrowOrNull" -"aggThrowResample" -"aggThrowSimpleState" -"aggThrowState" -"alphaTokens" -"analysisOfVariance" -"analysisOfVarianceArgMax" -"analysisOfVarianceArgMin" -"analysisOfVarianceArray" -"analysisOfVarianceDistinct" -"analysisOfVarianceForEach" -"analysisOfVarianceIf" -"analysisOfVarianceMap" -"analysisOfVarianceMerge" -"analysisOfVarianceNull" -"analysisOfVarianceOrDefault" -"analysisOfVarianceOrNull" -"analysisOfVarianceResample" -"analysisOfVarianceSimpleState" -"analysisOfVarianceState" -"and" -"anova" -"anovaArgMax" -"anovaArgMin" -"anovaArray" -"anovaDistinct" -"anovaForEach" -"anovaIf" -"anovaMap" -"anovaMerge" -"anovaNull" -"anovaOrDefault" -"anovaOrNull" -"anovaResample" -"anovaSimpleState" -"anovaState" -"any" -"anyArgMax" -"anyArgMin" -"anyArray" -"anyDistinct" -"anyForEach" -"anyHeavy" -"anyHeavyArgMax" -"anyHeavyArgMin" -"anyHeavyArray" -"anyHeavyDistinct" -"anyHeavyForEach" -"anyHeavyIf" -"anyHeavyMap" -"anyHeavyMerge" -"anyHeavyNull" -"anyHeavyOrDefault" -"anyHeavyOrNull" -"anyHeavyResample" -"anyHeavySimpleState" -"anyHeavyState" -"anyIf" -"anyLast" -"anyLastArgMax" -"anyLastArgMin" -"anyLastArray" -"anyLastDistinct" -"anyLastForEach" -"anyLastIf" -"anyLastMap" -"anyLastMerge" -"anyLastNull" -"anyLastOrDefault" -"anyLastOrNull" -"anyLastResample" -"anyLastSimpleState" -"anyLastState" -"anyLast_respect_nulls" -"anyLast_respect_nullsArgMax" -"anyLast_respect_nullsArgMin" -"anyLast_respect_nullsArray" -"anyLast_respect_nullsDistinct" -"anyLast_respect_nullsForEach" -"anyLast_respect_nullsIf" -"anyLast_respect_nullsMap" -"anyLast_respect_nullsMerge" -"anyLast_respect_nullsNull" -"anyLast_respect_nullsOrDefault" -"anyLast_respect_nullsOrNull" -"anyLast_respect_nullsResample" -"anyLast_respect_nullsSimpleState" -"anyLast_respect_nullsState" -"anyMap" -"anyMerge" -"anyNull" -"anyOrDefault" -"anyOrNull" -"anyResample" -"anySimpleState" -"anyState" -"any_respect_nulls" -"any_respect_nullsArgMax" -"any_respect_nullsArgMin" -"any_respect_nullsArray" -"any_respect_nullsDistinct" -"any_respect_nullsForEach" -"any_respect_nullsIf" -"any_respect_nullsMap" -"any_respect_nullsMerge" -"any_respect_nullsNull" -"any_respect_nullsOrDefault" -"any_respect_nullsOrNull" -"any_respect_nullsResample" -"any_respect_nullsSimpleState" -"any_respect_nullsState" -"any_value" -"any_valueArgMax" -"any_valueArgMin" -"any_valueArray" -"any_valueDistinct" -"any_valueForEach" -"any_valueIf" -"any_valueMap" -"any_valueMerge" -"any_valueNull" -"any_valueOrDefault" -"any_valueOrNull" -"any_valueResample" -"any_valueSimpleState" -"any_valueState" -"any_value_respect_nulls" -"any_value_respect_nullsArgMax" -"any_value_respect_nullsArgMin" -"any_value_respect_nullsArray" -"any_value_respect_nullsDistinct" -"any_value_respect_nullsForEach" -"any_value_respect_nullsIf" -"any_value_respect_nullsMap" -"any_value_respect_nullsMerge" -"any_value_respect_nullsNull" -"any_value_respect_nullsOrDefault" -"any_value_respect_nullsOrNull" -"any_value_respect_nullsResample" -"any_value_respect_nullsSimpleState" -"any_value_respect_nullsState" -"appendTrailingCharIfAbsent" -"approx_top_count" -"approx_top_countArgMax" -"approx_top_countArgMin" -"approx_top_countArray" -"approx_top_countDistinct" -"approx_top_countForEach" -"approx_top_countIf" -"approx_top_countMap" -"approx_top_countMerge" -"approx_top_countNull" -"approx_top_countOrDefault" -"approx_top_countOrNull" -"approx_top_countResample" -"approx_top_countSimpleState" -"approx_top_countState" -"approx_top_k" -"approx_top_kArgMax" -"approx_top_kArgMin" -"approx_top_kArray" -"approx_top_kDistinct" -"approx_top_kForEach" -"approx_top_kIf" -"approx_top_kMap" -"approx_top_kMerge" -"approx_top_kNull" -"approx_top_kOrDefault" -"approx_top_kOrNull" -"approx_top_kResample" -"approx_top_kSimpleState" -"approx_top_kState" -"approx_top_sum" -"approx_top_sumArgMax" -"approx_top_sumArgMin" -"approx_top_sumArray" -"approx_top_sumDistinct" -"approx_top_sumForEach" -"approx_top_sumIf" -"approx_top_sumMap" -"approx_top_sumMerge" -"approx_top_sumNull" -"approx_top_sumOrDefault" -"approx_top_sumOrNull" -"approx_top_sumResample" -"approx_top_sumSimpleState" -"approx_top_sumState" -"argMax" -"argMaxArgMax" -"argMaxArgMin" -"argMaxArray" -"argMaxDistinct" -"argMaxForEach" -"argMaxIf" -"argMaxMap" -"argMaxMerge" -"argMaxNull" -"argMaxOrDefault" -"argMaxOrNull" -"argMaxResample" -"argMaxSimpleState" -"argMaxState" -"argMin" -"argMinArgMax" -"argMinArgMin" -"argMinArray" -"argMinDistinct" -"argMinForEach" -"argMinIf" -"argMinMap" -"argMinMerge" -"argMinNull" -"argMinOrDefault" -"argMinOrNull" -"argMinResample" -"argMinSimpleState" -"argMinState" -"array" -"arrayAUC" -"arrayAll" -"arrayAvg" -"arrayCompact" -"arrayConcat" -"arrayCount" -"arrayCumSum" -"arrayCumSumNonNegative" -"arrayDifference" -"arrayDistinct" -"arrayDotProduct" -"arrayElement" -"arrayEnumerate" -"arrayEnumerateDense" -"arrayEnumerateDenseRanked" -"arrayEnumerateUniq" -"arrayEnumerateUniqRanked" -"arrayExists" -"arrayFill" -"arrayFilter" -"arrayFirst" -"arrayFirstIndex" -"arrayFirstOrNull" -"arrayFlatten" -"arrayFold" -"arrayIntersect" -"arrayJaccardIndex" -"arrayJoin" -"arrayLast" -"arrayLastIndex" -"arrayLastOrNull" -"arrayMap" -"arrayMax" -"arrayMin" -"arrayPartialReverseSort" -"arrayPartialShuffle" -"arrayPartialSort" -"arrayPopBack" -"arrayPopFront" -"arrayProduct" -"arrayPushBack" -"arrayPushFront" -"arrayRandomSample" -"arrayReduce" -"arrayReduceInRanges" -"arrayResize" -"arrayReverse" -"arrayReverseFill" -"arrayReverseSort" -"arrayReverseSplit" -"arrayRotateLeft" -"arrayRotateRight" -"arrayShiftLeft" -"arrayShiftRight" -"arrayShingles" -"arrayShuffle" -"arraySlice" -"arraySort" -"arraySplit" -"arrayStringConcat" -"arraySum" -"arrayUniq" -"arrayWithConstant" -"arrayZip" -"array_agg" -"array_aggArgMax" -"array_aggArgMin" -"array_aggArray" -"array_aggDistinct" -"array_aggForEach" -"array_aggIf" -"array_aggMap" -"array_aggMerge" -"array_aggNull" -"array_aggOrDefault" -"array_aggOrNull" -"array_aggResample" -"array_aggSimpleState" -"array_aggState" -"array_concat_agg" -"array_concat_aggArgMax" -"array_concat_aggArgMin" -"array_concat_aggArray" -"array_concat_aggDistinct" -"array_concat_aggForEach" -"array_concat_aggIf" -"array_concat_aggMap" -"array_concat_aggMerge" -"array_concat_aggNull" -"array_concat_aggOrDefault" -"array_concat_aggOrNull" -"array_concat_aggResample" -"array_concat_aggSimpleState" -"array_concat_aggState" -"ascii" -"asin" -"asinh" -"assumeNotNull" -"atan" -"atan2" -"atanh" -"avg" -"avgArgMax" -"avgArgMin" -"avgArray" -"avgDistinct" -"avgForEach" -"avgIf" -"avgMap" -"avgMerge" -"avgNull" -"avgOrDefault" -"avgOrNull" -"avgResample" -"avgSimpleState" -"avgState" -"avgWeighted" -"avgWeightedArgMax" -"avgWeightedArgMin" -"avgWeightedArray" -"avgWeightedDistinct" -"avgWeightedForEach" -"avgWeightedIf" -"avgWeightedMap" -"avgWeightedMerge" -"avgWeightedNull" -"avgWeightedOrDefault" -"avgWeightedOrNull" -"avgWeightedResample" -"avgWeightedSimpleState" -"avgWeightedState" -"bar" -"base58Decode" -"base58Encode" -"base64Decode" -"base64Encode" -"base64URLDecode" -"base64URLEncode" -"basename" -"bin" -"bitAnd" -"bitCount" -"bitHammingDistance" -"bitNot" -"bitOr" -"bitPositionsToArray" -"bitRotateLeft" -"bitRotateRight" -"bitShiftLeft" -"bitShiftRight" -"bitSlice" -"bitTest" -"bitTestAll" -"bitTestAny" -"bitXor" -"bitmapAnd" -"bitmapAndCardinality" -"bitmapAndnot" -"bitmapAndnotCardinality" -"bitmapBuild" -"bitmapCardinality" -"bitmapContains" -"bitmapHasAll" -"bitmapHasAny" -"bitmapMax" -"bitmapMin" -"bitmapOr" -"bitmapOrCardinality" -"bitmapSubsetInRange" -"bitmapSubsetLimit" -"bitmapToArray" -"bitmapTransform" -"bitmapXor" -"bitmapXorCardinality" -"bitmaskToArray" -"bitmaskToList" -"blockNumber" -"blockSerializedSize" -"blockSize" -"boundingRatio" -"boundingRatioArgMax" -"boundingRatioArgMin" -"boundingRatioArray" -"boundingRatioDistinct" -"boundingRatioForEach" -"boundingRatioIf" -"boundingRatioMap" -"boundingRatioMerge" -"boundingRatioNull" -"boundingRatioOrDefault" -"boundingRatioOrNull" -"boundingRatioResample" -"boundingRatioSimpleState" -"boundingRatioState" -"buildId" -"byteHammingDistance" -"byteSize" -"byteSlice" -"byteSwap" -"caseWithExpr" -"caseWithExpression" -"caseWithoutExpr" -"caseWithoutExpression" -"catboostEvaluate" -"categoricalInformationValue" -"categoricalInformationValueArgMax" -"categoricalInformationValueArgMin" -"categoricalInformationValueArray" -"categoricalInformationValueDistinct" -"categoricalInformationValueForEach" -"categoricalInformationValueIf" -"categoricalInformationValueMap" -"categoricalInformationValueMerge" -"categoricalInformationValueNull" -"categoricalInformationValueOrDefault" -"categoricalInformationValueOrNull" -"categoricalInformationValueResample" -"categoricalInformationValueSimpleState" -"categoricalInformationValueState" -"cbrt" -"ceil" -"ceiling" -"changeDay" -"changeHour" -"changeMinute" -"changeMonth" -"changeSecond" -"changeYear" -"char" -"cityHash64" -"clamp" -"coalesce" -"concat" -"concatAssumeInjective" -"concatWithSeparator" -"concatWithSeparatorAssumeInjective" -"concat_ws" -"connectionId" -"connection_id" -"contingency" -"contingencyArgMax" -"contingencyArgMin" -"contingencyArray" -"contingencyDistinct" -"contingencyForEach" -"contingencyIf" -"contingencyMap" -"contingencyMerge" -"contingencyNull" -"contingencyOrDefault" -"contingencyOrNull" -"contingencyResample" -"contingencySimpleState" -"contingencyState" -"convertCharset" -"corr" -"corrArgMax" -"corrArgMin" -"corrArray" -"corrDistinct" -"corrForEach" -"corrIf" -"corrMap" -"corrMatrix" -"corrMatrixArgMax" -"corrMatrixArgMin" -"corrMatrixArray" -"corrMatrixDistinct" -"corrMatrixForEach" -"corrMatrixIf" -"corrMatrixMap" -"corrMatrixMerge" -"corrMatrixNull" -"corrMatrixOrDefault" -"corrMatrixOrNull" -"corrMatrixResample" -"corrMatrixSimpleState" -"corrMatrixState" -"corrMerge" -"corrNull" -"corrOrDefault" -"corrOrNull" -"corrResample" -"corrSimpleState" -"corrStable" -"corrStableArgMax" -"corrStableArgMin" -"corrStableArray" -"corrStableDistinct" -"corrStableForEach" -"corrStableIf" -"corrStableMap" -"corrStableMerge" -"corrStableNull" -"corrStableOrDefault" -"corrStableOrNull" -"corrStableResample" -"corrStableSimpleState" -"corrStableState" -"corrState" -"cos" -"cosh" -"cosineDistance" -"count" -"countArgMax" -"countArgMin" -"countArray" -"countDigits" -"countDistinct" -"countEqual" -"countForEach" -"countIf" -"countMap" -"countMatches" -"countMatchesCaseInsensitive" -"countMerge" -"countNull" -"countOrDefault" -"countOrNull" -"countResample" -"countSimpleState" -"countState" -"countSubstrings" -"countSubstringsCaseInsensitive" -"countSubstringsCaseInsensitiveUTF8" -"covarPop" -"covarPopArgMax" -"covarPopArgMin" -"covarPopArray" -"covarPopDistinct" -"covarPopForEach" -"covarPopIf" -"covarPopMap" -"covarPopMatrix" -"covarPopMatrixArgMax" -"covarPopMatrixArgMin" -"covarPopMatrixArray" -"covarPopMatrixDistinct" -"covarPopMatrixForEach" -"covarPopMatrixIf" -"covarPopMatrixMap" -"covarPopMatrixMerge" -"covarPopMatrixNull" -"covarPopMatrixOrDefault" -"covarPopMatrixOrNull" -"covarPopMatrixResample" -"covarPopMatrixSimpleState" -"covarPopMatrixState" -"covarPopMerge" -"covarPopNull" -"covarPopOrDefault" -"covarPopOrNull" -"covarPopResample" -"covarPopSimpleState" -"covarPopStable" -"covarPopStableArgMax" -"covarPopStableArgMin" -"covarPopStableArray" -"covarPopStableDistinct" -"covarPopStableForEach" -"covarPopStableIf" -"covarPopStableMap" -"covarPopStableMerge" -"covarPopStableNull" -"covarPopStableOrDefault" -"covarPopStableOrNull" -"covarPopStableResample" -"covarPopStableSimpleState" -"covarPopStableState" -"covarPopState" -"covarSamp" -"covarSampArgMax" -"covarSampArgMin" -"covarSampArray" -"covarSampDistinct" -"covarSampForEach" -"covarSampIf" -"covarSampMap" -"covarSampMatrix" -"covarSampMatrixArgMax" -"covarSampMatrixArgMin" -"covarSampMatrixArray" -"covarSampMatrixDistinct" -"covarSampMatrixForEach" -"covarSampMatrixIf" -"covarSampMatrixMap" -"covarSampMatrixMerge" -"covarSampMatrixNull" -"covarSampMatrixOrDefault" -"covarSampMatrixOrNull" -"covarSampMatrixResample" -"covarSampMatrixSimpleState" -"covarSampMatrixState" -"covarSampMerge" -"covarSampNull" -"covarSampOrDefault" -"covarSampOrNull" -"covarSampResample" -"covarSampSimpleState" -"covarSampStable" -"covarSampStableArgMax" -"covarSampStableArgMin" -"covarSampStableArray" -"covarSampStableDistinct" -"covarSampStableForEach" -"covarSampStableIf" -"covarSampStableMap" -"covarSampStableMerge" -"covarSampStableNull" -"covarSampStableOrDefault" -"covarSampStableOrNull" -"covarSampStableResample" -"covarSampStableSimpleState" -"covarSampStableState" -"covarSampState" -"cramersV" -"cramersVArgMax" -"cramersVArgMin" -"cramersVArray" -"cramersVBiasCorrected" -"cramersVBiasCorrectedArgMax" -"cramersVBiasCorrectedArgMin" -"cramersVBiasCorrectedArray" -"cramersVBiasCorrectedDistinct" -"cramersVBiasCorrectedForEach" -"cramersVBiasCorrectedIf" -"cramersVBiasCorrectedMap" -"cramersVBiasCorrectedMerge" -"cramersVBiasCorrectedNull" -"cramersVBiasCorrectedOrDefault" -"cramersVBiasCorrectedOrNull" -"cramersVBiasCorrectedResample" -"cramersVBiasCorrectedSimpleState" -"cramersVBiasCorrectedState" -"cramersVDistinct" -"cramersVForEach" -"cramersVIf" -"cramersVMap" -"cramersVMerge" -"cramersVNull" -"cramersVOrDefault" -"cramersVOrNull" -"cramersVResample" -"cramersVSimpleState" -"cramersVState" -"curdate" -"currentDatabase" -"currentProfiles" -"currentRoles" -"currentSchemas" -"currentUser" -"current_database" -"current_date" -"current_schemas" -"current_timestamp" -"current_user" -"cutFragment" -"cutIPv6" -"cutQueryString" -"cutQueryStringAndFragment" -"cutToFirstSignificantSubdomain" -"cutToFirstSignificantSubdomainCustom" -"cutToFirstSignificantSubdomainCustomRFC" -"cutToFirstSignificantSubdomainCustomWithWWW" -"cutToFirstSignificantSubdomainCustomWithWWWRFC" -"cutToFirstSignificantSubdomainRFC" -"cutToFirstSignificantSubdomainWithWWW" -"cutToFirstSignificantSubdomainWithWWWRFC" -"cutURLParameter" -"cutWWW" -"damerauLevenshteinDistance" -"dateDiff" -"dateName" -"dateTime64ToSnowflake" -"dateTime64ToSnowflakeID" -"dateTimeToSnowflake" -"dateTimeToSnowflakeID" -"dateTrunc" -"date_diff" -"decodeHTMLComponent" -"decodeURLComponent" -"decodeURLFormComponent" -"decodeXMLComponent" -"decrypt" -"defaultProfiles" -"defaultRoles" -"defaultValueOfArgumentType" -"defaultValueOfTypeName" -"degrees" -"deltaSum" -"deltaSumArgMax" -"deltaSumArgMin" -"deltaSumArray" -"deltaSumDistinct" -"deltaSumForEach" -"deltaSumIf" -"deltaSumMap" -"deltaSumMerge" -"deltaSumNull" -"deltaSumOrDefault" -"deltaSumOrNull" -"deltaSumResample" -"deltaSumSimpleState" -"deltaSumState" -"deltaSumTimestamp" -"deltaSumTimestampArgMax" -"deltaSumTimestampArgMin" -"deltaSumTimestampArray" -"deltaSumTimestampDistinct" -"deltaSumTimestampForEach" -"deltaSumTimestampIf" -"deltaSumTimestampMap" -"deltaSumTimestampMerge" -"deltaSumTimestampNull" -"deltaSumTimestampOrDefault" -"deltaSumTimestampOrNull" -"deltaSumTimestampResample" -"deltaSumTimestampSimpleState" -"deltaSumTimestampState" -"demangle" -"denseRank" -"denseRankArgMax" -"denseRankArgMin" -"denseRankArray" -"denseRankDistinct" -"denseRankForEach" -"denseRankIf" -"denseRankMap" -"denseRankMerge" -"denseRankNull" -"denseRankOrDefault" -"denseRankOrNull" -"denseRankResample" -"denseRankSimpleState" -"denseRankState" -"dense_rank" -"dense_rankArgMax" -"dense_rankArgMin" -"dense_rankArray" -"dense_rankDistinct" -"dense_rankForEach" -"dense_rankIf" -"dense_rankMap" -"dense_rankMerge" -"dense_rankNull" -"dense_rankOrDefault" -"dense_rankOrNull" -"dense_rankResample" -"dense_rankSimpleState" -"dense_rankState" -"detectCharset" -"detectLanguage" -"detectLanguageMixed" -"detectLanguageUnknown" -"detectProgrammingLanguage" -"detectTonality" -"dictGet" -"dictGetAll" -"dictGetChildren" -"dictGetDate" -"dictGetDateOrDefault" -"dictGetDateTime" -"dictGetDateTimeOrDefault" -"dictGetDescendants" -"dictGetFloat32" -"dictGetFloat32OrDefault" -"dictGetFloat64" -"dictGetFloat64OrDefault" -"dictGetHierarchy" -"dictGetIPv4" -"dictGetIPv4OrDefault" -"dictGetIPv6" -"dictGetIPv6OrDefault" -"dictGetInt16" -"dictGetInt16OrDefault" -"dictGetInt32" -"dictGetInt32OrDefault" -"dictGetInt64" -"dictGetInt64OrDefault" -"dictGetInt8" -"dictGetInt8OrDefault" -"dictGetOrDefault" -"dictGetOrNull" -"dictGetString" -"dictGetStringOrDefault" -"dictGetUInt16" -"dictGetUInt16OrDefault" -"dictGetUInt32" -"dictGetUInt32OrDefault" -"dictGetUInt64" -"dictGetUInt64OrDefault" -"dictGetUInt8" -"dictGetUInt8OrDefault" -"dictGetUUID" -"dictGetUUIDOrDefault" -"dictHas" -"dictIsIn" -"displayName" -"distanceL1" -"distanceL2" -"distanceL2Squared" -"distanceLinf" -"distanceLp" -"divide" -"divideDecimal" -"domain" -"domainRFC" -"domainWithoutWWW" -"domainWithoutWWWRFC" -"dotProduct" -"dumpColumnStructure" -"dynamicElement" -"dynamicType" -"e" -"editDistance" -"editDistanceUTF8" -"empty" -"emptyArrayDate" -"emptyArrayDateTime" -"emptyArrayFloat32" -"emptyArrayFloat64" -"emptyArrayInt16" -"emptyArrayInt32" -"emptyArrayInt64" -"emptyArrayInt8" -"emptyArrayString" -"emptyArrayToSingle" -"emptyArrayUInt16" -"emptyArrayUInt32" -"emptyArrayUInt64" -"emptyArrayUInt8" -"enabledProfiles" -"enabledRoles" -"encodeURLComponent" -"encodeURLFormComponent" -"encodeXMLComponent" -"encrypt" -"endsWith" -"endsWithUTF8" -"entropy" -"entropyArgMax" -"entropyArgMin" -"entropyArray" -"entropyDistinct" -"entropyForEach" -"entropyIf" -"entropyMap" -"entropyMerge" -"entropyNull" -"entropyOrDefault" -"entropyOrNull" -"entropyResample" -"entropySimpleState" -"entropyState" -"equals" -"erf" -"erfc" -"errorCodeToName" -"evalMLMethod" -"exp" -"exp10" -"exp2" -"exponentialMovingAverage" -"exponentialMovingAverageArgMax" -"exponentialMovingAverageArgMin" -"exponentialMovingAverageArray" -"exponentialMovingAverageDistinct" -"exponentialMovingAverageForEach" -"exponentialMovingAverageIf" -"exponentialMovingAverageMap" -"exponentialMovingAverageMerge" -"exponentialMovingAverageNull" -"exponentialMovingAverageOrDefault" -"exponentialMovingAverageOrNull" -"exponentialMovingAverageResample" -"exponentialMovingAverageSimpleState" -"exponentialMovingAverageState" -"exponentialTimeDecayedAvg" -"exponentialTimeDecayedAvgArgMax" -"exponentialTimeDecayedAvgArgMin" -"exponentialTimeDecayedAvgArray" -"exponentialTimeDecayedAvgDistinct" -"exponentialTimeDecayedAvgForEach" -"exponentialTimeDecayedAvgIf" -"exponentialTimeDecayedAvgMap" -"exponentialTimeDecayedAvgMerge" -"exponentialTimeDecayedAvgNull" -"exponentialTimeDecayedAvgOrDefault" -"exponentialTimeDecayedAvgOrNull" -"exponentialTimeDecayedAvgResample" -"exponentialTimeDecayedAvgSimpleState" -"exponentialTimeDecayedAvgState" -"exponentialTimeDecayedCount" -"exponentialTimeDecayedCountArgMax" -"exponentialTimeDecayedCountArgMin" -"exponentialTimeDecayedCountArray" -"exponentialTimeDecayedCountDistinct" -"exponentialTimeDecayedCountForEach" -"exponentialTimeDecayedCountIf" -"exponentialTimeDecayedCountMap" -"exponentialTimeDecayedCountMerge" -"exponentialTimeDecayedCountNull" -"exponentialTimeDecayedCountOrDefault" -"exponentialTimeDecayedCountOrNull" -"exponentialTimeDecayedCountResample" -"exponentialTimeDecayedCountSimpleState" -"exponentialTimeDecayedCountState" -"exponentialTimeDecayedMax" -"exponentialTimeDecayedMaxArgMax" -"exponentialTimeDecayedMaxArgMin" -"exponentialTimeDecayedMaxArray" -"exponentialTimeDecayedMaxDistinct" -"exponentialTimeDecayedMaxForEach" -"exponentialTimeDecayedMaxIf" -"exponentialTimeDecayedMaxMap" -"exponentialTimeDecayedMaxMerge" -"exponentialTimeDecayedMaxNull" -"exponentialTimeDecayedMaxOrDefault" -"exponentialTimeDecayedMaxOrNull" -"exponentialTimeDecayedMaxResample" -"exponentialTimeDecayedMaxSimpleState" -"exponentialTimeDecayedMaxState" -"exponentialTimeDecayedSum" -"exponentialTimeDecayedSumArgMax" -"exponentialTimeDecayedSumArgMin" -"exponentialTimeDecayedSumArray" -"exponentialTimeDecayedSumDistinct" -"exponentialTimeDecayedSumForEach" -"exponentialTimeDecayedSumIf" -"exponentialTimeDecayedSumMap" -"exponentialTimeDecayedSumMerge" -"exponentialTimeDecayedSumNull" -"exponentialTimeDecayedSumOrDefault" -"exponentialTimeDecayedSumOrNull" -"exponentialTimeDecayedSumResample" -"exponentialTimeDecayedSumSimpleState" -"exponentialTimeDecayedSumState" -"extract" -"extractAll" -"extractAllGroups" -"extractAllGroupsHorizontal" -"extractAllGroupsVertical" -"extractGroups" -"extractKeyValuePairs" -"extractKeyValuePairsWithEscaping" -"extractTextFromHTML" -"extractURLParameter" -"extractURLParameterNames" -"extractURLParameters" -"factorial" -"farmFingerprint64" -"farmHash64" -"file" -"filesystemAvailable" -"filesystemCapacity" -"filesystemUnreserved" -"finalizeAggregation" -"firstLine" -"firstSignificantSubdomain" -"firstSignificantSubdomainCustom" -"firstSignificantSubdomainCustomRFC" -"firstSignificantSubdomainRFC" -"first_value" -"first_valueArgMax" -"first_valueArgMin" -"first_valueArray" -"first_valueDistinct" -"first_valueForEach" -"first_valueIf" -"first_valueMap" -"first_valueMerge" -"first_valueNull" -"first_valueOrDefault" -"first_valueOrNull" -"first_valueResample" -"first_valueSimpleState" -"first_valueState" -"first_value_respect_nulls" -"first_value_respect_nullsArgMax" -"first_value_respect_nullsArgMin" -"first_value_respect_nullsArray" -"first_value_respect_nullsDistinct" -"first_value_respect_nullsForEach" -"first_value_respect_nullsIf" -"first_value_respect_nullsMap" -"first_value_respect_nullsMerge" -"first_value_respect_nullsNull" -"first_value_respect_nullsOrDefault" -"first_value_respect_nullsOrNull" -"first_value_respect_nullsResample" -"first_value_respect_nullsSimpleState" -"first_value_respect_nullsState" -"flameGraph" -"flameGraphArgMax" -"flameGraphArgMin" -"flameGraphArray" -"flameGraphDistinct" -"flameGraphForEach" -"flameGraphIf" -"flameGraphMap" -"flameGraphMerge" -"flameGraphNull" -"flameGraphOrDefault" -"flameGraphOrNull" -"flameGraphResample" -"flameGraphSimpleState" -"flameGraphState" -"flatten" -"flattenTuple" -"floor" -"format" -"formatDateTime" -"formatDateTimeInJodaSyntax" -"formatQuery" -"formatQueryOrNull" -"formatQuerySingleLine" -"formatQuerySingleLineOrNull" -"formatReadableDecimalSize" -"formatReadableQuantity" -"formatReadableSize" -"formatReadableTimeDelta" -"formatRow" -"formatRowNoNewline" -"fragment" -"fromDaysSinceYearZero" -"fromDaysSinceYearZero32" -"fromModifiedJulianDay" -"fromModifiedJulianDayOrNull" -"fromUTCTimestamp" -"fromUnixTimestamp" -"fromUnixTimestamp64Micro" -"fromUnixTimestamp64Milli" -"fromUnixTimestamp64Nano" -"fromUnixTimestampInJodaSyntax" -"from_utc_timestamp" -"fullHostName" -"fuzzBits" -"gccMurmurHash" -"gcd" -"generateRandomStructure" -"generateSnowflakeID" -"generateULID" -"generateUUIDv4" -"generateUUIDv7" -"geoDistance" -"geoToH3" -"geoToS2" -"geohashDecode" -"geohashEncode" -"geohashesInBox" -"getClientHTTPHeader" -"getMacro" -"getOSKernelVersion" -"getServerPort" -"getSetting" -"getSizeOfEnumType" -"getSubcolumn" -"getTypeSerializationStreams" -"globalIn" -"globalInIgnoreSet" -"globalNotIn" -"globalNotInIgnoreSet" -"globalNotNullIn" -"globalNotNullInIgnoreSet" -"globalNullIn" -"globalNullInIgnoreSet" -"globalVariable" -"greatCircleAngle" -"greatCircleDistance" -"greater" -"greaterOrEquals" -"greatest" -"groupArray" -"groupArrayArgMax" -"groupArrayArgMin" -"groupArrayArray" -"groupArrayDistinct" -"groupArrayForEach" -"groupArrayIf" -"groupArrayInsertAt" -"groupArrayInsertAtArgMax" -"groupArrayInsertAtArgMin" -"groupArrayInsertAtArray" -"groupArrayInsertAtDistinct" -"groupArrayInsertAtForEach" -"groupArrayInsertAtIf" -"groupArrayInsertAtMap" -"groupArrayInsertAtMerge" -"groupArrayInsertAtNull" -"groupArrayInsertAtOrDefault" -"groupArrayInsertAtOrNull" -"groupArrayInsertAtResample" -"groupArrayInsertAtSimpleState" -"groupArrayInsertAtState" -"groupArrayIntersect" -"groupArrayIntersectArgMax" -"groupArrayIntersectArgMin" -"groupArrayIntersectArray" -"groupArrayIntersectDistinct" -"groupArrayIntersectForEach" -"groupArrayIntersectIf" -"groupArrayIntersectMap" -"groupArrayIntersectMerge" -"groupArrayIntersectNull" -"groupArrayIntersectOrDefault" -"groupArrayIntersectOrNull" -"groupArrayIntersectResample" -"groupArrayIntersectSimpleState" -"groupArrayIntersectState" -"groupArrayLast" -"groupArrayLastArgMax" -"groupArrayLastArgMin" -"groupArrayLastArray" -"groupArrayLastDistinct" -"groupArrayLastForEach" -"groupArrayLastIf" -"groupArrayLastMap" -"groupArrayLastMerge" -"groupArrayLastNull" -"groupArrayLastOrDefault" -"groupArrayLastOrNull" -"groupArrayLastResample" -"groupArrayLastSimpleState" -"groupArrayLastState" -"groupArrayMap" -"groupArrayMerge" -"groupArrayMovingAvg" -"groupArrayMovingAvgArgMax" -"groupArrayMovingAvgArgMin" -"groupArrayMovingAvgArray" -"groupArrayMovingAvgDistinct" -"groupArrayMovingAvgForEach" -"groupArrayMovingAvgIf" -"groupArrayMovingAvgMap" -"groupArrayMovingAvgMerge" -"groupArrayMovingAvgNull" -"groupArrayMovingAvgOrDefault" -"groupArrayMovingAvgOrNull" -"groupArrayMovingAvgResample" -"groupArrayMovingAvgSimpleState" -"groupArrayMovingAvgState" -"groupArrayMovingSum" -"groupArrayMovingSumArgMax" -"groupArrayMovingSumArgMin" -"groupArrayMovingSumArray" -"groupArrayMovingSumDistinct" -"groupArrayMovingSumForEach" -"groupArrayMovingSumIf" -"groupArrayMovingSumMap" -"groupArrayMovingSumMerge" -"groupArrayMovingSumNull" -"groupArrayMovingSumOrDefault" -"groupArrayMovingSumOrNull" -"groupArrayMovingSumResample" -"groupArrayMovingSumSimpleState" -"groupArrayMovingSumState" -"groupArrayNull" -"groupArrayOrDefault" -"groupArrayOrNull" -"groupArrayResample" -"groupArraySample" -"groupArraySampleArgMax" -"groupArraySampleArgMin" -"groupArraySampleArray" -"groupArraySampleDistinct" -"groupArraySampleForEach" -"groupArraySampleIf" -"groupArraySampleMap" -"groupArraySampleMerge" -"groupArraySampleNull" -"groupArraySampleOrDefault" -"groupArraySampleOrNull" -"groupArraySampleResample" -"groupArraySampleSimpleState" -"groupArraySampleState" -"groupArraySimpleState" -"groupArraySorted" -"groupArraySortedArgMax" -"groupArraySortedArgMin" -"groupArraySortedArray" -"groupArraySortedDistinct" -"groupArraySortedForEach" -"groupArraySortedIf" -"groupArraySortedMap" -"groupArraySortedMerge" -"groupArraySortedNull" -"groupArraySortedOrDefault" -"groupArraySortedOrNull" -"groupArraySortedResample" -"groupArraySortedSimpleState" -"groupArraySortedState" -"groupArrayState" -"groupBitAnd" -"groupBitAndArgMax" -"groupBitAndArgMin" -"groupBitAndArray" -"groupBitAndDistinct" -"groupBitAndForEach" -"groupBitAndIf" -"groupBitAndMap" -"groupBitAndMerge" -"groupBitAndNull" -"groupBitAndOrDefault" -"groupBitAndOrNull" -"groupBitAndResample" -"groupBitAndSimpleState" -"groupBitAndState" -"groupBitOr" -"groupBitOrArgMax" -"groupBitOrArgMin" -"groupBitOrArray" -"groupBitOrDistinct" -"groupBitOrForEach" -"groupBitOrIf" -"groupBitOrMap" -"groupBitOrMerge" -"groupBitOrNull" -"groupBitOrOrDefault" -"groupBitOrOrNull" -"groupBitOrResample" -"groupBitOrSimpleState" -"groupBitOrState" -"groupBitXor" -"groupBitXorArgMax" -"groupBitXorArgMin" -"groupBitXorArray" -"groupBitXorDistinct" -"groupBitXorForEach" -"groupBitXorIf" -"groupBitXorMap" -"groupBitXorMerge" -"groupBitXorNull" -"groupBitXorOrDefault" -"groupBitXorOrNull" -"groupBitXorResample" -"groupBitXorSimpleState" -"groupBitXorState" -"groupBitmap" -"groupBitmapAnd" -"groupBitmapAndArgMax" -"groupBitmapAndArgMin" -"groupBitmapAndArray" -"groupBitmapAndDistinct" -"groupBitmapAndForEach" -"groupBitmapAndIf" -"groupBitmapAndMap" -"groupBitmapAndMerge" -"groupBitmapAndNull" -"groupBitmapAndOrDefault" -"groupBitmapAndOrNull" -"groupBitmapAndResample" -"groupBitmapAndSimpleState" -"groupBitmapAndState" -"groupBitmapArgMax" -"groupBitmapArgMin" -"groupBitmapArray" -"groupBitmapDistinct" -"groupBitmapForEach" -"groupBitmapIf" -"groupBitmapMap" -"groupBitmapMerge" -"groupBitmapNull" -"groupBitmapOr" -"groupBitmapOrArgMax" -"groupBitmapOrArgMin" -"groupBitmapOrArray" -"groupBitmapOrDefault" -"groupBitmapOrDistinct" -"groupBitmapOrForEach" -"groupBitmapOrIf" -"groupBitmapOrMap" -"groupBitmapOrMerge" -"groupBitmapOrNull" -"groupBitmapOrNull" -"groupBitmapOrOrDefault" -"groupBitmapOrOrNull" -"groupBitmapOrResample" -"groupBitmapOrSimpleState" -"groupBitmapOrState" -"groupBitmapResample" -"groupBitmapSimpleState" -"groupBitmapState" -"groupBitmapXor" -"groupBitmapXorArgMax" -"groupBitmapXorArgMin" -"groupBitmapXorArray" -"groupBitmapXorDistinct" -"groupBitmapXorForEach" -"groupBitmapXorIf" -"groupBitmapXorMap" -"groupBitmapXorMerge" -"groupBitmapXorNull" -"groupBitmapXorOrDefault" -"groupBitmapXorOrNull" -"groupBitmapXorResample" -"groupBitmapXorSimpleState" -"groupBitmapXorState" -"groupConcat" -"groupConcatArgMax" -"groupConcatArgMin" -"groupConcatArray" -"groupConcatDistinct" -"groupConcatForEach" -"groupConcatIf" -"groupConcatMap" -"groupConcatMerge" -"groupConcatNull" -"groupConcatOrDefault" -"groupConcatOrNull" -"groupConcatResample" -"groupConcatSimpleState" -"groupConcatState" -"groupUniqArray" -"groupUniqArrayArgMax" -"groupUniqArrayArgMin" -"groupUniqArrayArray" -"groupUniqArrayDistinct" -"groupUniqArrayForEach" -"groupUniqArrayIf" -"groupUniqArrayMap" -"groupUniqArrayMerge" -"groupUniqArrayNull" -"groupUniqArrayOrDefault" -"groupUniqArrayOrNull" -"groupUniqArrayResample" -"groupUniqArraySimpleState" -"groupUniqArrayState" -"group_concat" -"group_concatArgMax" -"group_concatArgMin" -"group_concatArray" -"group_concatDistinct" -"group_concatForEach" -"group_concatIf" -"group_concatMap" -"group_concatMerge" -"group_concatNull" -"group_concatOrDefault" -"group_concatOrNull" -"group_concatResample" -"group_concatSimpleState" -"group_concatState" -"h3CellAreaM2" -"h3CellAreaRads2" -"h3Distance" -"h3EdgeAngle" -"h3EdgeLengthKm" -"h3EdgeLengthM" -"h3ExactEdgeLengthKm" -"h3ExactEdgeLengthM" -"h3ExactEdgeLengthRads" -"h3GetBaseCell" -"h3GetDestinationIndexFromUnidirectionalEdge" -"h3GetFaces" -"h3GetIndexesFromUnidirectionalEdge" -"h3GetOriginIndexFromUnidirectionalEdge" -"h3GetPentagonIndexes" -"h3GetRes0Indexes" -"h3GetResolution" -"h3GetUnidirectionalEdge" -"h3GetUnidirectionalEdgeBoundary" -"h3GetUnidirectionalEdgesFromHexagon" -"h3HexAreaKm2" -"h3HexAreaM2" -"h3HexRing" -"h3IndexesAreNeighbors" -"h3IsPentagon" -"h3IsResClassIII" -"h3IsValid" -"h3Line" -"h3NumHexagons" -"h3PointDistKm" -"h3PointDistM" -"h3PointDistRads" -"h3ToCenterChild" -"h3ToChildren" -"h3ToGeo" -"h3ToGeoBoundary" -"h3ToParent" -"h3ToString" -"h3UnidirectionalEdgeIsValid" -"h3kRing" -"halfMD5" -"has" -"hasAll" -"hasAny" -"hasColumnInTable" -"hasSubsequence" -"hasSubsequenceCaseInsensitive" -"hasSubsequenceCaseInsensitiveUTF8" -"hasSubsequenceUTF8" -"hasSubstr" -"hasThreadFuzzer" -"hasToken" -"hasTokenCaseInsensitive" -"hasTokenCaseInsensitiveOrNull" -"hasTokenOrNull" -"hex" -"hilbertDecode" -"hilbertEncode" -"histogram" -"histogramArgMax" -"histogramArgMin" -"histogramArray" -"histogramDistinct" -"histogramForEach" -"histogramIf" -"histogramMap" -"histogramMerge" -"histogramNull" -"histogramOrDefault" -"histogramOrNull" -"histogramResample" -"histogramSimpleState" -"histogramState" -"hiveHash" -"hop" -"hopEnd" -"hopStart" -"hostName" -"hostname" -"hypot" -"identity" -"idnaDecode" -"idnaEncode" -"if" -"ifNotFinite" -"ifNull" -"ignore" -"ilike" -"in" -"inIgnoreSet" -"indexHint" -"indexOf" -"initcap" -"initcapUTF8" -"initialQueryID" -"initial_query_id" -"initializeAggregation" -"instr" -"intDiv" -"intDivOrZero" -"intExp10" -"intExp2" -"intHash32" -"intHash64" -"intervalLengthSum" -"intervalLengthSumArgMax" -"intervalLengthSumArgMin" -"intervalLengthSumArray" -"intervalLengthSumDistinct" -"intervalLengthSumForEach" -"intervalLengthSumIf" -"intervalLengthSumMap" -"intervalLengthSumMerge" -"intervalLengthSumNull" -"intervalLengthSumOrDefault" -"intervalLengthSumOrNull" -"intervalLengthSumResample" -"intervalLengthSumSimpleState" -"intervalLengthSumState" -"isConstant" -"isDecimalOverflow" -"isFinite" -"isIPAddressInRange" -"isIPv4String" -"isIPv6String" -"isInfinite" -"isNaN" -"isNotDistinctFrom" -"isNotNull" -"isNull" -"isNullable" -"isValidJSON" -"isValidUTF8" -"isZeroOrNull" -"jaroSimilarity" -"jaroWinklerSimilarity" -"javaHash" -"javaHashUTF16LE" -"joinGet" -"joinGetOrNull" -"jsonMergePatch" -"jumpConsistentHash" -"kafkaMurmurHash" -"kolmogorovSmirnovTest" -"kolmogorovSmirnovTestArgMax" -"kolmogorovSmirnovTestArgMin" -"kolmogorovSmirnovTestArray" -"kolmogorovSmirnovTestDistinct" -"kolmogorovSmirnovTestForEach" -"kolmogorovSmirnovTestIf" -"kolmogorovSmirnovTestMap" -"kolmogorovSmirnovTestMerge" -"kolmogorovSmirnovTestNull" -"kolmogorovSmirnovTestOrDefault" -"kolmogorovSmirnovTestOrNull" -"kolmogorovSmirnovTestResample" -"kolmogorovSmirnovTestSimpleState" -"kolmogorovSmirnovTestState" -"kostikConsistentHash" -"kql_array_sort_asc" -"kql_array_sort_desc" -"kurtPop" -"kurtPopArgMax" -"kurtPopArgMin" -"kurtPopArray" -"kurtPopDistinct" -"kurtPopForEach" -"kurtPopIf" -"kurtPopMap" -"kurtPopMerge" -"kurtPopNull" -"kurtPopOrDefault" -"kurtPopOrNull" -"kurtPopResample" -"kurtPopSimpleState" -"kurtPopState" -"kurtSamp" -"kurtSampArgMax" -"kurtSampArgMin" -"kurtSampArray" -"kurtSampDistinct" -"kurtSampForEach" -"kurtSampIf" -"kurtSampMap" -"kurtSampMerge" -"kurtSampNull" -"kurtSampOrDefault" -"kurtSampOrNull" -"kurtSampResample" -"kurtSampSimpleState" -"kurtSampState" -"lagInFrame" -"lagInFrameArgMax" -"lagInFrameArgMin" -"lagInFrameArray" -"lagInFrameDistinct" -"lagInFrameForEach" -"lagInFrameIf" -"lagInFrameMap" -"lagInFrameMerge" -"lagInFrameNull" -"lagInFrameOrDefault" -"lagInFrameOrNull" -"lagInFrameResample" -"lagInFrameSimpleState" -"lagInFrameState" -"largestTriangleThreeBuckets" -"largestTriangleThreeBucketsArgMax" -"largestTriangleThreeBucketsArgMin" -"largestTriangleThreeBucketsArray" -"largestTriangleThreeBucketsDistinct" -"largestTriangleThreeBucketsForEach" -"largestTriangleThreeBucketsIf" -"largestTriangleThreeBucketsMap" -"largestTriangleThreeBucketsMerge" -"largestTriangleThreeBucketsNull" -"largestTriangleThreeBucketsOrDefault" -"largestTriangleThreeBucketsOrNull" -"largestTriangleThreeBucketsResample" -"largestTriangleThreeBucketsSimpleState" -"largestTriangleThreeBucketsState" -"last_value" -"last_valueArgMax" -"last_valueArgMin" -"last_valueArray" -"last_valueDistinct" -"last_valueForEach" -"last_valueIf" -"last_valueMap" -"last_valueMerge" -"last_valueNull" -"last_valueOrDefault" -"last_valueOrNull" -"last_valueResample" -"last_valueSimpleState" -"last_valueState" -"last_value_respect_nulls" -"last_value_respect_nullsArgMax" -"last_value_respect_nullsArgMin" -"last_value_respect_nullsArray" -"last_value_respect_nullsDistinct" -"last_value_respect_nullsForEach" -"last_value_respect_nullsIf" -"last_value_respect_nullsMap" -"last_value_respect_nullsMerge" -"last_value_respect_nullsNull" -"last_value_respect_nullsOrDefault" -"last_value_respect_nullsOrNull" -"last_value_respect_nullsResample" -"last_value_respect_nullsSimpleState" -"last_value_respect_nullsState" -"lcase" -"lcm" -"leadInFrame" -"leadInFrameArgMax" -"leadInFrameArgMin" -"leadInFrameArray" -"leadInFrameDistinct" -"leadInFrameForEach" -"leadInFrameIf" -"leadInFrameMap" -"leadInFrameMerge" -"leadInFrameNull" -"leadInFrameOrDefault" -"leadInFrameOrNull" -"leadInFrameResample" -"leadInFrameSimpleState" -"leadInFrameState" -"least" -"left" -"leftPad" -"leftPadUTF8" -"leftUTF8" -"lemmatize" -"length" -"lengthUTF8" -"less" -"lessOrEquals" -"levenshteinDistance" -"levenshteinDistanceUTF8" -"lgamma" -"like" -"ln" -"locate" -"log" -"log10" -"log1p" -"log2" -"logTrace" -"lowCardinalityIndices" -"lowCardinalityKeys" -"lower" -"lowerUTF8" -"lpad" -"ltrim" -"lttb" -"lttbArgMax" -"lttbArgMin" -"lttbArray" -"lttbDistinct" -"lttbForEach" -"lttbIf" -"lttbMap" -"lttbMerge" -"lttbNull" -"lttbOrDefault" -"lttbOrNull" -"lttbResample" -"lttbSimpleState" -"lttbState" -"makeDate" -"makeDate32" -"makeDateTime" -"makeDateTime64" -"mannWhitneyUTest" -"mannWhitneyUTestArgMax" -"mannWhitneyUTestArgMin" -"mannWhitneyUTestArray" -"mannWhitneyUTestDistinct" -"mannWhitneyUTestForEach" -"mannWhitneyUTestIf" -"mannWhitneyUTestMap" -"mannWhitneyUTestMerge" -"mannWhitneyUTestNull" -"mannWhitneyUTestOrDefault" -"mannWhitneyUTestOrNull" -"mannWhitneyUTestResample" -"mannWhitneyUTestSimpleState" -"mannWhitneyUTestState" -"map" -"mapAdd" -"mapAll" -"mapApply" -"mapConcat" -"mapContains" -"mapContainsKeyLike" -"mapExists" -"mapExtractKeyLike" -"mapFilter" -"mapFromArrays" -"mapFromString" -"mapKeys" -"mapPartialReverseSort" -"mapPartialSort" -"mapPopulateSeries" -"mapReverseSort" -"mapSort" -"mapSubtract" -"mapUpdate" -"mapValues" -"match" -"materialize" -"max" -"max2" -"maxArgMax" -"maxArgMin" -"maxArray" -"maxDistinct" -"maxForEach" -"maxIf" -"maxIntersections" -"maxIntersectionsArgMax" -"maxIntersectionsArgMin" -"maxIntersectionsArray" -"maxIntersectionsDistinct" -"maxIntersectionsForEach" -"maxIntersectionsIf" -"maxIntersectionsMap" -"maxIntersectionsMerge" -"maxIntersectionsNull" -"maxIntersectionsOrDefault" -"maxIntersectionsOrNull" -"maxIntersectionsPosition" -"maxIntersectionsPositionArgMax" -"maxIntersectionsPositionArgMin" -"maxIntersectionsPositionArray" -"maxIntersectionsPositionDistinct" -"maxIntersectionsPositionForEach" -"maxIntersectionsPositionIf" -"maxIntersectionsPositionMap" -"maxIntersectionsPositionMerge" -"maxIntersectionsPositionNull" -"maxIntersectionsPositionOrDefault" -"maxIntersectionsPositionOrNull" -"maxIntersectionsPositionResample" -"maxIntersectionsPositionSimpleState" -"maxIntersectionsPositionState" -"maxIntersectionsResample" -"maxIntersectionsSimpleState" -"maxIntersectionsState" -"maxMap" -"maxMappedArrays" -"maxMappedArraysArgMax" -"maxMappedArraysArgMin" -"maxMappedArraysArray" -"maxMappedArraysDistinct" -"maxMappedArraysForEach" -"maxMappedArraysIf" -"maxMappedArraysMap" -"maxMappedArraysMerge" -"maxMappedArraysNull" -"maxMappedArraysOrDefault" -"maxMappedArraysOrNull" -"maxMappedArraysResample" -"maxMappedArraysSimpleState" -"maxMappedArraysState" -"maxMerge" -"maxNull" -"maxOrDefault" -"maxOrNull" -"maxResample" -"maxSimpleState" -"maxState" -"meanZTest" -"meanZTestArgMax" -"meanZTestArgMin" -"meanZTestArray" -"meanZTestDistinct" -"meanZTestForEach" -"meanZTestIf" -"meanZTestMap" -"meanZTestMerge" -"meanZTestNull" -"meanZTestOrDefault" -"meanZTestOrNull" -"meanZTestResample" -"meanZTestSimpleState" -"meanZTestState" -"median" -"medianArgMax" -"medianArgMin" -"medianArray" -"medianBFloat16" -"medianBFloat16ArgMax" -"medianBFloat16ArgMin" -"medianBFloat16Array" -"medianBFloat16Distinct" -"medianBFloat16ForEach" -"medianBFloat16If" -"medianBFloat16Map" -"medianBFloat16Merge" -"medianBFloat16Null" -"medianBFloat16OrDefault" -"medianBFloat16OrNull" -"medianBFloat16Resample" -"medianBFloat16SimpleState" -"medianBFloat16State" -"medianBFloat16Weighted" -"medianBFloat16WeightedArgMax" -"medianBFloat16WeightedArgMin" -"medianBFloat16WeightedArray" -"medianBFloat16WeightedDistinct" -"medianBFloat16WeightedForEach" -"medianBFloat16WeightedIf" -"medianBFloat16WeightedMap" -"medianBFloat16WeightedMerge" -"medianBFloat16WeightedNull" -"medianBFloat16WeightedOrDefault" -"medianBFloat16WeightedOrNull" -"medianBFloat16WeightedResample" -"medianBFloat16WeightedSimpleState" -"medianBFloat16WeightedState" -"medianDD" -"medianDDArgMax" -"medianDDArgMin" -"medianDDArray" -"medianDDDistinct" -"medianDDForEach" -"medianDDIf" -"medianDDMap" -"medianDDMerge" -"medianDDNull" -"medianDDOrDefault" -"medianDDOrNull" -"medianDDResample" -"medianDDSimpleState" -"medianDDState" -"medianDeterministic" -"medianDeterministicArgMax" -"medianDeterministicArgMin" -"medianDeterministicArray" -"medianDeterministicDistinct" -"medianDeterministicForEach" -"medianDeterministicIf" -"medianDeterministicMap" -"medianDeterministicMerge" -"medianDeterministicNull" -"medianDeterministicOrDefault" -"medianDeterministicOrNull" -"medianDeterministicResample" -"medianDeterministicSimpleState" -"medianDeterministicState" -"medianDistinct" -"medianExact" -"medianExactArgMax" -"medianExactArgMin" -"medianExactArray" -"medianExactDistinct" -"medianExactForEach" -"medianExactHigh" -"medianExactHighArgMax" -"medianExactHighArgMin" -"medianExactHighArray" -"medianExactHighDistinct" -"medianExactHighForEach" -"medianExactHighIf" -"medianExactHighMap" -"medianExactHighMerge" -"medianExactHighNull" -"medianExactHighOrDefault" -"medianExactHighOrNull" -"medianExactHighResample" -"medianExactHighSimpleState" -"medianExactHighState" -"medianExactIf" -"medianExactLow" -"medianExactLowArgMax" -"medianExactLowArgMin" -"medianExactLowArray" -"medianExactLowDistinct" -"medianExactLowForEach" -"medianExactLowIf" -"medianExactLowMap" -"medianExactLowMerge" -"medianExactLowNull" -"medianExactLowOrDefault" -"medianExactLowOrNull" -"medianExactLowResample" -"medianExactLowSimpleState" -"medianExactLowState" -"medianExactMap" -"medianExactMerge" -"medianExactNull" -"medianExactOrDefault" -"medianExactOrNull" -"medianExactResample" -"medianExactSimpleState" -"medianExactState" -"medianExactWeighted" -"medianExactWeightedArgMax" -"medianExactWeightedArgMin" -"medianExactWeightedArray" -"medianExactWeightedDistinct" -"medianExactWeightedForEach" -"medianExactWeightedIf" -"medianExactWeightedMap" -"medianExactWeightedMerge" -"medianExactWeightedNull" -"medianExactWeightedOrDefault" -"medianExactWeightedOrNull" -"medianExactWeightedResample" -"medianExactWeightedSimpleState" -"medianExactWeightedState" -"medianForEach" -"medianGK" -"medianGKArgMax" -"medianGKArgMin" -"medianGKArray" -"medianGKDistinct" -"medianGKForEach" -"medianGKIf" -"medianGKMap" -"medianGKMerge" -"medianGKNull" -"medianGKOrDefault" -"medianGKOrNull" -"medianGKResample" -"medianGKSimpleState" -"medianGKState" -"medianIf" -"medianInterpolatedWeighted" -"medianInterpolatedWeightedArgMax" -"medianInterpolatedWeightedArgMin" -"medianInterpolatedWeightedArray" -"medianInterpolatedWeightedDistinct" -"medianInterpolatedWeightedForEach" -"medianInterpolatedWeightedIf" -"medianInterpolatedWeightedMap" -"medianInterpolatedWeightedMerge" -"medianInterpolatedWeightedNull" -"medianInterpolatedWeightedOrDefault" -"medianInterpolatedWeightedOrNull" -"medianInterpolatedWeightedResample" -"medianInterpolatedWeightedSimpleState" -"medianInterpolatedWeightedState" -"medianMap" -"medianMerge" -"medianNull" -"medianOrDefault" -"medianOrNull" -"medianResample" -"medianSimpleState" -"medianState" -"medianTDigest" -"medianTDigestArgMax" -"medianTDigestArgMin" -"medianTDigestArray" -"medianTDigestDistinct" -"medianTDigestForEach" -"medianTDigestIf" -"medianTDigestMap" -"medianTDigestMerge" -"medianTDigestNull" -"medianTDigestOrDefault" -"medianTDigestOrNull" -"medianTDigestResample" -"medianTDigestSimpleState" -"medianTDigestState" -"medianTDigestWeighted" -"medianTDigestWeightedArgMax" -"medianTDigestWeightedArgMin" -"medianTDigestWeightedArray" -"medianTDigestWeightedDistinct" -"medianTDigestWeightedForEach" -"medianTDigestWeightedIf" -"medianTDigestWeightedMap" -"medianTDigestWeightedMerge" -"medianTDigestWeightedNull" -"medianTDigestWeightedOrDefault" -"medianTDigestWeightedOrNull" -"medianTDigestWeightedResample" -"medianTDigestWeightedSimpleState" -"medianTDigestWeightedState" -"medianTiming" -"medianTimingArgMax" -"medianTimingArgMin" -"medianTimingArray" -"medianTimingDistinct" -"medianTimingForEach" -"medianTimingIf" -"medianTimingMap" -"medianTimingMerge" -"medianTimingNull" -"medianTimingOrDefault" -"medianTimingOrNull" -"medianTimingResample" -"medianTimingSimpleState" -"medianTimingState" -"medianTimingWeighted" -"medianTimingWeightedArgMax" -"medianTimingWeightedArgMin" -"medianTimingWeightedArray" -"medianTimingWeightedDistinct" -"medianTimingWeightedForEach" -"medianTimingWeightedIf" -"medianTimingWeightedMap" -"medianTimingWeightedMerge" -"medianTimingWeightedNull" -"medianTimingWeightedOrDefault" -"medianTimingWeightedOrNull" -"medianTimingWeightedResample" -"medianTimingWeightedSimpleState" -"medianTimingWeightedState" -"metroHash64" -"mid" -"min" -"min2" -"minArgMax" -"minArgMin" -"minArray" -"minDistinct" -"minForEach" -"minIf" -"minMap" -"minMappedArrays" -"minMappedArraysArgMax" -"minMappedArraysArgMin" -"minMappedArraysArray" -"minMappedArraysDistinct" -"minMappedArraysForEach" -"minMappedArraysIf" -"minMappedArraysMap" -"minMappedArraysMerge" -"minMappedArraysNull" -"minMappedArraysOrDefault" -"minMappedArraysOrNull" -"minMappedArraysResample" -"minMappedArraysSimpleState" -"minMappedArraysState" -"minMerge" -"minNull" -"minOrDefault" -"minOrNull" -"minResample" -"minSampleSizeContinous" -"minSampleSizeContinuous" -"minSampleSizeConversion" -"minSimpleState" -"minState" -"minus" -"mismatches" -"mod" -"modulo" -"moduloLegacy" -"moduloOrZero" -"monthName" -"mortonDecode" -"mortonEncode" -"multiFuzzyMatchAllIndices" -"multiFuzzyMatchAny" -"multiFuzzyMatchAnyIndex" -"multiIf" -"multiMatchAllIndices" -"multiMatchAny" -"multiMatchAnyIndex" -"multiSearchAllPositions" -"multiSearchAllPositionsCaseInsensitive" -"multiSearchAllPositionsCaseInsensitiveUTF8" -"multiSearchAllPositionsUTF8" -"multiSearchAny" -"multiSearchAnyCaseInsensitive" -"multiSearchAnyCaseInsensitiveUTF8" -"multiSearchAnyUTF8" -"multiSearchFirstIndex" -"multiSearchFirstIndexCaseInsensitive" -"multiSearchFirstIndexCaseInsensitiveUTF8" -"multiSearchFirstIndexUTF8" -"multiSearchFirstPosition" -"multiSearchFirstPositionCaseInsensitive" -"multiSearchFirstPositionCaseInsensitiveUTF8" -"multiSearchFirstPositionUTF8" -"multiply" -"multiplyDecimal" -"murmurHash2_32" -"murmurHash2_64" -"murmurHash3_128" -"murmurHash3_32" -"murmurHash3_64" -"negate" -"neighbor" -"nested" -"netloc" -"ngramDistance" -"ngramDistanceCaseInsensitive" -"ngramDistanceCaseInsensitiveUTF8" -"ngramDistanceUTF8" -"ngramMinHash" -"ngramMinHashArg" -"ngramMinHashArgCaseInsensitive" -"ngramMinHashArgCaseInsensitiveUTF8" -"ngramMinHashArgUTF8" -"ngramMinHashCaseInsensitive" -"ngramMinHashCaseInsensitiveUTF8" -"ngramMinHashUTF8" -"ngramSearch" -"ngramSearchCaseInsensitive" -"ngramSearchCaseInsensitiveUTF8" -"ngramSearchUTF8" -"ngramSimHash" -"ngramSimHashCaseInsensitive" -"ngramSimHashCaseInsensitiveUTF8" -"ngramSimHashUTF8" -"ngrams" -"nonNegativeDerivative" -"nonNegativeDerivativeArgMax" -"nonNegativeDerivativeArgMin" -"nonNegativeDerivativeArray" -"nonNegativeDerivativeDistinct" -"nonNegativeDerivativeForEach" -"nonNegativeDerivativeIf" -"nonNegativeDerivativeMap" -"nonNegativeDerivativeMerge" -"nonNegativeDerivativeNull" -"nonNegativeDerivativeOrDefault" -"nonNegativeDerivativeOrNull" -"nonNegativeDerivativeResample" -"nonNegativeDerivativeSimpleState" -"nonNegativeDerivativeState" -"normL1" -"normL2" -"normL2Squared" -"normLinf" -"normLp" -"normalizeL1" -"normalizeL2" -"normalizeLinf" -"normalizeLp" -"normalizeQuery" -"normalizeQueryKeepNames" -"normalizeUTF8NFC" -"normalizeUTF8NFD" -"normalizeUTF8NFKC" -"normalizeUTF8NFKD" -"normalizedQueryHash" -"normalizedQueryHashKeepNames" -"not" -"notEmpty" -"notEquals" -"notILike" -"notIn" -"notInIgnoreSet" -"notLike" -"notNullIn" -"notNullInIgnoreSet" -"nothing" -"nothingArgMax" -"nothingArgMin" -"nothingArray" -"nothingDistinct" -"nothingForEach" -"nothingIf" -"nothingMap" -"nothingMerge" -"nothingNull" -"nothingNull" -"nothingNullArgMax" -"nothingNullArgMin" -"nothingNullArray" -"nothingNullDistinct" -"nothingNullForEach" -"nothingNullIf" -"nothingNullMap" -"nothingNullMerge" -"nothingNullNull" -"nothingNullOrDefault" -"nothingNullOrNull" -"nothingNullResample" -"nothingNullSimpleState" -"nothingNullState" -"nothingOrDefault" -"nothingOrNull" -"nothingResample" -"nothingSimpleState" -"nothingState" -"nothingUInt64" -"nothingUInt64ArgMax" -"nothingUInt64ArgMin" -"nothingUInt64Array" -"nothingUInt64Distinct" -"nothingUInt64ForEach" -"nothingUInt64If" -"nothingUInt64Map" -"nothingUInt64Merge" -"nothingUInt64Null" -"nothingUInt64OrDefault" -"nothingUInt64OrNull" -"nothingUInt64Resample" -"nothingUInt64SimpleState" -"nothingUInt64State" -"now" -"now64" -"nowInBlock" -"nth_value" -"nth_valueArgMax" -"nth_valueArgMin" -"nth_valueArray" -"nth_valueDistinct" -"nth_valueForEach" -"nth_valueIf" -"nth_valueMap" -"nth_valueMerge" -"nth_valueNull" -"nth_valueOrDefault" -"nth_valueOrNull" -"nth_valueResample" -"nth_valueSimpleState" -"nth_valueState" -"ntile" -"ntileArgMax" -"ntileArgMin" -"ntileArray" -"ntileDistinct" -"ntileForEach" -"ntileIf" -"ntileMap" -"ntileMerge" -"ntileNull" -"ntileOrDefault" -"ntileOrNull" -"ntileResample" -"ntileSimpleState" -"ntileState" -"nullIf" -"nullIn" -"nullInIgnoreSet" -"or" -"parseDateTime" -"parseDateTime32BestEffort" -"parseDateTime32BestEffortOrNull" -"parseDateTime32BestEffortOrZero" -"parseDateTime64BestEffort" -"parseDateTime64BestEffortOrNull" -"parseDateTime64BestEffortOrZero" -"parseDateTime64BestEffortUS" -"parseDateTime64BestEffortUSOrNull" -"parseDateTime64BestEffortUSOrZero" -"parseDateTimeBestEffort" -"parseDateTimeBestEffortOrNull" -"parseDateTimeBestEffortOrZero" -"parseDateTimeBestEffortUS" -"parseDateTimeBestEffortUSOrNull" -"parseDateTimeBestEffortUSOrZero" -"parseDateTimeInJodaSyntax" -"parseDateTimeInJodaSyntaxOrNull" -"parseDateTimeInJodaSyntaxOrZero" -"parseDateTimeOrNull" -"parseDateTimeOrZero" -"parseReadableSize" -"parseReadableSizeOrNull" -"parseReadableSizeOrZero" -"parseTimeDelta" -"partitionID" -"partitionId" -"path" -"pathFull" -"percentRank" -"percentRankArgMax" -"percentRankArgMin" -"percentRankArray" -"percentRankDistinct" -"percentRankForEach" -"percentRankIf" -"percentRankMap" -"percentRankMerge" -"percentRankNull" -"percentRankOrDefault" -"percentRankOrNull" -"percentRankResample" -"percentRankSimpleState" -"percentRankState" -"percent_rank" -"percent_rankArgMax" -"percent_rankArgMin" -"percent_rankArray" -"percent_rankDistinct" -"percent_rankForEach" -"percent_rankIf" -"percent_rankMap" -"percent_rankMerge" -"percent_rankNull" -"percent_rankOrDefault" -"percent_rankOrNull" -"percent_rankResample" -"percent_rankSimpleState" -"percent_rankState" -"pi" -"plus" -"pmod" -"pointInEllipses" -"pointInPolygon" -"polygonAreaCartesian" -"polygonAreaSpherical" -"polygonConvexHullCartesian" -"polygonPerimeterCartesian" -"polygonPerimeterSpherical" -"polygonsDistanceCartesian" -"polygonsDistanceSpherical" -"polygonsEqualsCartesian" -"polygonsIntersectionCartesian" -"polygonsIntersectionSpherical" -"polygonsSymDifferenceCartesian" -"polygonsSymDifferenceSpherical" -"polygonsUnionCartesian" -"polygonsUnionSpherical" -"polygonsWithinCartesian" -"polygonsWithinSpherical" -"port" -"portRFC" -"position" -"positionCaseInsensitive" -"positionCaseInsensitiveUTF8" -"positionUTF8" -"positiveModulo" -"positive_modulo" -"pow" -"power" -"printf" -"proportionsZTest" -"protocol" -"punycodeDecode" -"punycodeEncode" -"quantile" -"quantileArgMax" -"quantileArgMin" -"quantileArray" -"quantileBFloat16" -"quantileBFloat16ArgMax" -"quantileBFloat16ArgMin" -"quantileBFloat16Array" -"quantileBFloat16Distinct" -"quantileBFloat16ForEach" -"quantileBFloat16If" -"quantileBFloat16Map" -"quantileBFloat16Merge" -"quantileBFloat16Null" -"quantileBFloat16OrDefault" -"quantileBFloat16OrNull" -"quantileBFloat16Resample" -"quantileBFloat16SimpleState" -"quantileBFloat16State" -"quantileBFloat16Weighted" -"quantileBFloat16WeightedArgMax" -"quantileBFloat16WeightedArgMin" -"quantileBFloat16WeightedArray" -"quantileBFloat16WeightedDistinct" -"quantileBFloat16WeightedForEach" -"quantileBFloat16WeightedIf" -"quantileBFloat16WeightedMap" -"quantileBFloat16WeightedMerge" -"quantileBFloat16WeightedNull" -"quantileBFloat16WeightedOrDefault" -"quantileBFloat16WeightedOrNull" -"quantileBFloat16WeightedResample" -"quantileBFloat16WeightedSimpleState" -"quantileBFloat16WeightedState" -"quantileDD" -"quantileDDArgMax" -"quantileDDArgMin" -"quantileDDArray" -"quantileDDDistinct" -"quantileDDForEach" -"quantileDDIf" -"quantileDDMap" -"quantileDDMerge" -"quantileDDNull" -"quantileDDOrDefault" -"quantileDDOrNull" -"quantileDDResample" -"quantileDDSimpleState" -"quantileDDState" -"quantileDeterministic" -"quantileDeterministicArgMax" -"quantileDeterministicArgMin" -"quantileDeterministicArray" -"quantileDeterministicDistinct" -"quantileDeterministicForEach" -"quantileDeterministicIf" -"quantileDeterministicMap" -"quantileDeterministicMerge" -"quantileDeterministicNull" -"quantileDeterministicOrDefault" -"quantileDeterministicOrNull" -"quantileDeterministicResample" -"quantileDeterministicSimpleState" -"quantileDeterministicState" -"quantileDistinct" -"quantileExact" -"quantileExactArgMax" -"quantileExactArgMin" -"quantileExactArray" -"quantileExactDistinct" -"quantileExactExclusive" -"quantileExactExclusiveArgMax" -"quantileExactExclusiveArgMin" -"quantileExactExclusiveArray" -"quantileExactExclusiveDistinct" -"quantileExactExclusiveForEach" -"quantileExactExclusiveIf" -"quantileExactExclusiveMap" -"quantileExactExclusiveMerge" -"quantileExactExclusiveNull" -"quantileExactExclusiveOrDefault" -"quantileExactExclusiveOrNull" -"quantileExactExclusiveResample" -"quantileExactExclusiveSimpleState" -"quantileExactExclusiveState" -"quantileExactForEach" -"quantileExactHigh" -"quantileExactHighArgMax" -"quantileExactHighArgMin" -"quantileExactHighArray" -"quantileExactHighDistinct" -"quantileExactHighForEach" -"quantileExactHighIf" -"quantileExactHighMap" -"quantileExactHighMerge" -"quantileExactHighNull" -"quantileExactHighOrDefault" -"quantileExactHighOrNull" -"quantileExactHighResample" -"quantileExactHighSimpleState" -"quantileExactHighState" -"quantileExactIf" -"quantileExactInclusive" -"quantileExactInclusiveArgMax" -"quantileExactInclusiveArgMin" -"quantileExactInclusiveArray" -"quantileExactInclusiveDistinct" -"quantileExactInclusiveForEach" -"quantileExactInclusiveIf" -"quantileExactInclusiveMap" -"quantileExactInclusiveMerge" -"quantileExactInclusiveNull" -"quantileExactInclusiveOrDefault" -"quantileExactInclusiveOrNull" -"quantileExactInclusiveResample" -"quantileExactInclusiveSimpleState" -"quantileExactInclusiveState" -"quantileExactLow" -"quantileExactLowArgMax" -"quantileExactLowArgMin" -"quantileExactLowArray" -"quantileExactLowDistinct" -"quantileExactLowForEach" -"quantileExactLowIf" -"quantileExactLowMap" -"quantileExactLowMerge" -"quantileExactLowNull" -"quantileExactLowOrDefault" -"quantileExactLowOrNull" -"quantileExactLowResample" -"quantileExactLowSimpleState" -"quantileExactLowState" -"quantileExactMap" -"quantileExactMerge" -"quantileExactNull" -"quantileExactOrDefault" -"quantileExactOrNull" -"quantileExactResample" -"quantileExactSimpleState" -"quantileExactState" -"quantileExactWeighted" -"quantileExactWeightedArgMax" -"quantileExactWeightedArgMin" -"quantileExactWeightedArray" -"quantileExactWeightedDistinct" -"quantileExactWeightedForEach" -"quantileExactWeightedIf" -"quantileExactWeightedMap" -"quantileExactWeightedMerge" -"quantileExactWeightedNull" -"quantileExactWeightedOrDefault" -"quantileExactWeightedOrNull" -"quantileExactWeightedResample" -"quantileExactWeightedSimpleState" -"quantileExactWeightedState" -"quantileForEach" -"quantileGK" -"quantileGKArgMax" -"quantileGKArgMin" -"quantileGKArray" -"quantileGKDistinct" -"quantileGKForEach" -"quantileGKIf" -"quantileGKMap" -"quantileGKMerge" -"quantileGKNull" -"quantileGKOrDefault" -"quantileGKOrNull" -"quantileGKResample" -"quantileGKSimpleState" -"quantileGKState" -"quantileIf" -"quantileInterpolatedWeighted" -"quantileInterpolatedWeightedArgMax" -"quantileInterpolatedWeightedArgMin" -"quantileInterpolatedWeightedArray" -"quantileInterpolatedWeightedDistinct" -"quantileInterpolatedWeightedForEach" -"quantileInterpolatedWeightedIf" -"quantileInterpolatedWeightedMap" -"quantileInterpolatedWeightedMerge" -"quantileInterpolatedWeightedNull" -"quantileInterpolatedWeightedOrDefault" -"quantileInterpolatedWeightedOrNull" -"quantileInterpolatedWeightedResample" -"quantileInterpolatedWeightedSimpleState" -"quantileInterpolatedWeightedState" -"quantileMap" -"quantileMerge" -"quantileNull" -"quantileOrDefault" -"quantileOrNull" -"quantileResample" -"quantileSimpleState" -"quantileState" -"quantileTDigest" -"quantileTDigestArgMax" -"quantileTDigestArgMin" -"quantileTDigestArray" -"quantileTDigestDistinct" -"quantileTDigestForEach" -"quantileTDigestIf" -"quantileTDigestMap" -"quantileTDigestMerge" -"quantileTDigestNull" -"quantileTDigestOrDefault" -"quantileTDigestOrNull" -"quantileTDigestResample" -"quantileTDigestSimpleState" -"quantileTDigestState" -"quantileTDigestWeighted" -"quantileTDigestWeightedArgMax" -"quantileTDigestWeightedArgMin" -"quantileTDigestWeightedArray" -"quantileTDigestWeightedDistinct" -"quantileTDigestWeightedForEach" -"quantileTDigestWeightedIf" -"quantileTDigestWeightedMap" -"quantileTDigestWeightedMerge" -"quantileTDigestWeightedNull" -"quantileTDigestWeightedOrDefault" -"quantileTDigestWeightedOrNull" -"quantileTDigestWeightedResample" -"quantileTDigestWeightedSimpleState" -"quantileTDigestWeightedState" -"quantileTiming" -"quantileTimingArgMax" -"quantileTimingArgMin" -"quantileTimingArray" -"quantileTimingDistinct" -"quantileTimingForEach" -"quantileTimingIf" -"quantileTimingMap" -"quantileTimingMerge" -"quantileTimingNull" -"quantileTimingOrDefault" -"quantileTimingOrNull" -"quantileTimingResample" -"quantileTimingSimpleState" -"quantileTimingState" -"quantileTimingWeighted" -"quantileTimingWeightedArgMax" -"quantileTimingWeightedArgMin" -"quantileTimingWeightedArray" -"quantileTimingWeightedDistinct" -"quantileTimingWeightedForEach" -"quantileTimingWeightedIf" -"quantileTimingWeightedMap" -"quantileTimingWeightedMerge" -"quantileTimingWeightedNull" -"quantileTimingWeightedOrDefault" -"quantileTimingWeightedOrNull" -"quantileTimingWeightedResample" -"quantileTimingWeightedSimpleState" -"quantileTimingWeightedState" -"quantiles" -"quantilesArgMax" -"quantilesArgMin" -"quantilesArray" -"quantilesBFloat16" -"quantilesBFloat16ArgMax" -"quantilesBFloat16ArgMin" -"quantilesBFloat16Array" -"quantilesBFloat16Distinct" -"quantilesBFloat16ForEach" -"quantilesBFloat16If" -"quantilesBFloat16Map" -"quantilesBFloat16Merge" -"quantilesBFloat16Null" -"quantilesBFloat16OrDefault" -"quantilesBFloat16OrNull" -"quantilesBFloat16Resample" -"quantilesBFloat16SimpleState" -"quantilesBFloat16State" -"quantilesBFloat16Weighted" -"quantilesBFloat16WeightedArgMax" -"quantilesBFloat16WeightedArgMin" -"quantilesBFloat16WeightedArray" -"quantilesBFloat16WeightedDistinct" -"quantilesBFloat16WeightedForEach" -"quantilesBFloat16WeightedIf" -"quantilesBFloat16WeightedMap" -"quantilesBFloat16WeightedMerge" -"quantilesBFloat16WeightedNull" -"quantilesBFloat16WeightedOrDefault" -"quantilesBFloat16WeightedOrNull" -"quantilesBFloat16WeightedResample" -"quantilesBFloat16WeightedSimpleState" -"quantilesBFloat16WeightedState" -"quantilesDD" -"quantilesDDArgMax" -"quantilesDDArgMin" -"quantilesDDArray" -"quantilesDDDistinct" -"quantilesDDForEach" -"quantilesDDIf" -"quantilesDDMap" -"quantilesDDMerge" -"quantilesDDNull" -"quantilesDDOrDefault" -"quantilesDDOrNull" -"quantilesDDResample" -"quantilesDDSimpleState" -"quantilesDDState" -"quantilesDeterministic" -"quantilesDeterministicArgMax" -"quantilesDeterministicArgMin" -"quantilesDeterministicArray" -"quantilesDeterministicDistinct" -"quantilesDeterministicForEach" -"quantilesDeterministicIf" -"quantilesDeterministicMap" -"quantilesDeterministicMerge" -"quantilesDeterministicNull" -"quantilesDeterministicOrDefault" -"quantilesDeterministicOrNull" -"quantilesDeterministicResample" -"quantilesDeterministicSimpleState" -"quantilesDeterministicState" -"quantilesDistinct" -"quantilesExact" -"quantilesExactArgMax" -"quantilesExactArgMin" -"quantilesExactArray" -"quantilesExactDistinct" -"quantilesExactExclusive" -"quantilesExactExclusiveArgMax" -"quantilesExactExclusiveArgMin" -"quantilesExactExclusiveArray" -"quantilesExactExclusiveDistinct" -"quantilesExactExclusiveForEach" -"quantilesExactExclusiveIf" -"quantilesExactExclusiveMap" -"quantilesExactExclusiveMerge" -"quantilesExactExclusiveNull" -"quantilesExactExclusiveOrDefault" -"quantilesExactExclusiveOrNull" -"quantilesExactExclusiveResample" -"quantilesExactExclusiveSimpleState" -"quantilesExactExclusiveState" -"quantilesExactForEach" -"quantilesExactHigh" -"quantilesExactHighArgMax" -"quantilesExactHighArgMin" -"quantilesExactHighArray" -"quantilesExactHighDistinct" -"quantilesExactHighForEach" -"quantilesExactHighIf" -"quantilesExactHighMap" -"quantilesExactHighMerge" -"quantilesExactHighNull" -"quantilesExactHighOrDefault" -"quantilesExactHighOrNull" -"quantilesExactHighResample" -"quantilesExactHighSimpleState" -"quantilesExactHighState" -"quantilesExactIf" -"quantilesExactInclusive" -"quantilesExactInclusiveArgMax" -"quantilesExactInclusiveArgMin" -"quantilesExactInclusiveArray" -"quantilesExactInclusiveDistinct" -"quantilesExactInclusiveForEach" -"quantilesExactInclusiveIf" -"quantilesExactInclusiveMap" -"quantilesExactInclusiveMerge" -"quantilesExactInclusiveNull" -"quantilesExactInclusiveOrDefault" -"quantilesExactInclusiveOrNull" -"quantilesExactInclusiveResample" -"quantilesExactInclusiveSimpleState" -"quantilesExactInclusiveState" -"quantilesExactLow" -"quantilesExactLowArgMax" -"quantilesExactLowArgMin" -"quantilesExactLowArray" -"quantilesExactLowDistinct" -"quantilesExactLowForEach" -"quantilesExactLowIf" -"quantilesExactLowMap" -"quantilesExactLowMerge" -"quantilesExactLowNull" -"quantilesExactLowOrDefault" -"quantilesExactLowOrNull" -"quantilesExactLowResample" -"quantilesExactLowSimpleState" -"quantilesExactLowState" -"quantilesExactMap" -"quantilesExactMerge" -"quantilesExactNull" -"quantilesExactOrDefault" -"quantilesExactOrNull" -"quantilesExactResample" -"quantilesExactSimpleState" -"quantilesExactState" -"quantilesExactWeighted" -"quantilesExactWeightedArgMax" -"quantilesExactWeightedArgMin" -"quantilesExactWeightedArray" -"quantilesExactWeightedDistinct" -"quantilesExactWeightedForEach" -"quantilesExactWeightedIf" -"quantilesExactWeightedMap" -"quantilesExactWeightedMerge" -"quantilesExactWeightedNull" -"quantilesExactWeightedOrDefault" -"quantilesExactWeightedOrNull" -"quantilesExactWeightedResample" -"quantilesExactWeightedSimpleState" -"quantilesExactWeightedState" -"quantilesForEach" -"quantilesGK" -"quantilesGKArgMax" -"quantilesGKArgMin" -"quantilesGKArray" -"quantilesGKDistinct" -"quantilesGKForEach" -"quantilesGKIf" -"quantilesGKMap" -"quantilesGKMerge" -"quantilesGKNull" -"quantilesGKOrDefault" -"quantilesGKOrNull" -"quantilesGKResample" -"quantilesGKSimpleState" -"quantilesGKState" -"quantilesIf" -"quantilesInterpolatedWeighted" -"quantilesInterpolatedWeightedArgMax" -"quantilesInterpolatedWeightedArgMin" -"quantilesInterpolatedWeightedArray" -"quantilesInterpolatedWeightedDistinct" -"quantilesInterpolatedWeightedForEach" -"quantilesInterpolatedWeightedIf" -"quantilesInterpolatedWeightedMap" -"quantilesInterpolatedWeightedMerge" -"quantilesInterpolatedWeightedNull" -"quantilesInterpolatedWeightedOrDefault" -"quantilesInterpolatedWeightedOrNull" -"quantilesInterpolatedWeightedResample" -"quantilesInterpolatedWeightedSimpleState" -"quantilesInterpolatedWeightedState" -"quantilesMap" -"quantilesMerge" -"quantilesNull" -"quantilesOrDefault" -"quantilesOrNull" -"quantilesResample" -"quantilesSimpleState" -"quantilesState" -"quantilesTDigest" -"quantilesTDigestArgMax" -"quantilesTDigestArgMin" -"quantilesTDigestArray" -"quantilesTDigestDistinct" -"quantilesTDigestForEach" -"quantilesTDigestIf" -"quantilesTDigestMap" -"quantilesTDigestMerge" -"quantilesTDigestNull" -"quantilesTDigestOrDefault" -"quantilesTDigestOrNull" -"quantilesTDigestResample" -"quantilesTDigestSimpleState" -"quantilesTDigestState" -"quantilesTDigestWeighted" -"quantilesTDigestWeightedArgMax" -"quantilesTDigestWeightedArgMin" -"quantilesTDigestWeightedArray" -"quantilesTDigestWeightedDistinct" -"quantilesTDigestWeightedForEach" -"quantilesTDigestWeightedIf" -"quantilesTDigestWeightedMap" -"quantilesTDigestWeightedMerge" -"quantilesTDigestWeightedNull" -"quantilesTDigestWeightedOrDefault" -"quantilesTDigestWeightedOrNull" -"quantilesTDigestWeightedResample" -"quantilesTDigestWeightedSimpleState" -"quantilesTDigestWeightedState" -"quantilesTiming" -"quantilesTimingArgMax" -"quantilesTimingArgMin" -"quantilesTimingArray" -"quantilesTimingDistinct" -"quantilesTimingForEach" -"quantilesTimingIf" -"quantilesTimingMap" -"quantilesTimingMerge" -"quantilesTimingNull" -"quantilesTimingOrDefault" -"quantilesTimingOrNull" -"quantilesTimingResample" -"quantilesTimingSimpleState" -"quantilesTimingState" -"quantilesTimingWeighted" -"quantilesTimingWeightedArgMax" -"quantilesTimingWeightedArgMin" -"quantilesTimingWeightedArray" -"quantilesTimingWeightedDistinct" -"quantilesTimingWeightedForEach" -"quantilesTimingWeightedIf" -"quantilesTimingWeightedMap" -"quantilesTimingWeightedMerge" -"quantilesTimingWeightedNull" -"quantilesTimingWeightedOrDefault" -"quantilesTimingWeightedOrNull" -"quantilesTimingWeightedResample" -"quantilesTimingWeightedSimpleState" -"quantilesTimingWeightedState" -"queryID" -"queryString" -"queryStringAndFragment" -"query_id" -"radians" -"rand" -"rand32" -"rand64" -"randBernoulli" -"randBinomial" -"randCanonical" -"randChiSquared" -"randConstant" -"randExponential" -"randFisherF" -"randLogNormal" -"randNegativeBinomial" -"randNormal" -"randPoisson" -"randStudentT" -"randUniform" -"randomFixedString" -"randomPrintableASCII" -"randomString" -"randomStringUTF8" -"range" -"rank" -"rankArgMax" -"rankArgMin" -"rankArray" -"rankCorr" -"rankCorrArgMax" -"rankCorrArgMin" -"rankCorrArray" -"rankCorrDistinct" -"rankCorrForEach" -"rankCorrIf" -"rankCorrMap" -"rankCorrMerge" -"rankCorrNull" -"rankCorrOrDefault" -"rankCorrOrNull" -"rankCorrResample" -"rankCorrSimpleState" -"rankCorrState" -"rankDistinct" -"rankForEach" -"rankIf" -"rankMap" -"rankMerge" -"rankNull" -"rankOrDefault" -"rankOrNull" -"rankResample" -"rankSimpleState" -"rankState" -"readWKTLineString" -"readWKTMultiLineString" -"readWKTMultiPolygon" -"readWKTPoint" -"readWKTPolygon" -"readWKTRing" -"regexpExtract" -"regexpQuoteMeta" -"regionHierarchy" -"regionIn" -"regionToArea" -"regionToCity" -"regionToContinent" -"regionToCountry" -"regionToDistrict" -"regionToName" -"regionToPopulation" -"regionToTopContinent" -"reinterpret" -"reinterpretAsDate" -"reinterpretAsDateTime" -"reinterpretAsFixedString" -"reinterpretAsFloat32" -"reinterpretAsFloat64" -"reinterpretAsInt128" -"reinterpretAsInt16" -"reinterpretAsInt256" -"reinterpretAsInt32" -"reinterpretAsInt64" -"reinterpretAsInt8" -"reinterpretAsString" -"reinterpretAsUInt128" -"reinterpretAsUInt16" -"reinterpretAsUInt256" -"reinterpretAsUInt32" -"reinterpretAsUInt64" -"reinterpretAsUInt8" -"reinterpretAsUUID" -"repeat" -"replace" -"replaceAll" -"replaceOne" -"replaceRegexpAll" -"replaceRegexpOne" -"replicate" -"retention" -"retentionArgMax" -"retentionArgMin" -"retentionArray" -"retentionDistinct" -"retentionForEach" -"retentionIf" -"retentionMap" -"retentionMerge" -"retentionNull" -"retentionOrDefault" -"retentionOrNull" -"retentionResample" -"retentionSimpleState" -"retentionState" -"reverse" -"reverseUTF8" -"revision" -"right" -"rightPad" -"rightPadUTF8" -"rightUTF8" -"round" -"roundAge" -"roundBankers" -"roundDown" -"roundDuration" -"roundToExp2" -"rowNumberInAllBlocks" -"rowNumberInBlock" -"row_number" -"row_numberArgMax" -"row_numberArgMin" -"row_numberArray" -"row_numberDistinct" -"row_numberForEach" -"row_numberIf" -"row_numberMap" -"row_numberMerge" -"row_numberNull" -"row_numberOrDefault" -"row_numberOrNull" -"row_numberResample" -"row_numberSimpleState" -"row_numberState" -"rpad" -"rtrim" -"runningAccumulate" -"runningConcurrency" -"runningDifference" -"runningDifferenceStartingWithFirstValue" -"s2CapContains" -"s2CapUnion" -"s2CellsIntersect" -"s2GetNeighbors" -"s2RectAdd" -"s2RectContains" -"s2RectIntersection" -"s2RectUnion" -"s2ToGeo" -"scalarProduct" -"sequenceCount" -"sequenceCountArgMax" -"sequenceCountArgMin" -"sequenceCountArray" -"sequenceCountDistinct" -"sequenceCountForEach" -"sequenceCountIf" -"sequenceCountMap" -"sequenceCountMerge" -"sequenceCountNull" -"sequenceCountOrDefault" -"sequenceCountOrNull" -"sequenceCountResample" -"sequenceCountSimpleState" -"sequenceCountState" -"sequenceMatch" -"sequenceMatchArgMax" -"sequenceMatchArgMin" -"sequenceMatchArray" -"sequenceMatchDistinct" -"sequenceMatchForEach" -"sequenceMatchIf" -"sequenceMatchMap" -"sequenceMatchMerge" -"sequenceMatchNull" -"sequenceMatchOrDefault" -"sequenceMatchOrNull" -"sequenceMatchResample" -"sequenceMatchSimpleState" -"sequenceMatchState" -"sequenceNextNode" -"sequenceNextNodeArgMax" -"sequenceNextNodeArgMin" -"sequenceNextNodeArray" -"sequenceNextNodeDistinct" -"sequenceNextNodeForEach" -"sequenceNextNodeIf" -"sequenceNextNodeMap" -"sequenceNextNodeMerge" -"sequenceNextNodeNull" -"sequenceNextNodeOrDefault" -"sequenceNextNodeOrNull" -"sequenceNextNodeResample" -"sequenceNextNodeSimpleState" -"sequenceNextNodeState" -"seriesDecomposeSTL" -"seriesOutliersDetectTukey" -"seriesPeriodDetectFFT" -"serverTimeZone" -"serverTimezone" -"serverUUID" -"shardCount" -"shardNum" -"showCertificate" -"sigmoid" -"sign" -"simpleJSONExtractBool" -"simpleJSONExtractFloat" -"simpleJSONExtractInt" -"simpleJSONExtractRaw" -"simpleJSONExtractString" -"simpleJSONExtractUInt" -"simpleJSONHas" -"simpleLinearRegression" -"simpleLinearRegressionArgMax" -"simpleLinearRegressionArgMin" -"simpleLinearRegressionArray" -"simpleLinearRegressionDistinct" -"simpleLinearRegressionForEach" -"simpleLinearRegressionIf" -"simpleLinearRegressionMap" -"simpleLinearRegressionMerge" -"simpleLinearRegressionNull" -"simpleLinearRegressionOrDefault" -"simpleLinearRegressionOrNull" -"simpleLinearRegressionResample" -"simpleLinearRegressionSimpleState" -"simpleLinearRegressionState" -"sin" -"singleValueOrNull" -"singleValueOrNullArgMax" -"singleValueOrNullArgMin" -"singleValueOrNullArray" -"singleValueOrNullDistinct" -"singleValueOrNullForEach" -"singleValueOrNullIf" -"singleValueOrNullMap" -"singleValueOrNullMerge" -"singleValueOrNullNull" -"singleValueOrNullOrDefault" -"singleValueOrNullOrNull" -"singleValueOrNullResample" -"singleValueOrNullSimpleState" -"singleValueOrNullState" -"sinh" -"sipHash128" -"sipHash128Keyed" -"sipHash128Reference" -"sipHash128ReferenceKeyed" -"sipHash64" -"sipHash64Keyed" -"skewPop" -"skewPopArgMax" -"skewPopArgMin" -"skewPopArray" -"skewPopDistinct" -"skewPopForEach" -"skewPopIf" -"skewPopMap" -"skewPopMerge" -"skewPopNull" -"skewPopOrDefault" -"skewPopOrNull" -"skewPopResample" -"skewPopSimpleState" -"skewPopState" -"skewSamp" -"skewSampArgMax" -"skewSampArgMin" -"skewSampArray" -"skewSampDistinct" -"skewSampForEach" -"skewSampIf" -"skewSampMap" -"skewSampMerge" -"skewSampNull" -"skewSampOrDefault" -"skewSampOrNull" -"skewSampResample" -"skewSampSimpleState" -"skewSampState" -"sleep" -"sleepEachRow" -"snowflakeIDToDateTime" -"snowflakeIDToDateTime64" -"snowflakeToDateTime" -"snowflakeToDateTime64" -"soundex" -"space" -"sparkBar" -"sparkBarArgMax" -"sparkBarArgMin" -"sparkBarArray" -"sparkBarDistinct" -"sparkBarForEach" -"sparkBarIf" -"sparkBarMap" -"sparkBarMerge" -"sparkBarNull" -"sparkBarOrDefault" -"sparkBarOrNull" -"sparkBarResample" -"sparkBarSimpleState" -"sparkBarState" -"sparkbar" -"sparkbarArgMax" -"sparkbarArgMin" -"sparkbarArray" -"sparkbarDistinct" -"sparkbarForEach" -"sparkbarIf" -"sparkbarMap" -"sparkbarMerge" -"sparkbarNull" -"sparkbarOrDefault" -"sparkbarOrNull" -"sparkbarResample" -"sparkbarSimpleState" -"sparkbarState" -"splitByAlpha" -"splitByChar" -"splitByNonAlpha" -"splitByRegexp" -"splitByString" -"splitByWhitespace" -"sqid" -"sqidDecode" -"sqidEncode" -"sqrt" -"startsWith" -"startsWithUTF8" -"stddevPop" -"stddevPopArgMax" -"stddevPopArgMin" -"stddevPopArray" -"stddevPopDistinct" -"stddevPopForEach" -"stddevPopIf" -"stddevPopMap" -"stddevPopMerge" -"stddevPopNull" -"stddevPopOrDefault" -"stddevPopOrNull" -"stddevPopResample" -"stddevPopSimpleState" -"stddevPopStable" -"stddevPopStableArgMax" -"stddevPopStableArgMin" -"stddevPopStableArray" -"stddevPopStableDistinct" -"stddevPopStableForEach" -"stddevPopStableIf" -"stddevPopStableMap" -"stddevPopStableMerge" -"stddevPopStableNull" -"stddevPopStableOrDefault" -"stddevPopStableOrNull" -"stddevPopStableResample" -"stddevPopStableSimpleState" -"stddevPopStableState" -"stddevPopState" -"stddevSamp" -"stddevSampArgMax" -"stddevSampArgMin" -"stddevSampArray" -"stddevSampDistinct" -"stddevSampForEach" -"stddevSampIf" -"stddevSampMap" -"stddevSampMerge" -"stddevSampNull" -"stddevSampOrDefault" -"stddevSampOrNull" -"stddevSampResample" -"stddevSampSimpleState" -"stddevSampStable" -"stddevSampStableArgMax" -"stddevSampStableArgMin" -"stddevSampStableArray" -"stddevSampStableDistinct" -"stddevSampStableForEach" -"stddevSampStableIf" -"stddevSampStableMap" -"stddevSampStableMerge" -"stddevSampStableNull" -"stddevSampStableOrDefault" -"stddevSampStableOrNull" -"stddevSampStableResample" -"stddevSampStableSimpleState" -"stddevSampStableState" -"stddevSampState" -"stem" -"stochasticLinearRegression" -"stochasticLinearRegressionArgMax" -"stochasticLinearRegressionArgMin" -"stochasticLinearRegressionArray" -"stochasticLinearRegressionDistinct" -"stochasticLinearRegressionForEach" -"stochasticLinearRegressionIf" -"stochasticLinearRegressionMap" -"stochasticLinearRegressionMerge" -"stochasticLinearRegressionNull" -"stochasticLinearRegressionOrDefault" -"stochasticLinearRegressionOrNull" -"stochasticLinearRegressionResample" -"stochasticLinearRegressionSimpleState" -"stochasticLinearRegressionState" -"stochasticLogisticRegression" -"stochasticLogisticRegressionArgMax" -"stochasticLogisticRegressionArgMin" -"stochasticLogisticRegressionArray" -"stochasticLogisticRegressionDistinct" -"stochasticLogisticRegressionForEach" -"stochasticLogisticRegressionIf" -"stochasticLogisticRegressionMap" -"stochasticLogisticRegressionMerge" -"stochasticLogisticRegressionNull" -"stochasticLogisticRegressionOrDefault" -"stochasticLogisticRegressionOrNull" -"stochasticLogisticRegressionResample" -"stochasticLogisticRegressionSimpleState" -"stochasticLogisticRegressionState" -"str_to_date" -"str_to_map" -"stringJaccardIndex" -"stringJaccardIndexUTF8" -"stringToH3" -"structureToCapnProtoSchema" -"structureToProtobufSchema" -"studentTTest" -"studentTTestArgMax" -"studentTTestArgMin" -"studentTTestArray" -"studentTTestDistinct" -"studentTTestForEach" -"studentTTestIf" -"studentTTestMap" -"studentTTestMerge" -"studentTTestNull" -"studentTTestOrDefault" -"studentTTestOrNull" -"studentTTestResample" -"studentTTestSimpleState" -"studentTTestState" -"subBitmap" -"subDate" -"substr" -"substring" -"substringIndex" -"substringIndexUTF8" -"substringUTF8" -"subtractDays" -"subtractHours" -"subtractInterval" -"subtractMicroseconds" -"subtractMilliseconds" -"subtractMinutes" -"subtractMonths" -"subtractNanoseconds" -"subtractQuarters" -"subtractSeconds" -"subtractTupleOfIntervals" -"subtractWeeks" -"subtractYears" -"sum" -"sumArgMax" -"sumArgMin" -"sumArray" -"sumCount" -"sumCountArgMax" -"sumCountArgMin" -"sumCountArray" -"sumCountDistinct" -"sumCountForEach" -"sumCountIf" -"sumCountMap" -"sumCountMerge" -"sumCountNull" -"sumCountOrDefault" -"sumCountOrNull" -"sumCountResample" -"sumCountSimpleState" -"sumCountState" -"sumDistinct" -"sumForEach" -"sumIf" -"sumKahan" -"sumKahanArgMax" -"sumKahanArgMin" -"sumKahanArray" -"sumKahanDistinct" -"sumKahanForEach" -"sumKahanIf" -"sumKahanMap" -"sumKahanMerge" -"sumKahanNull" -"sumKahanOrDefault" -"sumKahanOrNull" -"sumKahanResample" -"sumKahanSimpleState" -"sumKahanState" -"sumMap" -"sumMapFiltered" -"sumMapFilteredArgMax" -"sumMapFilteredArgMin" -"sumMapFilteredArray" -"sumMapFilteredDistinct" -"sumMapFilteredForEach" -"sumMapFilteredIf" -"sumMapFilteredMap" -"sumMapFilteredMerge" -"sumMapFilteredNull" -"sumMapFilteredOrDefault" -"sumMapFilteredOrNull" -"sumMapFilteredResample" -"sumMapFilteredSimpleState" -"sumMapFilteredState" -"sumMapFilteredWithOverflow" -"sumMapFilteredWithOverflowArgMax" -"sumMapFilteredWithOverflowArgMin" -"sumMapFilteredWithOverflowArray" -"sumMapFilteredWithOverflowDistinct" -"sumMapFilteredWithOverflowForEach" -"sumMapFilteredWithOverflowIf" -"sumMapFilteredWithOverflowMap" -"sumMapFilteredWithOverflowMerge" -"sumMapFilteredWithOverflowNull" -"sumMapFilteredWithOverflowOrDefault" -"sumMapFilteredWithOverflowOrNull" -"sumMapFilteredWithOverflowResample" -"sumMapFilteredWithOverflowSimpleState" -"sumMapFilteredWithOverflowState" -"sumMapWithOverflow" -"sumMapWithOverflowArgMax" -"sumMapWithOverflowArgMin" -"sumMapWithOverflowArray" -"sumMapWithOverflowDistinct" -"sumMapWithOverflowForEach" -"sumMapWithOverflowIf" -"sumMapWithOverflowMap" -"sumMapWithOverflowMerge" -"sumMapWithOverflowNull" -"sumMapWithOverflowOrDefault" -"sumMapWithOverflowOrNull" -"sumMapWithOverflowResample" -"sumMapWithOverflowSimpleState" -"sumMapWithOverflowState" -"sumMappedArrays" -"sumMappedArraysArgMax" -"sumMappedArraysArgMin" -"sumMappedArraysArray" -"sumMappedArraysDistinct" -"sumMappedArraysForEach" -"sumMappedArraysIf" -"sumMappedArraysMap" -"sumMappedArraysMerge" -"sumMappedArraysNull" -"sumMappedArraysOrDefault" -"sumMappedArraysOrNull" -"sumMappedArraysResample" -"sumMappedArraysSimpleState" -"sumMappedArraysState" -"sumMerge" -"sumNull" -"sumOrDefault" -"sumOrNull" -"sumResample" -"sumSimpleState" -"sumState" -"sumWithOverflow" -"sumWithOverflowArgMax" -"sumWithOverflowArgMin" -"sumWithOverflowArray" -"sumWithOverflowDistinct" -"sumWithOverflowForEach" -"sumWithOverflowIf" -"sumWithOverflowMap" -"sumWithOverflowMerge" -"sumWithOverflowNull" -"sumWithOverflowOrDefault" -"sumWithOverflowOrNull" -"sumWithOverflowResample" -"sumWithOverflowSimpleState" -"sumWithOverflowState" -"svg" -"synonyms" -"tan" -"tanh" -"tcpPort" -"tgamma" -"theilsU" -"theilsUArgMax" -"theilsUArgMin" -"theilsUArray" -"theilsUDistinct" -"theilsUForEach" -"theilsUIf" -"theilsUMap" -"theilsUMerge" -"theilsUNull" -"theilsUOrDefault" -"theilsUOrNull" -"theilsUResample" -"theilsUSimpleState" -"theilsUState" -"throwIf" -"tid" -"timeDiff" -"timeSlot" -"timeSlots" -"timeZone" -"timeZoneOf" -"timeZoneOffset" -"timestamp" -"timestampDiff" -"timestamp_diff" -"timezone" -"timezoneOf" -"timezoneOffset" -"toBool" -"toColumnTypeName" -"toDate" -"toDate32" -"toDate32OrDefault" -"toDate32OrNull" -"toDate32OrZero" -"toDateOrDefault" -"toDateOrNull" -"toDateOrZero" -"toDateTime" -"toDateTime32" -"toDateTime64" -"toDateTime64OrDefault" -"toDateTime64OrNull" -"toDateTime64OrZero" -"toDateTimeOrDefault" -"toDateTimeOrNull" -"toDateTimeOrZero" -"toDayOfMonth" -"toDayOfWeek" -"toDayOfYear" -"toDaysSinceYearZero" -"toDecimal128" -"toDecimal128OrDefault" -"toDecimal128OrNull" -"toDecimal128OrZero" -"toDecimal256" -"toDecimal256OrDefault" -"toDecimal256OrNull" -"toDecimal256OrZero" -"toDecimal32" -"toDecimal32OrDefault" -"toDecimal32OrNull" -"toDecimal32OrZero" -"toDecimal64" -"toDecimal64OrDefault" -"toDecimal64OrNull" -"toDecimal64OrZero" -"toDecimalString" -"toFixedString" -"toFloat32" -"toFloat32OrDefault" -"toFloat32OrNull" -"toFloat32OrZero" -"toFloat64" -"toFloat64OrDefault" -"toFloat64OrNull" -"toFloat64OrZero" -"toHour" -"toIPv4" -"toIPv4OrDefault" -"toIPv4OrNull" -"toIPv4OrZero" -"toIPv6" -"toIPv6OrDefault" -"toIPv6OrNull" -"toIPv6OrZero" -"toISOWeek" -"toISOYear" -"toInt128" -"toInt128OrDefault" -"toInt128OrNull" -"toInt128OrZero" -"toInt16" -"toInt16OrDefault" -"toInt16OrNull" -"toInt16OrZero" -"toInt256" -"toInt256OrDefault" -"toInt256OrNull" -"toInt256OrZero" -"toInt32" -"toInt32OrDefault" -"toInt32OrNull" -"toInt32OrZero" -"toInt64" -"toInt64OrDefault" -"toInt64OrNull" -"toInt64OrZero" -"toInt8" -"toInt8OrDefault" -"toInt8OrNull" -"toInt8OrZero" -"toIntervalDay" -"toIntervalHour" -"toIntervalMicrosecond" -"toIntervalMillisecond" -"toIntervalMinute" -"toIntervalMonth" -"toIntervalNanosecond" -"toIntervalQuarter" -"toIntervalSecond" -"toIntervalWeek" -"toIntervalYear" -"toJSONString" -"toLastDayOfMonth" -"toLastDayOfWeek" -"toLowCardinality" -"toMillisecond" -"toMinute" -"toModifiedJulianDay" -"toModifiedJulianDayOrNull" -"toMonday" -"toMonth" -"toNullable" -"toQuarter" -"toRelativeDayNum" -"toRelativeHourNum" -"toRelativeMinuteNum" -"toRelativeMonthNum" -"toRelativeQuarterNum" -"toRelativeSecondNum" -"toRelativeWeekNum" -"toRelativeYearNum" -"toSecond" -"toStartOfDay" -"toStartOfFifteenMinutes" -"toStartOfFiveMinute" -"toStartOfFiveMinutes" -"toStartOfHour" -"toStartOfISOYear" -"toStartOfInterval" -"toStartOfMicrosecond" -"toStartOfMillisecond" -"toStartOfMinute" -"toStartOfMonth" -"toStartOfNanosecond" -"toStartOfQuarter" -"toStartOfSecond" -"toStartOfTenMinutes" -"toStartOfWeek" -"toStartOfYear" -"toString" -"toStringCutToZero" -"toTime" -"toTimeZone" -"toTimezone" -"toTypeName" -"toUInt128" -"toUInt128OrDefault" -"toUInt128OrNull" -"toUInt128OrZero" -"toUInt16" -"toUInt16OrDefault" -"toUInt16OrNull" -"toUInt16OrZero" -"toUInt256" -"toUInt256OrDefault" -"toUInt256OrNull" -"toUInt256OrZero" -"toUInt32" -"toUInt32OrDefault" -"toUInt32OrNull" -"toUInt32OrZero" -"toUInt64" -"toUInt64OrDefault" -"toUInt64OrNull" -"toUInt64OrZero" -"toUInt8" -"toUInt8OrDefault" -"toUInt8OrNull" -"toUInt8OrZero" -"toUTCTimestamp" -"toUUID" -"toUUIDOrDefault" -"toUUIDOrNull" -"toUUIDOrZero" -"toUnixTimestamp" -"toUnixTimestamp64Micro" -"toUnixTimestamp64Milli" -"toUnixTimestamp64Nano" -"toValidUTF8" -"toWeek" -"toYYYYMM" -"toYYYYMMDD" -"toYYYYMMDDhhmmss" -"toYear" -"toYearWeek" -"to_utc_timestamp" -"today" -"tokens" -"topK" -"topKArgMax" -"topKArgMin" -"topKArray" -"topKDistinct" -"topKForEach" -"topKIf" -"topKMap" -"topKMerge" -"topKNull" -"topKOrDefault" -"topKOrNull" -"topKResample" -"topKSimpleState" -"topKState" -"topKWeighted" -"topKWeightedArgMax" -"topKWeightedArgMin" -"topKWeightedArray" -"topKWeightedDistinct" -"topKWeightedForEach" -"topKWeightedIf" -"topKWeightedMap" -"topKWeightedMerge" -"topKWeightedNull" -"topKWeightedOrDefault" -"topKWeightedOrNull" -"topKWeightedResample" -"topKWeightedSimpleState" -"topKWeightedState" -"topLevelDomain" -"topLevelDomainRFC" -"transactionID" -"transactionLatestSnapshot" -"transactionOldestSnapshot" -"transform" -"translate" -"translateUTF8" -"trim" -"trimBoth" -"trimLeft" -"trimRight" -"trunc" -"truncate" -"tryBase58Decode" -"tryBase64Decode" -"tryBase64URLDecode" -"tryDecrypt" -"tryIdnaEncode" -"tryPunycodeDecode" -"tumble" -"tumbleEnd" -"tumbleStart" -"tuple" -"tupleConcat" -"tupleDivide" -"tupleDivideByNumber" -"tupleElement" -"tupleHammingDistance" -"tupleIntDiv" -"tupleIntDivByNumber" -"tupleIntDivOrZero" -"tupleIntDivOrZeroByNumber" -"tupleMinus" -"tupleModulo" -"tupleModuloByNumber" -"tupleMultiply" -"tupleMultiplyByNumber" -"tupleNames" -"tupleNegate" -"tuplePlus" -"tupleToNameValuePairs" -"ucase" -"unbin" -"unhex" -"uniq" -"uniqArgMax" -"uniqArgMin" -"uniqArray" -"uniqCombined" -"uniqCombined64" -"uniqCombined64ArgMax" -"uniqCombined64ArgMin" -"uniqCombined64Array" -"uniqCombined64Distinct" -"uniqCombined64ForEach" -"uniqCombined64If" -"uniqCombined64Map" -"uniqCombined64Merge" -"uniqCombined64Null" -"uniqCombined64OrDefault" -"uniqCombined64OrNull" -"uniqCombined64Resample" -"uniqCombined64SimpleState" -"uniqCombined64State" -"uniqCombinedArgMax" -"uniqCombinedArgMin" -"uniqCombinedArray" -"uniqCombinedDistinct" -"uniqCombinedForEach" -"uniqCombinedIf" -"uniqCombinedMap" -"uniqCombinedMerge" -"uniqCombinedNull" -"uniqCombinedOrDefault" -"uniqCombinedOrNull" -"uniqCombinedResample" -"uniqCombinedSimpleState" -"uniqCombinedState" -"uniqDistinct" -"uniqExact" -"uniqExactArgMax" -"uniqExactArgMin" -"uniqExactArray" -"uniqExactDistinct" -"uniqExactForEach" -"uniqExactIf" -"uniqExactMap" -"uniqExactMerge" -"uniqExactNull" -"uniqExactOrDefault" -"uniqExactOrNull" -"uniqExactResample" -"uniqExactSimpleState" -"uniqExactState" -"uniqForEach" -"uniqHLL12" -"uniqHLL12ArgMax" -"uniqHLL12ArgMin" -"uniqHLL12Array" -"uniqHLL12Distinct" -"uniqHLL12ForEach" -"uniqHLL12If" -"uniqHLL12Map" -"uniqHLL12Merge" -"uniqHLL12Null" -"uniqHLL12OrDefault" -"uniqHLL12OrNull" -"uniqHLL12Resample" -"uniqHLL12SimpleState" -"uniqHLL12State" -"uniqIf" -"uniqMap" -"uniqMerge" -"uniqNull" -"uniqOrDefault" -"uniqOrNull" -"uniqResample" -"uniqSimpleState" -"uniqState" -"uniqTheta" -"uniqThetaArgMax" -"uniqThetaArgMin" -"uniqThetaArray" -"uniqThetaDistinct" -"uniqThetaForEach" -"uniqThetaIf" -"uniqThetaIntersect" -"uniqThetaMap" -"uniqThetaMerge" -"uniqThetaNot" -"uniqThetaNull" -"uniqThetaOrDefault" -"uniqThetaOrNull" -"uniqThetaResample" -"uniqThetaSimpleState" -"uniqThetaState" -"uniqThetaUnion" -"uniqUpTo" -"uniqUpToArgMax" -"uniqUpToArgMin" -"uniqUpToArray" -"uniqUpToDistinct" -"uniqUpToForEach" -"uniqUpToIf" -"uniqUpToMap" -"uniqUpToMerge" -"uniqUpToNull" -"uniqUpToOrDefault" -"uniqUpToOrNull" -"uniqUpToResample" -"uniqUpToSimpleState" -"uniqUpToState" -"upper" -"upperUTF8" -"uptime" -"user" -"validateNestedArraySizes" -"varPop" -"varPopArgMax" -"varPopArgMin" -"varPopArray" -"varPopDistinct" -"varPopForEach" -"varPopIf" -"varPopMap" -"varPopMerge" -"varPopNull" -"varPopOrDefault" -"varPopOrNull" -"varPopResample" -"varPopSimpleState" -"varPopStable" -"varPopStableArgMax" -"varPopStableArgMin" -"varPopStableArray" -"varPopStableDistinct" -"varPopStableForEach" -"varPopStableIf" -"varPopStableMap" -"varPopStableMerge" -"varPopStableNull" -"varPopStableOrDefault" -"varPopStableOrNull" -"varPopStableResample" -"varPopStableSimpleState" -"varPopStableState" -"varPopState" -"varSamp" -"varSampArgMax" -"varSampArgMin" -"varSampArray" -"varSampDistinct" -"varSampForEach" -"varSampIf" -"varSampMap" -"varSampMerge" -"varSampNull" -"varSampOrDefault" -"varSampOrNull" -"varSampResample" -"varSampSimpleState" -"varSampStable" -"varSampStableArgMax" -"varSampStableArgMin" -"varSampStableArray" -"varSampStableDistinct" -"varSampStableForEach" -"varSampStableIf" -"varSampStableMap" -"varSampStableMerge" -"varSampStableNull" -"varSampStableOrDefault" -"varSampStableOrNull" -"varSampStableResample" -"varSampStableSimpleState" -"varSampStableState" -"varSampState" -"variantElement" -"variantType" -"vectorDifference" -"vectorSum" -"version" -"visibleWidth" -"visitParamExtractBool" -"visitParamExtractFloat" -"visitParamExtractInt" -"visitParamExtractRaw" -"visitParamExtractString" -"visitParamExtractUInt" -"visitParamHas" -"week" -"welchTTest" -"welchTTestArgMax" -"welchTTestArgMin" -"welchTTestArray" -"welchTTestDistinct" -"welchTTestForEach" -"welchTTestIf" -"welchTTestMap" -"welchTTestMerge" -"welchTTestNull" -"welchTTestOrDefault" -"welchTTestOrNull" -"welchTTestResample" -"welchTTestSimpleState" -"welchTTestState" -"widthBucket" -"width_bucket" -"windowFunnel" -"windowFunnelArgMax" -"windowFunnelArgMin" -"windowFunnelArray" -"windowFunnelDistinct" -"windowFunnelForEach" -"windowFunnelIf" -"windowFunnelMap" -"windowFunnelMerge" -"windowFunnelNull" -"windowFunnelOrDefault" -"windowFunnelOrNull" -"windowFunnelResample" -"windowFunnelSimpleState" -"windowFunnelState" -"windowID" -"wkt" -"wordShingleMinHash" -"wordShingleMinHashArg" -"wordShingleMinHashArgCaseInsensitive" -"wordShingleMinHashArgCaseInsensitiveUTF8" -"wordShingleMinHashArgUTF8" -"wordShingleMinHashCaseInsensitive" -"wordShingleMinHashCaseInsensitiveUTF8" -"wordShingleMinHashUTF8" -"wordShingleSimHash" -"wordShingleSimHashCaseInsensitive" -"wordShingleSimHashCaseInsensitiveUTF8" -"wordShingleSimHashUTF8" -"wyHash64" -"xor" -"xxHash32" -"xxHash64" -"xxh3" -"yandexConsistentHash" -"yearweek" -"yesterday" -"zookeeperSessionUptime" diff --git a/tests/fuzz/update_dict.sh b/tests/fuzz/update_dict.sh new file mode 100755 index 00000000000..a83c9167129 --- /dev/null +++ b/tests/fuzz/update_dict.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +set -euo pipefail + +SCRIPT_DIR=$(dirname "$(realpath "$0")") +ROOT_PATH="$(git rev-parse --show-toplevel)" +CLICKHOUSE_BIN="${CLICKHOUSE_BIN:-$ROOT_PATH/build/programs/clickhouse}" +DICTIONARIES_DIR="$SCRIPT_DIR/dictionaries" + +echo "Generating functions dict" +$CLICKHOUSE_BIN local -q "SELECT * FROM (SELECT DISTINCT concat('\"', name, '\"') as res FROM system.functions ORDER BY name UNION ALL SELECT concat('\"', a.name, b.name, '\"') as res FROM system.functions as a CROSS JOIN system.aggregate_function_combinators as b WHERE a.is_aggregate = 1) ORDER BY res" > "$DICTIONARIES_DIR/functions.dict" + +echo "Generating data types dict" +$CLICKHOUSE_BIN local -q "SELECT DISTINCT concat('\"', name, '\"') as res FROM system.data_type_families ORDER BY name" > "$DICTIONARIES_DIR/datatypes.dict" + +echo "Generating keywords dict" +$CLICKHOUSE_BIN local -q "SELECT DISTINCT concat('\"', keyword, '\"') as res FROM system.keywords ORDER BY keyword" > "$DICTIONARIES_DIR/keywords.dict" + +echo "Merging dictionaries into all.dict" +cat "$DICTIONARIES_DIR"/* | LC_ALL=C sort | uniq > "$SCRIPT_DIR/all.dict" \ No newline at end of file From 69893aaa25e3c459f3480955534e05456e7aaa64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Aug 2024 16:19:25 +0200 Subject: [PATCH 0887/1722] Lower memory usage --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c70cbe1fe45..874095e39dc 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -391,8 +391,8 @@ done # wait for minio to flush its batch if it has any sleep 1 clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE" -clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" -clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" +clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" +clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" # Stop server so we can safely read data with clickhouse-local. # Why do we read data with clickhouse-local? From 20e20b97c9bddf3a88b0ba8e3a388fe54496fcb2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 16:20:27 +0200 Subject: [PATCH 0888/1722] Fix test storage_join_direct_join --- tests/performance/storage_join_direct_join.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index 867108ac2b7..0e67abb275e 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -16,4 +16,7 @@ SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 + + DROP TABLE IF EXISTS keys + DROP TABLE IF EXISTS dict From 0593650565d717c773fec265056256de98f86f7f Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 13 Aug 2024 14:30:42 +0000 Subject: [PATCH 0889/1722] Update dict files Previous commit had the files from the script ran from the root directory, which returned an error. --- tests/fuzz/dictionaries/functions.dict | 4283 ++++++++++++++++++++++++ 1 file changed, 4283 insertions(+) diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index e69de29bb2d..e562595fb67 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -0,0 +1,4283 @@ +"BIT_AND" +"BIT_ANDArgMax" +"BIT_ANDArgMin" +"BIT_ANDArray" +"BIT_ANDDistinct" +"BIT_ANDForEach" +"BIT_ANDIf" +"BIT_ANDMap" +"BIT_ANDMerge" +"BIT_ANDNull" +"BIT_ANDOrDefault" +"BIT_ANDOrNull" +"BIT_ANDResample" +"BIT_ANDSimpleState" +"BIT_ANDState" +"BIT_OR" +"BIT_ORArgMax" +"BIT_ORArgMin" +"BIT_ORArray" +"BIT_ORDistinct" +"BIT_ORForEach" +"BIT_ORIf" +"BIT_ORMap" +"BIT_ORMerge" +"BIT_ORNull" +"BIT_OROrDefault" +"BIT_OROrNull" +"BIT_ORResample" +"BIT_ORSimpleState" +"BIT_ORState" +"BIT_XOR" +"BIT_XORArgMax" +"BIT_XORArgMin" +"BIT_XORArray" +"BIT_XORDistinct" +"BIT_XORForEach" +"BIT_XORIf" +"BIT_XORMap" +"BIT_XORMerge" +"BIT_XORNull" +"BIT_XOROrDefault" +"BIT_XOROrNull" +"BIT_XORResample" +"BIT_XORSimpleState" +"BIT_XORState" +"BLAKE3" +"CAST" +"CHARACTER_LENGTH" +"CHAR_LENGTH" +"COVAR_POP" +"COVAR_POPArgMax" +"COVAR_POPArgMin" +"COVAR_POPArray" +"COVAR_POPDistinct" +"COVAR_POPForEach" +"COVAR_POPIf" +"COVAR_POPMap" +"COVAR_POPMerge" +"COVAR_POPNull" +"COVAR_POPOrDefault" +"COVAR_POPOrNull" +"COVAR_POPResample" +"COVAR_POPSimpleState" +"COVAR_POPState" +"COVAR_SAMP" +"COVAR_SAMPArgMax" +"COVAR_SAMPArgMin" +"COVAR_SAMPArray" +"COVAR_SAMPDistinct" +"COVAR_SAMPForEach" +"COVAR_SAMPIf" +"COVAR_SAMPMap" +"COVAR_SAMPMerge" +"COVAR_SAMPNull" +"COVAR_SAMPOrDefault" +"COVAR_SAMPOrNull" +"COVAR_SAMPResample" +"COVAR_SAMPSimpleState" +"COVAR_SAMPState" +"CRC32" +"CRC32IEEE" +"CRC64" +"DATABASE" +"DATE" +"DATE_DIFF" +"DATE_FORMAT" +"DATE_TRUNC" +"DAY" +"DAYOFMONTH" +"DAYOFWEEK" +"DAYOFYEAR" +"FORMAT_BYTES" +"FQDN" +"FROM_BASE64" +"FROM_DAYS" +"FROM_UNIXTIME" +"HOUR" +"INET6_ATON" +"INET6_NTOA" +"INET_ATON" +"INET_NTOA" +"IPv4CIDRToRange" +"IPv4NumToString" +"IPv4NumToStringClassC" +"IPv4StringToNum" +"IPv4StringToNumOrDefault" +"IPv4StringToNumOrNull" +"IPv4ToIPv6" +"IPv6CIDRToRange" +"IPv6NumToString" +"IPv6StringToNum" +"IPv6StringToNumOrDefault" +"IPv6StringToNumOrNull" +"JSONArrayLength" +"JSONExtract" +"JSONExtractArrayRaw" +"JSONExtractBool" +"JSONExtractFloat" +"JSONExtractInt" +"JSONExtractKeys" +"JSONExtractKeysAndValues" +"JSONExtractKeysAndValuesRaw" +"JSONExtractRaw" +"JSONExtractString" +"JSONExtractUInt" +"JSONHas" +"JSONKey" +"JSONLength" +"JSONMergePatch" +"JSONType" +"JSON_ARRAY_LENGTH" +"JSON_EXISTS" +"JSON_QUERY" +"JSON_VALUE" +"L1Distance" +"L1Norm" +"L1Normalize" +"L2Distance" +"L2Norm" +"L2Normalize" +"L2SquaredDistance" +"L2SquaredNorm" +"LAST_DAY" +"LinfDistance" +"LinfNorm" +"LinfNormalize" +"LpDistance" +"LpNorm" +"LpNormalize" +"MACNumToString" +"MACStringToNum" +"MACStringToOUI" +"MAP_FROM_ARRAYS" +"MD4" +"MD5" +"MILLISECOND" +"MINUTE" +"MONTH" +"OCTET_LENGTH" +"QUARTER" +"REGEXP_EXTRACT" +"REGEXP_MATCHES" +"REGEXP_REPLACE" +"SCHEMA" +"SECOND" +"SHA1" +"SHA224" +"SHA256" +"SHA384" +"SHA512" +"SHA512_256" +"STD" +"STDArgMax" +"STDArgMin" +"STDArray" +"STDDEV_POP" +"STDDEV_POPArgMax" +"STDDEV_POPArgMin" +"STDDEV_POPArray" +"STDDEV_POPDistinct" +"STDDEV_POPForEach" +"STDDEV_POPIf" +"STDDEV_POPMap" +"STDDEV_POPMerge" +"STDDEV_POPNull" +"STDDEV_POPOrDefault" +"STDDEV_POPOrNull" +"STDDEV_POPResample" +"STDDEV_POPSimpleState" +"STDDEV_POPState" +"STDDEV_SAMP" +"STDDEV_SAMPArgMax" +"STDDEV_SAMPArgMin" +"STDDEV_SAMPArray" +"STDDEV_SAMPDistinct" +"STDDEV_SAMPForEach" +"STDDEV_SAMPIf" +"STDDEV_SAMPMap" +"STDDEV_SAMPMerge" +"STDDEV_SAMPNull" +"STDDEV_SAMPOrDefault" +"STDDEV_SAMPOrNull" +"STDDEV_SAMPResample" +"STDDEV_SAMPSimpleState" +"STDDEV_SAMPState" +"STDDistinct" +"STDForEach" +"STDIf" +"STDMap" +"STDMerge" +"STDNull" +"STDOrDefault" +"STDOrNull" +"STDResample" +"STDSimpleState" +"STDState" +"SUBSTRING_INDEX" +"SVG" +"TIMESTAMP_DIFF" +"TO_BASE64" +"TO_DAYS" +"TO_UNIXTIME" +"ULIDStringToDateTime" +"URLHash" +"URLHierarchy" +"URLPathHierarchy" +"UTCTimestamp" +"UTC_timestamp" +"UUIDNumToString" +"UUIDStringToNum" +"UUIDToNum" +"UUIDv7ToDateTime" +"VAR_POP" +"VAR_POPArgMax" +"VAR_POPArgMin" +"VAR_POPArray" +"VAR_POPDistinct" +"VAR_POPForEach" +"VAR_POPIf" +"VAR_POPMap" +"VAR_POPMerge" +"VAR_POPNull" +"VAR_POPOrDefault" +"VAR_POPOrNull" +"VAR_POPResample" +"VAR_POPSimpleState" +"VAR_POPState" +"VAR_SAMP" +"VAR_SAMPArgMax" +"VAR_SAMPArgMin" +"VAR_SAMPArray" +"VAR_SAMPDistinct" +"VAR_SAMPForEach" +"VAR_SAMPIf" +"VAR_SAMPMap" +"VAR_SAMPMerge" +"VAR_SAMPNull" +"VAR_SAMPOrDefault" +"VAR_SAMPOrNull" +"VAR_SAMPResample" +"VAR_SAMPSimpleState" +"VAR_SAMPState" +"YEAR" +"YYYYMMDDToDate" +"YYYYMMDDToDate32" +"YYYYMMDDhhmmssToDateTime" +"YYYYMMDDhhmmssToDateTime64" +"_CAST" +"__actionName" +"__bitBoolMaskAnd" +"__bitBoolMaskOr" +"__bitSwapLastTwo" +"__bitWrapperFunc" +"__getScalar" +"__scalarSubqueryResult" +"abs" +"accurateCast" +"accurateCastOrDefault" +"accurateCastOrNull" +"acos" +"acosh" +"addDate" +"addDays" +"addHours" +"addInterval" +"addMicroseconds" +"addMilliseconds" +"addMinutes" +"addMonths" +"addNanoseconds" +"addQuarters" +"addSeconds" +"addTupleOfIntervals" +"addWeeks" +"addYears" +"addressToLine" +"addressToLineWithInlines" +"addressToSymbol" +"aes_decrypt_mysql" +"aes_encrypt_mysql" +"age" +"aggThrow" +"aggThrowArgMax" +"aggThrowArgMin" +"aggThrowArray" +"aggThrowDistinct" +"aggThrowForEach" +"aggThrowIf" +"aggThrowMap" +"aggThrowMerge" +"aggThrowNull" +"aggThrowOrDefault" +"aggThrowOrNull" +"aggThrowResample" +"aggThrowSimpleState" +"aggThrowState" +"alphaTokens" +"analysisOfVariance" +"analysisOfVarianceArgMax" +"analysisOfVarianceArgMin" +"analysisOfVarianceArray" +"analysisOfVarianceDistinct" +"analysisOfVarianceForEach" +"analysisOfVarianceIf" +"analysisOfVarianceMap" +"analysisOfVarianceMerge" +"analysisOfVarianceNull" +"analysisOfVarianceOrDefault" +"analysisOfVarianceOrNull" +"analysisOfVarianceResample" +"analysisOfVarianceSimpleState" +"analysisOfVarianceState" +"and" +"anova" +"anovaArgMax" +"anovaArgMin" +"anovaArray" +"anovaDistinct" +"anovaForEach" +"anovaIf" +"anovaMap" +"anovaMerge" +"anovaNull" +"anovaOrDefault" +"anovaOrNull" +"anovaResample" +"anovaSimpleState" +"anovaState" +"any" +"anyArgMax" +"anyArgMin" +"anyArray" +"anyDistinct" +"anyForEach" +"anyHeavy" +"anyHeavyArgMax" +"anyHeavyArgMin" +"anyHeavyArray" +"anyHeavyDistinct" +"anyHeavyForEach" +"anyHeavyIf" +"anyHeavyMap" +"anyHeavyMerge" +"anyHeavyNull" +"anyHeavyOrDefault" +"anyHeavyOrNull" +"anyHeavyResample" +"anyHeavySimpleState" +"anyHeavyState" +"anyIf" +"anyLast" +"anyLastArgMax" +"anyLastArgMin" +"anyLastArray" +"anyLastDistinct" +"anyLastForEach" +"anyLastIf" +"anyLastMap" +"anyLastMerge" +"anyLastNull" +"anyLastOrDefault" +"anyLastOrNull" +"anyLastResample" +"anyLastSimpleState" +"anyLastState" +"anyLast_respect_nulls" +"anyLast_respect_nullsArgMax" +"anyLast_respect_nullsArgMin" +"anyLast_respect_nullsArray" +"anyLast_respect_nullsDistinct" +"anyLast_respect_nullsForEach" +"anyLast_respect_nullsIf" +"anyLast_respect_nullsMap" +"anyLast_respect_nullsMerge" +"anyLast_respect_nullsNull" +"anyLast_respect_nullsOrDefault" +"anyLast_respect_nullsOrNull" +"anyLast_respect_nullsResample" +"anyLast_respect_nullsSimpleState" +"anyLast_respect_nullsState" +"anyMap" +"anyMerge" +"anyNull" +"anyOrDefault" +"anyOrNull" +"anyResample" +"anySimpleState" +"anyState" +"any_respect_nulls" +"any_respect_nullsArgMax" +"any_respect_nullsArgMin" +"any_respect_nullsArray" +"any_respect_nullsDistinct" +"any_respect_nullsForEach" +"any_respect_nullsIf" +"any_respect_nullsMap" +"any_respect_nullsMerge" +"any_respect_nullsNull" +"any_respect_nullsOrDefault" +"any_respect_nullsOrNull" +"any_respect_nullsResample" +"any_respect_nullsSimpleState" +"any_respect_nullsState" +"any_value" +"any_valueArgMax" +"any_valueArgMin" +"any_valueArray" +"any_valueDistinct" +"any_valueForEach" +"any_valueIf" +"any_valueMap" +"any_valueMerge" +"any_valueNull" +"any_valueOrDefault" +"any_valueOrNull" +"any_valueResample" +"any_valueSimpleState" +"any_valueState" +"any_value_respect_nulls" +"any_value_respect_nullsArgMax" +"any_value_respect_nullsArgMin" +"any_value_respect_nullsArray" +"any_value_respect_nullsDistinct" +"any_value_respect_nullsForEach" +"any_value_respect_nullsIf" +"any_value_respect_nullsMap" +"any_value_respect_nullsMerge" +"any_value_respect_nullsNull" +"any_value_respect_nullsOrDefault" +"any_value_respect_nullsOrNull" +"any_value_respect_nullsResample" +"any_value_respect_nullsSimpleState" +"any_value_respect_nullsState" +"appendTrailingCharIfAbsent" +"approx_top_count" +"approx_top_countArgMax" +"approx_top_countArgMin" +"approx_top_countArray" +"approx_top_countDistinct" +"approx_top_countForEach" +"approx_top_countIf" +"approx_top_countMap" +"approx_top_countMerge" +"approx_top_countNull" +"approx_top_countOrDefault" +"approx_top_countOrNull" +"approx_top_countResample" +"approx_top_countSimpleState" +"approx_top_countState" +"approx_top_k" +"approx_top_kArgMax" +"approx_top_kArgMin" +"approx_top_kArray" +"approx_top_kDistinct" +"approx_top_kForEach" +"approx_top_kIf" +"approx_top_kMap" +"approx_top_kMerge" +"approx_top_kNull" +"approx_top_kOrDefault" +"approx_top_kOrNull" +"approx_top_kResample" +"approx_top_kSimpleState" +"approx_top_kState" +"approx_top_sum" +"approx_top_sumArgMax" +"approx_top_sumArgMin" +"approx_top_sumArray" +"approx_top_sumDistinct" +"approx_top_sumForEach" +"approx_top_sumIf" +"approx_top_sumMap" +"approx_top_sumMerge" +"approx_top_sumNull" +"approx_top_sumOrDefault" +"approx_top_sumOrNull" +"approx_top_sumResample" +"approx_top_sumSimpleState" +"approx_top_sumState" +"argMax" +"argMaxArgMax" +"argMaxArgMin" +"argMaxArray" +"argMaxDistinct" +"argMaxForEach" +"argMaxIf" +"argMaxMap" +"argMaxMerge" +"argMaxNull" +"argMaxOrDefault" +"argMaxOrNull" +"argMaxResample" +"argMaxSimpleState" +"argMaxState" +"argMin" +"argMinArgMax" +"argMinArgMin" +"argMinArray" +"argMinDistinct" +"argMinForEach" +"argMinIf" +"argMinMap" +"argMinMerge" +"argMinNull" +"argMinOrDefault" +"argMinOrNull" +"argMinResample" +"argMinSimpleState" +"argMinState" +"array" +"arrayAUC" +"arrayAll" +"arrayAvg" +"arrayCompact" +"arrayConcat" +"arrayCount" +"arrayCumSum" +"arrayCumSumNonNegative" +"arrayDifference" +"arrayDistinct" +"arrayDotProduct" +"arrayElement" +"arrayEnumerate" +"arrayEnumerateDense" +"arrayEnumerateDenseRanked" +"arrayEnumerateUniq" +"arrayEnumerateUniqRanked" +"arrayExists" +"arrayFill" +"arrayFilter" +"arrayFirst" +"arrayFirstIndex" +"arrayFirstOrNull" +"arrayFlatten" +"arrayFold" +"arrayIntersect" +"arrayJaccardIndex" +"arrayJoin" +"arrayLast" +"arrayLastIndex" +"arrayLastOrNull" +"arrayMap" +"arrayMax" +"arrayMin" +"arrayPartialReverseSort" +"arrayPartialShuffle" +"arrayPartialSort" +"arrayPopBack" +"arrayPopFront" +"arrayProduct" +"arrayPushBack" +"arrayPushFront" +"arrayRandomSample" +"arrayReduce" +"arrayReduceInRanges" +"arrayResize" +"arrayReverse" +"arrayReverseFill" +"arrayReverseSort" +"arrayReverseSplit" +"arrayRotateLeft" +"arrayRotateRight" +"arrayShiftLeft" +"arrayShiftRight" +"arrayShingles" +"arrayShuffle" +"arraySlice" +"arraySort" +"arraySplit" +"arrayStringConcat" +"arraySum" +"arrayUniq" +"arrayWithConstant" +"arrayZip" +"array_agg" +"array_aggArgMax" +"array_aggArgMin" +"array_aggArray" +"array_aggDistinct" +"array_aggForEach" +"array_aggIf" +"array_aggMap" +"array_aggMerge" +"array_aggNull" +"array_aggOrDefault" +"array_aggOrNull" +"array_aggResample" +"array_aggSimpleState" +"array_aggState" +"array_concat_agg" +"array_concat_aggArgMax" +"array_concat_aggArgMin" +"array_concat_aggArray" +"array_concat_aggDistinct" +"array_concat_aggForEach" +"array_concat_aggIf" +"array_concat_aggMap" +"array_concat_aggMerge" +"array_concat_aggNull" +"array_concat_aggOrDefault" +"array_concat_aggOrNull" +"array_concat_aggResample" +"array_concat_aggSimpleState" +"array_concat_aggState" +"ascii" +"asin" +"asinh" +"assumeNotNull" +"atan" +"atan2" +"atanh" +"avg" +"avgArgMax" +"avgArgMin" +"avgArray" +"avgDistinct" +"avgForEach" +"avgIf" +"avgMap" +"avgMerge" +"avgNull" +"avgOrDefault" +"avgOrNull" +"avgResample" +"avgSimpleState" +"avgState" +"avgWeighted" +"avgWeightedArgMax" +"avgWeightedArgMin" +"avgWeightedArray" +"avgWeightedDistinct" +"avgWeightedForEach" +"avgWeightedIf" +"avgWeightedMap" +"avgWeightedMerge" +"avgWeightedNull" +"avgWeightedOrDefault" +"avgWeightedOrNull" +"avgWeightedResample" +"avgWeightedSimpleState" +"avgWeightedState" +"bar" +"base58Decode" +"base58Encode" +"base64Decode" +"base64Encode" +"base64URLDecode" +"base64URLEncode" +"basename" +"bin" +"bitAnd" +"bitCount" +"bitHammingDistance" +"bitNot" +"bitOr" +"bitPositionsToArray" +"bitRotateLeft" +"bitRotateRight" +"bitShiftLeft" +"bitShiftRight" +"bitSlice" +"bitTest" +"bitTestAll" +"bitTestAny" +"bitXor" +"bitmapAnd" +"bitmapAndCardinality" +"bitmapAndnot" +"bitmapAndnotCardinality" +"bitmapBuild" +"bitmapCardinality" +"bitmapContains" +"bitmapHasAll" +"bitmapHasAny" +"bitmapMax" +"bitmapMin" +"bitmapOr" +"bitmapOrCardinality" +"bitmapSubsetInRange" +"bitmapSubsetLimit" +"bitmapToArray" +"bitmapTransform" +"bitmapXor" +"bitmapXorCardinality" +"bitmaskToArray" +"bitmaskToList" +"blockNumber" +"blockSerializedSize" +"blockSize" +"boundingRatio" +"boundingRatioArgMax" +"boundingRatioArgMin" +"boundingRatioArray" +"boundingRatioDistinct" +"boundingRatioForEach" +"boundingRatioIf" +"boundingRatioMap" +"boundingRatioMerge" +"boundingRatioNull" +"boundingRatioOrDefault" +"boundingRatioOrNull" +"boundingRatioResample" +"boundingRatioSimpleState" +"boundingRatioState" +"buildId" +"byteHammingDistance" +"byteSize" +"byteSlice" +"byteSwap" +"caseWithExpr" +"caseWithExpression" +"caseWithoutExpr" +"caseWithoutExpression" +"catboostEvaluate" +"categoricalInformationValue" +"categoricalInformationValueArgMax" +"categoricalInformationValueArgMin" +"categoricalInformationValueArray" +"categoricalInformationValueDistinct" +"categoricalInformationValueForEach" +"categoricalInformationValueIf" +"categoricalInformationValueMap" +"categoricalInformationValueMerge" +"categoricalInformationValueNull" +"categoricalInformationValueOrDefault" +"categoricalInformationValueOrNull" +"categoricalInformationValueResample" +"categoricalInformationValueSimpleState" +"categoricalInformationValueState" +"cbrt" +"ceil" +"ceiling" +"changeDay" +"changeHour" +"changeMinute" +"changeMonth" +"changeSecond" +"changeYear" +"char" +"cityHash64" +"clamp" +"coalesce" +"concat" +"concatAssumeInjective" +"concatWithSeparator" +"concatWithSeparatorAssumeInjective" +"concat_ws" +"connectionId" +"connection_id" +"contingency" +"contingencyArgMax" +"contingencyArgMin" +"contingencyArray" +"contingencyDistinct" +"contingencyForEach" +"contingencyIf" +"contingencyMap" +"contingencyMerge" +"contingencyNull" +"contingencyOrDefault" +"contingencyOrNull" +"contingencyResample" +"contingencySimpleState" +"contingencyState" +"convertCharset" +"corr" +"corrArgMax" +"corrArgMin" +"corrArray" +"corrDistinct" +"corrForEach" +"corrIf" +"corrMap" +"corrMatrix" +"corrMatrixArgMax" +"corrMatrixArgMin" +"corrMatrixArray" +"corrMatrixDistinct" +"corrMatrixForEach" +"corrMatrixIf" +"corrMatrixMap" +"corrMatrixMerge" +"corrMatrixNull" +"corrMatrixOrDefault" +"corrMatrixOrNull" +"corrMatrixResample" +"corrMatrixSimpleState" +"corrMatrixState" +"corrMerge" +"corrNull" +"corrOrDefault" +"corrOrNull" +"corrResample" +"corrSimpleState" +"corrStable" +"corrStableArgMax" +"corrStableArgMin" +"corrStableArray" +"corrStableDistinct" +"corrStableForEach" +"corrStableIf" +"corrStableMap" +"corrStableMerge" +"corrStableNull" +"corrStableOrDefault" +"corrStableOrNull" +"corrStableResample" +"corrStableSimpleState" +"corrStableState" +"corrState" +"cos" +"cosh" +"cosineDistance" +"count" +"countArgMax" +"countArgMin" +"countArray" +"countDigits" +"countDistinct" +"countEqual" +"countForEach" +"countIf" +"countMap" +"countMatches" +"countMatchesCaseInsensitive" +"countMerge" +"countNull" +"countOrDefault" +"countOrNull" +"countResample" +"countSimpleState" +"countState" +"countSubstrings" +"countSubstringsCaseInsensitive" +"countSubstringsCaseInsensitiveUTF8" +"covarPop" +"covarPopArgMax" +"covarPopArgMin" +"covarPopArray" +"covarPopDistinct" +"covarPopForEach" +"covarPopIf" +"covarPopMap" +"covarPopMatrix" +"covarPopMatrixArgMax" +"covarPopMatrixArgMin" +"covarPopMatrixArray" +"covarPopMatrixDistinct" +"covarPopMatrixForEach" +"covarPopMatrixIf" +"covarPopMatrixMap" +"covarPopMatrixMerge" +"covarPopMatrixNull" +"covarPopMatrixOrDefault" +"covarPopMatrixOrNull" +"covarPopMatrixResample" +"covarPopMatrixSimpleState" +"covarPopMatrixState" +"covarPopMerge" +"covarPopNull" +"covarPopOrDefault" +"covarPopOrNull" +"covarPopResample" +"covarPopSimpleState" +"covarPopStable" +"covarPopStableArgMax" +"covarPopStableArgMin" +"covarPopStableArray" +"covarPopStableDistinct" +"covarPopStableForEach" +"covarPopStableIf" +"covarPopStableMap" +"covarPopStableMerge" +"covarPopStableNull" +"covarPopStableOrDefault" +"covarPopStableOrNull" +"covarPopStableResample" +"covarPopStableSimpleState" +"covarPopStableState" +"covarPopState" +"covarSamp" +"covarSampArgMax" +"covarSampArgMin" +"covarSampArray" +"covarSampDistinct" +"covarSampForEach" +"covarSampIf" +"covarSampMap" +"covarSampMatrix" +"covarSampMatrixArgMax" +"covarSampMatrixArgMin" +"covarSampMatrixArray" +"covarSampMatrixDistinct" +"covarSampMatrixForEach" +"covarSampMatrixIf" +"covarSampMatrixMap" +"covarSampMatrixMerge" +"covarSampMatrixNull" +"covarSampMatrixOrDefault" +"covarSampMatrixOrNull" +"covarSampMatrixResample" +"covarSampMatrixSimpleState" +"covarSampMatrixState" +"covarSampMerge" +"covarSampNull" +"covarSampOrDefault" +"covarSampOrNull" +"covarSampResample" +"covarSampSimpleState" +"covarSampStable" +"covarSampStableArgMax" +"covarSampStableArgMin" +"covarSampStableArray" +"covarSampStableDistinct" +"covarSampStableForEach" +"covarSampStableIf" +"covarSampStableMap" +"covarSampStableMerge" +"covarSampStableNull" +"covarSampStableOrDefault" +"covarSampStableOrNull" +"covarSampStableResample" +"covarSampStableSimpleState" +"covarSampStableState" +"covarSampState" +"cramersV" +"cramersVArgMax" +"cramersVArgMin" +"cramersVArray" +"cramersVBiasCorrected" +"cramersVBiasCorrectedArgMax" +"cramersVBiasCorrectedArgMin" +"cramersVBiasCorrectedArray" +"cramersVBiasCorrectedDistinct" +"cramersVBiasCorrectedForEach" +"cramersVBiasCorrectedIf" +"cramersVBiasCorrectedMap" +"cramersVBiasCorrectedMerge" +"cramersVBiasCorrectedNull" +"cramersVBiasCorrectedOrDefault" +"cramersVBiasCorrectedOrNull" +"cramersVBiasCorrectedResample" +"cramersVBiasCorrectedSimpleState" +"cramersVBiasCorrectedState" +"cramersVDistinct" +"cramersVForEach" +"cramersVIf" +"cramersVMap" +"cramersVMerge" +"cramersVNull" +"cramersVOrDefault" +"cramersVOrNull" +"cramersVResample" +"cramersVSimpleState" +"cramersVState" +"curdate" +"currentDatabase" +"currentProfiles" +"currentRoles" +"currentSchemas" +"currentUser" +"current_database" +"current_date" +"current_schemas" +"current_timestamp" +"current_user" +"cutFragment" +"cutIPv6" +"cutQueryString" +"cutQueryStringAndFragment" +"cutToFirstSignificantSubdomain" +"cutToFirstSignificantSubdomainCustom" +"cutToFirstSignificantSubdomainCustomRFC" +"cutToFirstSignificantSubdomainCustomWithWWW" +"cutToFirstSignificantSubdomainCustomWithWWWRFC" +"cutToFirstSignificantSubdomainRFC" +"cutToFirstSignificantSubdomainWithWWW" +"cutToFirstSignificantSubdomainWithWWWRFC" +"cutURLParameter" +"cutWWW" +"damerauLevenshteinDistance" +"dateDiff" +"dateName" +"dateTime64ToSnowflake" +"dateTime64ToSnowflakeID" +"dateTimeToSnowflake" +"dateTimeToSnowflakeID" +"dateTrunc" +"date_diff" +"decodeHTMLComponent" +"decodeURLComponent" +"decodeURLFormComponent" +"decodeXMLComponent" +"decrypt" +"defaultProfiles" +"defaultRoles" +"defaultValueOfArgumentType" +"defaultValueOfTypeName" +"degrees" +"deltaSum" +"deltaSumArgMax" +"deltaSumArgMin" +"deltaSumArray" +"deltaSumDistinct" +"deltaSumForEach" +"deltaSumIf" +"deltaSumMap" +"deltaSumMerge" +"deltaSumNull" +"deltaSumOrDefault" +"deltaSumOrNull" +"deltaSumResample" +"deltaSumSimpleState" +"deltaSumState" +"deltaSumTimestamp" +"deltaSumTimestampArgMax" +"deltaSumTimestampArgMin" +"deltaSumTimestampArray" +"deltaSumTimestampDistinct" +"deltaSumTimestampForEach" +"deltaSumTimestampIf" +"deltaSumTimestampMap" +"deltaSumTimestampMerge" +"deltaSumTimestampNull" +"deltaSumTimestampOrDefault" +"deltaSumTimestampOrNull" +"deltaSumTimestampResample" +"deltaSumTimestampSimpleState" +"deltaSumTimestampState" +"demangle" +"denseRank" +"denseRankArgMax" +"denseRankArgMin" +"denseRankArray" +"denseRankDistinct" +"denseRankForEach" +"denseRankIf" +"denseRankMap" +"denseRankMerge" +"denseRankNull" +"denseRankOrDefault" +"denseRankOrNull" +"denseRankResample" +"denseRankSimpleState" +"denseRankState" +"dense_rank" +"dense_rankArgMax" +"dense_rankArgMin" +"dense_rankArray" +"dense_rankDistinct" +"dense_rankForEach" +"dense_rankIf" +"dense_rankMap" +"dense_rankMerge" +"dense_rankNull" +"dense_rankOrDefault" +"dense_rankOrNull" +"dense_rankResample" +"dense_rankSimpleState" +"dense_rankState" +"detectCharset" +"detectLanguage" +"detectLanguageMixed" +"detectLanguageUnknown" +"detectProgrammingLanguage" +"detectTonality" +"dictGet" +"dictGetAll" +"dictGetChildren" +"dictGetDate" +"dictGetDateOrDefault" +"dictGetDateTime" +"dictGetDateTimeOrDefault" +"dictGetDescendants" +"dictGetFloat32" +"dictGetFloat32OrDefault" +"dictGetFloat64" +"dictGetFloat64OrDefault" +"dictGetHierarchy" +"dictGetIPv4" +"dictGetIPv4OrDefault" +"dictGetIPv6" +"dictGetIPv6OrDefault" +"dictGetInt16" +"dictGetInt16OrDefault" +"dictGetInt32" +"dictGetInt32OrDefault" +"dictGetInt64" +"dictGetInt64OrDefault" +"dictGetInt8" +"dictGetInt8OrDefault" +"dictGetOrDefault" +"dictGetOrNull" +"dictGetString" +"dictGetStringOrDefault" +"dictGetUInt16" +"dictGetUInt16OrDefault" +"dictGetUInt32" +"dictGetUInt32OrDefault" +"dictGetUInt64" +"dictGetUInt64OrDefault" +"dictGetUInt8" +"dictGetUInt8OrDefault" +"dictGetUUID" +"dictGetUUIDOrDefault" +"dictHas" +"dictIsIn" +"displayName" +"distanceL1" +"distanceL2" +"distanceL2Squared" +"distanceLinf" +"distanceLp" +"divide" +"divideDecimal" +"domain" +"domainRFC" +"domainWithoutWWW" +"domainWithoutWWWRFC" +"dotProduct" +"dumpColumnStructure" +"dynamicElement" +"dynamicType" +"e" +"editDistance" +"editDistanceUTF8" +"empty" +"emptyArrayDate" +"emptyArrayDateTime" +"emptyArrayFloat32" +"emptyArrayFloat64" +"emptyArrayInt16" +"emptyArrayInt32" +"emptyArrayInt64" +"emptyArrayInt8" +"emptyArrayString" +"emptyArrayToSingle" +"emptyArrayUInt16" +"emptyArrayUInt32" +"emptyArrayUInt64" +"emptyArrayUInt8" +"enabledProfiles" +"enabledRoles" +"encodeURLComponent" +"encodeURLFormComponent" +"encodeXMLComponent" +"encrypt" +"endsWith" +"endsWithUTF8" +"entropy" +"entropyArgMax" +"entropyArgMin" +"entropyArray" +"entropyDistinct" +"entropyForEach" +"entropyIf" +"entropyMap" +"entropyMerge" +"entropyNull" +"entropyOrDefault" +"entropyOrNull" +"entropyResample" +"entropySimpleState" +"entropyState" +"equals" +"erf" +"erfc" +"errorCodeToName" +"evalMLMethod" +"exp" +"exp10" +"exp2" +"exponentialMovingAverage" +"exponentialMovingAverageArgMax" +"exponentialMovingAverageArgMin" +"exponentialMovingAverageArray" +"exponentialMovingAverageDistinct" +"exponentialMovingAverageForEach" +"exponentialMovingAverageIf" +"exponentialMovingAverageMap" +"exponentialMovingAverageMerge" +"exponentialMovingAverageNull" +"exponentialMovingAverageOrDefault" +"exponentialMovingAverageOrNull" +"exponentialMovingAverageResample" +"exponentialMovingAverageSimpleState" +"exponentialMovingAverageState" +"exponentialTimeDecayedAvg" +"exponentialTimeDecayedAvgArgMax" +"exponentialTimeDecayedAvgArgMin" +"exponentialTimeDecayedAvgArray" +"exponentialTimeDecayedAvgDistinct" +"exponentialTimeDecayedAvgForEach" +"exponentialTimeDecayedAvgIf" +"exponentialTimeDecayedAvgMap" +"exponentialTimeDecayedAvgMerge" +"exponentialTimeDecayedAvgNull" +"exponentialTimeDecayedAvgOrDefault" +"exponentialTimeDecayedAvgOrNull" +"exponentialTimeDecayedAvgResample" +"exponentialTimeDecayedAvgSimpleState" +"exponentialTimeDecayedAvgState" +"exponentialTimeDecayedCount" +"exponentialTimeDecayedCountArgMax" +"exponentialTimeDecayedCountArgMin" +"exponentialTimeDecayedCountArray" +"exponentialTimeDecayedCountDistinct" +"exponentialTimeDecayedCountForEach" +"exponentialTimeDecayedCountIf" +"exponentialTimeDecayedCountMap" +"exponentialTimeDecayedCountMerge" +"exponentialTimeDecayedCountNull" +"exponentialTimeDecayedCountOrDefault" +"exponentialTimeDecayedCountOrNull" +"exponentialTimeDecayedCountResample" +"exponentialTimeDecayedCountSimpleState" +"exponentialTimeDecayedCountState" +"exponentialTimeDecayedMax" +"exponentialTimeDecayedMaxArgMax" +"exponentialTimeDecayedMaxArgMin" +"exponentialTimeDecayedMaxArray" +"exponentialTimeDecayedMaxDistinct" +"exponentialTimeDecayedMaxForEach" +"exponentialTimeDecayedMaxIf" +"exponentialTimeDecayedMaxMap" +"exponentialTimeDecayedMaxMerge" +"exponentialTimeDecayedMaxNull" +"exponentialTimeDecayedMaxOrDefault" +"exponentialTimeDecayedMaxOrNull" +"exponentialTimeDecayedMaxResample" +"exponentialTimeDecayedMaxSimpleState" +"exponentialTimeDecayedMaxState" +"exponentialTimeDecayedSum" +"exponentialTimeDecayedSumArgMax" +"exponentialTimeDecayedSumArgMin" +"exponentialTimeDecayedSumArray" +"exponentialTimeDecayedSumDistinct" +"exponentialTimeDecayedSumForEach" +"exponentialTimeDecayedSumIf" +"exponentialTimeDecayedSumMap" +"exponentialTimeDecayedSumMerge" +"exponentialTimeDecayedSumNull" +"exponentialTimeDecayedSumOrDefault" +"exponentialTimeDecayedSumOrNull" +"exponentialTimeDecayedSumResample" +"exponentialTimeDecayedSumSimpleState" +"exponentialTimeDecayedSumState" +"extract" +"extractAll" +"extractAllGroups" +"extractAllGroupsHorizontal" +"extractAllGroupsVertical" +"extractGroups" +"extractKeyValuePairs" +"extractKeyValuePairsWithEscaping" +"extractTextFromHTML" +"extractURLParameter" +"extractURLParameterNames" +"extractURLParameters" +"factorial" +"farmFingerprint64" +"farmHash64" +"file" +"filesystemAvailable" +"filesystemCapacity" +"filesystemUnreserved" +"finalizeAggregation" +"firstLine" +"firstSignificantSubdomain" +"firstSignificantSubdomainCustom" +"firstSignificantSubdomainCustomRFC" +"firstSignificantSubdomainRFC" +"first_value" +"first_valueArgMax" +"first_valueArgMin" +"first_valueArray" +"first_valueDistinct" +"first_valueForEach" +"first_valueIf" +"first_valueMap" +"first_valueMerge" +"first_valueNull" +"first_valueOrDefault" +"first_valueOrNull" +"first_valueResample" +"first_valueSimpleState" +"first_valueState" +"first_value_respect_nulls" +"first_value_respect_nullsArgMax" +"first_value_respect_nullsArgMin" +"first_value_respect_nullsArray" +"first_value_respect_nullsDistinct" +"first_value_respect_nullsForEach" +"first_value_respect_nullsIf" +"first_value_respect_nullsMap" +"first_value_respect_nullsMerge" +"first_value_respect_nullsNull" +"first_value_respect_nullsOrDefault" +"first_value_respect_nullsOrNull" +"first_value_respect_nullsResample" +"first_value_respect_nullsSimpleState" +"first_value_respect_nullsState" +"flameGraph" +"flameGraphArgMax" +"flameGraphArgMin" +"flameGraphArray" +"flameGraphDistinct" +"flameGraphForEach" +"flameGraphIf" +"flameGraphMap" +"flameGraphMerge" +"flameGraphNull" +"flameGraphOrDefault" +"flameGraphOrNull" +"flameGraphResample" +"flameGraphSimpleState" +"flameGraphState" +"flatten" +"flattenTuple" +"floor" +"format" +"formatDateTime" +"formatDateTimeInJodaSyntax" +"formatQuery" +"formatQueryOrNull" +"formatQuerySingleLine" +"formatQuerySingleLineOrNull" +"formatReadableDecimalSize" +"formatReadableQuantity" +"formatReadableSize" +"formatReadableTimeDelta" +"formatRow" +"formatRowNoNewline" +"fragment" +"fromDaysSinceYearZero" +"fromDaysSinceYearZero32" +"fromModifiedJulianDay" +"fromModifiedJulianDayOrNull" +"fromUTCTimestamp" +"fromUnixTimestamp" +"fromUnixTimestamp64Micro" +"fromUnixTimestamp64Milli" +"fromUnixTimestamp64Nano" +"fromUnixTimestampInJodaSyntax" +"from_utc_timestamp" +"fullHostName" +"fuzzBits" +"gccMurmurHash" +"gcd" +"generateRandomStructure" +"generateSnowflakeID" +"generateULID" +"generateUUIDv4" +"generateUUIDv7" +"geoDistance" +"geoToH3" +"geoToS2" +"geohashDecode" +"geohashEncode" +"geohashesInBox" +"getClientHTTPHeader" +"getMacro" +"getOSKernelVersion" +"getServerPort" +"getSetting" +"getSizeOfEnumType" +"getSubcolumn" +"getTypeSerializationStreams" +"globalIn" +"globalInIgnoreSet" +"globalNotIn" +"globalNotInIgnoreSet" +"globalNotNullIn" +"globalNotNullInIgnoreSet" +"globalNullIn" +"globalNullInIgnoreSet" +"globalVariable" +"greatCircleAngle" +"greatCircleDistance" +"greater" +"greaterOrEquals" +"greatest" +"groupArray" +"groupArrayArgMax" +"groupArrayArgMin" +"groupArrayArray" +"groupArrayDistinct" +"groupArrayForEach" +"groupArrayIf" +"groupArrayInsertAt" +"groupArrayInsertAtArgMax" +"groupArrayInsertAtArgMin" +"groupArrayInsertAtArray" +"groupArrayInsertAtDistinct" +"groupArrayInsertAtForEach" +"groupArrayInsertAtIf" +"groupArrayInsertAtMap" +"groupArrayInsertAtMerge" +"groupArrayInsertAtNull" +"groupArrayInsertAtOrDefault" +"groupArrayInsertAtOrNull" +"groupArrayInsertAtResample" +"groupArrayInsertAtSimpleState" +"groupArrayInsertAtState" +"groupArrayIntersect" +"groupArrayIntersectArgMax" +"groupArrayIntersectArgMin" +"groupArrayIntersectArray" +"groupArrayIntersectDistinct" +"groupArrayIntersectForEach" +"groupArrayIntersectIf" +"groupArrayIntersectMap" +"groupArrayIntersectMerge" +"groupArrayIntersectNull" +"groupArrayIntersectOrDefault" +"groupArrayIntersectOrNull" +"groupArrayIntersectResample" +"groupArrayIntersectSimpleState" +"groupArrayIntersectState" +"groupArrayLast" +"groupArrayLastArgMax" +"groupArrayLastArgMin" +"groupArrayLastArray" +"groupArrayLastDistinct" +"groupArrayLastForEach" +"groupArrayLastIf" +"groupArrayLastMap" +"groupArrayLastMerge" +"groupArrayLastNull" +"groupArrayLastOrDefault" +"groupArrayLastOrNull" +"groupArrayLastResample" +"groupArrayLastSimpleState" +"groupArrayLastState" +"groupArrayMap" +"groupArrayMerge" +"groupArrayMovingAvg" +"groupArrayMovingAvgArgMax" +"groupArrayMovingAvgArgMin" +"groupArrayMovingAvgArray" +"groupArrayMovingAvgDistinct" +"groupArrayMovingAvgForEach" +"groupArrayMovingAvgIf" +"groupArrayMovingAvgMap" +"groupArrayMovingAvgMerge" +"groupArrayMovingAvgNull" +"groupArrayMovingAvgOrDefault" +"groupArrayMovingAvgOrNull" +"groupArrayMovingAvgResample" +"groupArrayMovingAvgSimpleState" +"groupArrayMovingAvgState" +"groupArrayMovingSum" +"groupArrayMovingSumArgMax" +"groupArrayMovingSumArgMin" +"groupArrayMovingSumArray" +"groupArrayMovingSumDistinct" +"groupArrayMovingSumForEach" +"groupArrayMovingSumIf" +"groupArrayMovingSumMap" +"groupArrayMovingSumMerge" +"groupArrayMovingSumNull" +"groupArrayMovingSumOrDefault" +"groupArrayMovingSumOrNull" +"groupArrayMovingSumResample" +"groupArrayMovingSumSimpleState" +"groupArrayMovingSumState" +"groupArrayNull" +"groupArrayOrDefault" +"groupArrayOrNull" +"groupArrayResample" +"groupArraySample" +"groupArraySampleArgMax" +"groupArraySampleArgMin" +"groupArraySampleArray" +"groupArraySampleDistinct" +"groupArraySampleForEach" +"groupArraySampleIf" +"groupArraySampleMap" +"groupArraySampleMerge" +"groupArraySampleNull" +"groupArraySampleOrDefault" +"groupArraySampleOrNull" +"groupArraySampleResample" +"groupArraySampleSimpleState" +"groupArraySampleState" +"groupArraySimpleState" +"groupArraySorted" +"groupArraySortedArgMax" +"groupArraySortedArgMin" +"groupArraySortedArray" +"groupArraySortedDistinct" +"groupArraySortedForEach" +"groupArraySortedIf" +"groupArraySortedMap" +"groupArraySortedMerge" +"groupArraySortedNull" +"groupArraySortedOrDefault" +"groupArraySortedOrNull" +"groupArraySortedResample" +"groupArraySortedSimpleState" +"groupArraySortedState" +"groupArrayState" +"groupBitAnd" +"groupBitAndArgMax" +"groupBitAndArgMin" +"groupBitAndArray" +"groupBitAndDistinct" +"groupBitAndForEach" +"groupBitAndIf" +"groupBitAndMap" +"groupBitAndMerge" +"groupBitAndNull" +"groupBitAndOrDefault" +"groupBitAndOrNull" +"groupBitAndResample" +"groupBitAndSimpleState" +"groupBitAndState" +"groupBitOr" +"groupBitOrArgMax" +"groupBitOrArgMin" +"groupBitOrArray" +"groupBitOrDistinct" +"groupBitOrForEach" +"groupBitOrIf" +"groupBitOrMap" +"groupBitOrMerge" +"groupBitOrNull" +"groupBitOrOrDefault" +"groupBitOrOrNull" +"groupBitOrResample" +"groupBitOrSimpleState" +"groupBitOrState" +"groupBitXor" +"groupBitXorArgMax" +"groupBitXorArgMin" +"groupBitXorArray" +"groupBitXorDistinct" +"groupBitXorForEach" +"groupBitXorIf" +"groupBitXorMap" +"groupBitXorMerge" +"groupBitXorNull" +"groupBitXorOrDefault" +"groupBitXorOrNull" +"groupBitXorResample" +"groupBitXorSimpleState" +"groupBitXorState" +"groupBitmap" +"groupBitmapAnd" +"groupBitmapAndArgMax" +"groupBitmapAndArgMin" +"groupBitmapAndArray" +"groupBitmapAndDistinct" +"groupBitmapAndForEach" +"groupBitmapAndIf" +"groupBitmapAndMap" +"groupBitmapAndMerge" +"groupBitmapAndNull" +"groupBitmapAndOrDefault" +"groupBitmapAndOrNull" +"groupBitmapAndResample" +"groupBitmapAndSimpleState" +"groupBitmapAndState" +"groupBitmapArgMax" +"groupBitmapArgMin" +"groupBitmapArray" +"groupBitmapDistinct" +"groupBitmapForEach" +"groupBitmapIf" +"groupBitmapMap" +"groupBitmapMerge" +"groupBitmapNull" +"groupBitmapOr" +"groupBitmapOrArgMax" +"groupBitmapOrArgMin" +"groupBitmapOrArray" +"groupBitmapOrDefault" +"groupBitmapOrDistinct" +"groupBitmapOrForEach" +"groupBitmapOrIf" +"groupBitmapOrMap" +"groupBitmapOrMerge" +"groupBitmapOrNull" +"groupBitmapOrNull" +"groupBitmapOrOrDefault" +"groupBitmapOrOrNull" +"groupBitmapOrResample" +"groupBitmapOrSimpleState" +"groupBitmapOrState" +"groupBitmapResample" +"groupBitmapSimpleState" +"groupBitmapState" +"groupBitmapXor" +"groupBitmapXorArgMax" +"groupBitmapXorArgMin" +"groupBitmapXorArray" +"groupBitmapXorDistinct" +"groupBitmapXorForEach" +"groupBitmapXorIf" +"groupBitmapXorMap" +"groupBitmapXorMerge" +"groupBitmapXorNull" +"groupBitmapXorOrDefault" +"groupBitmapXorOrNull" +"groupBitmapXorResample" +"groupBitmapXorSimpleState" +"groupBitmapXorState" +"groupConcat" +"groupConcatArgMax" +"groupConcatArgMin" +"groupConcatArray" +"groupConcatDistinct" +"groupConcatForEach" +"groupConcatIf" +"groupConcatMap" +"groupConcatMerge" +"groupConcatNull" +"groupConcatOrDefault" +"groupConcatOrNull" +"groupConcatResample" +"groupConcatSimpleState" +"groupConcatState" +"groupUniqArray" +"groupUniqArrayArgMax" +"groupUniqArrayArgMin" +"groupUniqArrayArray" +"groupUniqArrayDistinct" +"groupUniqArrayForEach" +"groupUniqArrayIf" +"groupUniqArrayMap" +"groupUniqArrayMerge" +"groupUniqArrayNull" +"groupUniqArrayOrDefault" +"groupUniqArrayOrNull" +"groupUniqArrayResample" +"groupUniqArraySimpleState" +"groupUniqArrayState" +"group_concat" +"group_concatArgMax" +"group_concatArgMin" +"group_concatArray" +"group_concatDistinct" +"group_concatForEach" +"group_concatIf" +"group_concatMap" +"group_concatMerge" +"group_concatNull" +"group_concatOrDefault" +"group_concatOrNull" +"group_concatResample" +"group_concatSimpleState" +"group_concatState" +"h3CellAreaM2" +"h3CellAreaRads2" +"h3Distance" +"h3EdgeAngle" +"h3EdgeLengthKm" +"h3EdgeLengthM" +"h3ExactEdgeLengthKm" +"h3ExactEdgeLengthM" +"h3ExactEdgeLengthRads" +"h3GetBaseCell" +"h3GetDestinationIndexFromUnidirectionalEdge" +"h3GetFaces" +"h3GetIndexesFromUnidirectionalEdge" +"h3GetOriginIndexFromUnidirectionalEdge" +"h3GetPentagonIndexes" +"h3GetRes0Indexes" +"h3GetResolution" +"h3GetUnidirectionalEdge" +"h3GetUnidirectionalEdgeBoundary" +"h3GetUnidirectionalEdgesFromHexagon" +"h3HexAreaKm2" +"h3HexAreaM2" +"h3HexRing" +"h3IndexesAreNeighbors" +"h3IsPentagon" +"h3IsResClassIII" +"h3IsValid" +"h3Line" +"h3NumHexagons" +"h3PointDistKm" +"h3PointDistM" +"h3PointDistRads" +"h3ToCenterChild" +"h3ToChildren" +"h3ToGeo" +"h3ToGeoBoundary" +"h3ToParent" +"h3ToString" +"h3UnidirectionalEdgeIsValid" +"h3kRing" +"halfMD5" +"has" +"hasAll" +"hasAny" +"hasColumnInTable" +"hasSubsequence" +"hasSubsequenceCaseInsensitive" +"hasSubsequenceCaseInsensitiveUTF8" +"hasSubsequenceUTF8" +"hasSubstr" +"hasThreadFuzzer" +"hasToken" +"hasTokenCaseInsensitive" +"hasTokenCaseInsensitiveOrNull" +"hasTokenOrNull" +"hex" +"hilbertDecode" +"hilbertEncode" +"histogram" +"histogramArgMax" +"histogramArgMin" +"histogramArray" +"histogramDistinct" +"histogramForEach" +"histogramIf" +"histogramMap" +"histogramMerge" +"histogramNull" +"histogramOrDefault" +"histogramOrNull" +"histogramResample" +"histogramSimpleState" +"histogramState" +"hiveHash" +"hop" +"hopEnd" +"hopStart" +"hostName" +"hostname" +"hypot" +"identity" +"idnaDecode" +"idnaEncode" +"if" +"ifNotFinite" +"ifNull" +"ignore" +"ilike" +"in" +"inIgnoreSet" +"indexHint" +"indexOf" +"initcap" +"initcapUTF8" +"initialQueryID" +"initial_query_id" +"initializeAggregation" +"instr" +"intDiv" +"intDivOrZero" +"intExp10" +"intExp2" +"intHash32" +"intHash64" +"intervalLengthSum" +"intervalLengthSumArgMax" +"intervalLengthSumArgMin" +"intervalLengthSumArray" +"intervalLengthSumDistinct" +"intervalLengthSumForEach" +"intervalLengthSumIf" +"intervalLengthSumMap" +"intervalLengthSumMerge" +"intervalLengthSumNull" +"intervalLengthSumOrDefault" +"intervalLengthSumOrNull" +"intervalLengthSumResample" +"intervalLengthSumSimpleState" +"intervalLengthSumState" +"isConstant" +"isDecimalOverflow" +"isFinite" +"isIPAddressInRange" +"isIPv4String" +"isIPv6String" +"isInfinite" +"isNaN" +"isNotDistinctFrom" +"isNotNull" +"isNull" +"isNullable" +"isValidJSON" +"isValidUTF8" +"isZeroOrNull" +"jaroSimilarity" +"jaroWinklerSimilarity" +"javaHash" +"javaHashUTF16LE" +"joinGet" +"joinGetOrNull" +"jsonMergePatch" +"jumpConsistentHash" +"kafkaMurmurHash" +"kolmogorovSmirnovTest" +"kolmogorovSmirnovTestArgMax" +"kolmogorovSmirnovTestArgMin" +"kolmogorovSmirnovTestArray" +"kolmogorovSmirnovTestDistinct" +"kolmogorovSmirnovTestForEach" +"kolmogorovSmirnovTestIf" +"kolmogorovSmirnovTestMap" +"kolmogorovSmirnovTestMerge" +"kolmogorovSmirnovTestNull" +"kolmogorovSmirnovTestOrDefault" +"kolmogorovSmirnovTestOrNull" +"kolmogorovSmirnovTestResample" +"kolmogorovSmirnovTestSimpleState" +"kolmogorovSmirnovTestState" +"kostikConsistentHash" +"kql_array_sort_asc" +"kql_array_sort_desc" +"kurtPop" +"kurtPopArgMax" +"kurtPopArgMin" +"kurtPopArray" +"kurtPopDistinct" +"kurtPopForEach" +"kurtPopIf" +"kurtPopMap" +"kurtPopMerge" +"kurtPopNull" +"kurtPopOrDefault" +"kurtPopOrNull" +"kurtPopResample" +"kurtPopSimpleState" +"kurtPopState" +"kurtSamp" +"kurtSampArgMax" +"kurtSampArgMin" +"kurtSampArray" +"kurtSampDistinct" +"kurtSampForEach" +"kurtSampIf" +"kurtSampMap" +"kurtSampMerge" +"kurtSampNull" +"kurtSampOrDefault" +"kurtSampOrNull" +"kurtSampResample" +"kurtSampSimpleState" +"kurtSampState" +"lagInFrame" +"lagInFrameArgMax" +"lagInFrameArgMin" +"lagInFrameArray" +"lagInFrameDistinct" +"lagInFrameForEach" +"lagInFrameIf" +"lagInFrameMap" +"lagInFrameMerge" +"lagInFrameNull" +"lagInFrameOrDefault" +"lagInFrameOrNull" +"lagInFrameResample" +"lagInFrameSimpleState" +"lagInFrameState" +"largestTriangleThreeBuckets" +"largestTriangleThreeBucketsArgMax" +"largestTriangleThreeBucketsArgMin" +"largestTriangleThreeBucketsArray" +"largestTriangleThreeBucketsDistinct" +"largestTriangleThreeBucketsForEach" +"largestTriangleThreeBucketsIf" +"largestTriangleThreeBucketsMap" +"largestTriangleThreeBucketsMerge" +"largestTriangleThreeBucketsNull" +"largestTriangleThreeBucketsOrDefault" +"largestTriangleThreeBucketsOrNull" +"largestTriangleThreeBucketsResample" +"largestTriangleThreeBucketsSimpleState" +"largestTriangleThreeBucketsState" +"last_value" +"last_valueArgMax" +"last_valueArgMin" +"last_valueArray" +"last_valueDistinct" +"last_valueForEach" +"last_valueIf" +"last_valueMap" +"last_valueMerge" +"last_valueNull" +"last_valueOrDefault" +"last_valueOrNull" +"last_valueResample" +"last_valueSimpleState" +"last_valueState" +"last_value_respect_nulls" +"last_value_respect_nullsArgMax" +"last_value_respect_nullsArgMin" +"last_value_respect_nullsArray" +"last_value_respect_nullsDistinct" +"last_value_respect_nullsForEach" +"last_value_respect_nullsIf" +"last_value_respect_nullsMap" +"last_value_respect_nullsMerge" +"last_value_respect_nullsNull" +"last_value_respect_nullsOrDefault" +"last_value_respect_nullsOrNull" +"last_value_respect_nullsResample" +"last_value_respect_nullsSimpleState" +"last_value_respect_nullsState" +"lcase" +"lcm" +"leadInFrame" +"leadInFrameArgMax" +"leadInFrameArgMin" +"leadInFrameArray" +"leadInFrameDistinct" +"leadInFrameForEach" +"leadInFrameIf" +"leadInFrameMap" +"leadInFrameMerge" +"leadInFrameNull" +"leadInFrameOrDefault" +"leadInFrameOrNull" +"leadInFrameResample" +"leadInFrameSimpleState" +"leadInFrameState" +"least" +"left" +"leftPad" +"leftPadUTF8" +"leftUTF8" +"lemmatize" +"length" +"lengthUTF8" +"less" +"lessOrEquals" +"levenshteinDistance" +"levenshteinDistanceUTF8" +"lgamma" +"like" +"ln" +"locate" +"log" +"log10" +"log1p" +"log2" +"logTrace" +"lowCardinalityIndices" +"lowCardinalityKeys" +"lower" +"lowerUTF8" +"lpad" +"ltrim" +"lttb" +"lttbArgMax" +"lttbArgMin" +"lttbArray" +"lttbDistinct" +"lttbForEach" +"lttbIf" +"lttbMap" +"lttbMerge" +"lttbNull" +"lttbOrDefault" +"lttbOrNull" +"lttbResample" +"lttbSimpleState" +"lttbState" +"makeDate" +"makeDate32" +"makeDateTime" +"makeDateTime64" +"mannWhitneyUTest" +"mannWhitneyUTestArgMax" +"mannWhitneyUTestArgMin" +"mannWhitneyUTestArray" +"mannWhitneyUTestDistinct" +"mannWhitneyUTestForEach" +"mannWhitneyUTestIf" +"mannWhitneyUTestMap" +"mannWhitneyUTestMerge" +"mannWhitneyUTestNull" +"mannWhitneyUTestOrDefault" +"mannWhitneyUTestOrNull" +"mannWhitneyUTestResample" +"mannWhitneyUTestSimpleState" +"mannWhitneyUTestState" +"map" +"mapAdd" +"mapAll" +"mapApply" +"mapConcat" +"mapContains" +"mapContainsKeyLike" +"mapExists" +"mapExtractKeyLike" +"mapFilter" +"mapFromArrays" +"mapFromString" +"mapKeys" +"mapPartialReverseSort" +"mapPartialSort" +"mapPopulateSeries" +"mapReverseSort" +"mapSort" +"mapSubtract" +"mapUpdate" +"mapValues" +"match" +"materialize" +"max" +"max2" +"maxArgMax" +"maxArgMin" +"maxArray" +"maxDistinct" +"maxForEach" +"maxIf" +"maxIntersections" +"maxIntersectionsArgMax" +"maxIntersectionsArgMin" +"maxIntersectionsArray" +"maxIntersectionsDistinct" +"maxIntersectionsForEach" +"maxIntersectionsIf" +"maxIntersectionsMap" +"maxIntersectionsMerge" +"maxIntersectionsNull" +"maxIntersectionsOrDefault" +"maxIntersectionsOrNull" +"maxIntersectionsPosition" +"maxIntersectionsPositionArgMax" +"maxIntersectionsPositionArgMin" +"maxIntersectionsPositionArray" +"maxIntersectionsPositionDistinct" +"maxIntersectionsPositionForEach" +"maxIntersectionsPositionIf" +"maxIntersectionsPositionMap" +"maxIntersectionsPositionMerge" +"maxIntersectionsPositionNull" +"maxIntersectionsPositionOrDefault" +"maxIntersectionsPositionOrNull" +"maxIntersectionsPositionResample" +"maxIntersectionsPositionSimpleState" +"maxIntersectionsPositionState" +"maxIntersectionsResample" +"maxIntersectionsSimpleState" +"maxIntersectionsState" +"maxMap" +"maxMappedArrays" +"maxMappedArraysArgMax" +"maxMappedArraysArgMin" +"maxMappedArraysArray" +"maxMappedArraysDistinct" +"maxMappedArraysForEach" +"maxMappedArraysIf" +"maxMappedArraysMap" +"maxMappedArraysMerge" +"maxMappedArraysNull" +"maxMappedArraysOrDefault" +"maxMappedArraysOrNull" +"maxMappedArraysResample" +"maxMappedArraysSimpleState" +"maxMappedArraysState" +"maxMerge" +"maxNull" +"maxOrDefault" +"maxOrNull" +"maxResample" +"maxSimpleState" +"maxState" +"meanZTest" +"meanZTestArgMax" +"meanZTestArgMin" +"meanZTestArray" +"meanZTestDistinct" +"meanZTestForEach" +"meanZTestIf" +"meanZTestMap" +"meanZTestMerge" +"meanZTestNull" +"meanZTestOrDefault" +"meanZTestOrNull" +"meanZTestResample" +"meanZTestSimpleState" +"meanZTestState" +"median" +"medianArgMax" +"medianArgMin" +"medianArray" +"medianBFloat16" +"medianBFloat16ArgMax" +"medianBFloat16ArgMin" +"medianBFloat16Array" +"medianBFloat16Distinct" +"medianBFloat16ForEach" +"medianBFloat16If" +"medianBFloat16Map" +"medianBFloat16Merge" +"medianBFloat16Null" +"medianBFloat16OrDefault" +"medianBFloat16OrNull" +"medianBFloat16Resample" +"medianBFloat16SimpleState" +"medianBFloat16State" +"medianBFloat16Weighted" +"medianBFloat16WeightedArgMax" +"medianBFloat16WeightedArgMin" +"medianBFloat16WeightedArray" +"medianBFloat16WeightedDistinct" +"medianBFloat16WeightedForEach" +"medianBFloat16WeightedIf" +"medianBFloat16WeightedMap" +"medianBFloat16WeightedMerge" +"medianBFloat16WeightedNull" +"medianBFloat16WeightedOrDefault" +"medianBFloat16WeightedOrNull" +"medianBFloat16WeightedResample" +"medianBFloat16WeightedSimpleState" +"medianBFloat16WeightedState" +"medianDD" +"medianDDArgMax" +"medianDDArgMin" +"medianDDArray" +"medianDDDistinct" +"medianDDForEach" +"medianDDIf" +"medianDDMap" +"medianDDMerge" +"medianDDNull" +"medianDDOrDefault" +"medianDDOrNull" +"medianDDResample" +"medianDDSimpleState" +"medianDDState" +"medianDeterministic" +"medianDeterministicArgMax" +"medianDeterministicArgMin" +"medianDeterministicArray" +"medianDeterministicDistinct" +"medianDeterministicForEach" +"medianDeterministicIf" +"medianDeterministicMap" +"medianDeterministicMerge" +"medianDeterministicNull" +"medianDeterministicOrDefault" +"medianDeterministicOrNull" +"medianDeterministicResample" +"medianDeterministicSimpleState" +"medianDeterministicState" +"medianDistinct" +"medianExact" +"medianExactArgMax" +"medianExactArgMin" +"medianExactArray" +"medianExactDistinct" +"medianExactForEach" +"medianExactHigh" +"medianExactHighArgMax" +"medianExactHighArgMin" +"medianExactHighArray" +"medianExactHighDistinct" +"medianExactHighForEach" +"medianExactHighIf" +"medianExactHighMap" +"medianExactHighMerge" +"medianExactHighNull" +"medianExactHighOrDefault" +"medianExactHighOrNull" +"medianExactHighResample" +"medianExactHighSimpleState" +"medianExactHighState" +"medianExactIf" +"medianExactLow" +"medianExactLowArgMax" +"medianExactLowArgMin" +"medianExactLowArray" +"medianExactLowDistinct" +"medianExactLowForEach" +"medianExactLowIf" +"medianExactLowMap" +"medianExactLowMerge" +"medianExactLowNull" +"medianExactLowOrDefault" +"medianExactLowOrNull" +"medianExactLowResample" +"medianExactLowSimpleState" +"medianExactLowState" +"medianExactMap" +"medianExactMerge" +"medianExactNull" +"medianExactOrDefault" +"medianExactOrNull" +"medianExactResample" +"medianExactSimpleState" +"medianExactState" +"medianExactWeighted" +"medianExactWeightedArgMax" +"medianExactWeightedArgMin" +"medianExactWeightedArray" +"medianExactWeightedDistinct" +"medianExactWeightedForEach" +"medianExactWeightedIf" +"medianExactWeightedMap" +"medianExactWeightedMerge" +"medianExactWeightedNull" +"medianExactWeightedOrDefault" +"medianExactWeightedOrNull" +"medianExactWeightedResample" +"medianExactWeightedSimpleState" +"medianExactWeightedState" +"medianForEach" +"medianGK" +"medianGKArgMax" +"medianGKArgMin" +"medianGKArray" +"medianGKDistinct" +"medianGKForEach" +"medianGKIf" +"medianGKMap" +"medianGKMerge" +"medianGKNull" +"medianGKOrDefault" +"medianGKOrNull" +"medianGKResample" +"medianGKSimpleState" +"medianGKState" +"medianIf" +"medianInterpolatedWeighted" +"medianInterpolatedWeightedArgMax" +"medianInterpolatedWeightedArgMin" +"medianInterpolatedWeightedArray" +"medianInterpolatedWeightedDistinct" +"medianInterpolatedWeightedForEach" +"medianInterpolatedWeightedIf" +"medianInterpolatedWeightedMap" +"medianInterpolatedWeightedMerge" +"medianInterpolatedWeightedNull" +"medianInterpolatedWeightedOrDefault" +"medianInterpolatedWeightedOrNull" +"medianInterpolatedWeightedResample" +"medianInterpolatedWeightedSimpleState" +"medianInterpolatedWeightedState" +"medianMap" +"medianMerge" +"medianNull" +"medianOrDefault" +"medianOrNull" +"medianResample" +"medianSimpleState" +"medianState" +"medianTDigest" +"medianTDigestArgMax" +"medianTDigestArgMin" +"medianTDigestArray" +"medianTDigestDistinct" +"medianTDigestForEach" +"medianTDigestIf" +"medianTDigestMap" +"medianTDigestMerge" +"medianTDigestNull" +"medianTDigestOrDefault" +"medianTDigestOrNull" +"medianTDigestResample" +"medianTDigestSimpleState" +"medianTDigestState" +"medianTDigestWeighted" +"medianTDigestWeightedArgMax" +"medianTDigestWeightedArgMin" +"medianTDigestWeightedArray" +"medianTDigestWeightedDistinct" +"medianTDigestWeightedForEach" +"medianTDigestWeightedIf" +"medianTDigestWeightedMap" +"medianTDigestWeightedMerge" +"medianTDigestWeightedNull" +"medianTDigestWeightedOrDefault" +"medianTDigestWeightedOrNull" +"medianTDigestWeightedResample" +"medianTDigestWeightedSimpleState" +"medianTDigestWeightedState" +"medianTiming" +"medianTimingArgMax" +"medianTimingArgMin" +"medianTimingArray" +"medianTimingDistinct" +"medianTimingForEach" +"medianTimingIf" +"medianTimingMap" +"medianTimingMerge" +"medianTimingNull" +"medianTimingOrDefault" +"medianTimingOrNull" +"medianTimingResample" +"medianTimingSimpleState" +"medianTimingState" +"medianTimingWeighted" +"medianTimingWeightedArgMax" +"medianTimingWeightedArgMin" +"medianTimingWeightedArray" +"medianTimingWeightedDistinct" +"medianTimingWeightedForEach" +"medianTimingWeightedIf" +"medianTimingWeightedMap" +"medianTimingWeightedMerge" +"medianTimingWeightedNull" +"medianTimingWeightedOrDefault" +"medianTimingWeightedOrNull" +"medianTimingWeightedResample" +"medianTimingWeightedSimpleState" +"medianTimingWeightedState" +"metroHash64" +"mid" +"min" +"min2" +"minArgMax" +"minArgMin" +"minArray" +"minDistinct" +"minForEach" +"minIf" +"minMap" +"minMappedArrays" +"minMappedArraysArgMax" +"minMappedArraysArgMin" +"minMappedArraysArray" +"minMappedArraysDistinct" +"minMappedArraysForEach" +"minMappedArraysIf" +"minMappedArraysMap" +"minMappedArraysMerge" +"minMappedArraysNull" +"minMappedArraysOrDefault" +"minMappedArraysOrNull" +"minMappedArraysResample" +"minMappedArraysSimpleState" +"minMappedArraysState" +"minMerge" +"minNull" +"minOrDefault" +"minOrNull" +"minResample" +"minSampleSizeContinous" +"minSampleSizeContinuous" +"minSampleSizeConversion" +"minSimpleState" +"minState" +"minus" +"mismatches" +"mod" +"modulo" +"moduloLegacy" +"moduloOrZero" +"monthName" +"mortonDecode" +"mortonEncode" +"multiFuzzyMatchAllIndices" +"multiFuzzyMatchAny" +"multiFuzzyMatchAnyIndex" +"multiIf" +"multiMatchAllIndices" +"multiMatchAny" +"multiMatchAnyIndex" +"multiSearchAllPositions" +"multiSearchAllPositionsCaseInsensitive" +"multiSearchAllPositionsCaseInsensitiveUTF8" +"multiSearchAllPositionsUTF8" +"multiSearchAny" +"multiSearchAnyCaseInsensitive" +"multiSearchAnyCaseInsensitiveUTF8" +"multiSearchAnyUTF8" +"multiSearchFirstIndex" +"multiSearchFirstIndexCaseInsensitive" +"multiSearchFirstIndexCaseInsensitiveUTF8" +"multiSearchFirstIndexUTF8" +"multiSearchFirstPosition" +"multiSearchFirstPositionCaseInsensitive" +"multiSearchFirstPositionCaseInsensitiveUTF8" +"multiSearchFirstPositionUTF8" +"multiply" +"multiplyDecimal" +"murmurHash2_32" +"murmurHash2_64" +"murmurHash3_128" +"murmurHash3_32" +"murmurHash3_64" +"negate" +"neighbor" +"nested" +"netloc" +"ngramDistance" +"ngramDistanceCaseInsensitive" +"ngramDistanceCaseInsensitiveUTF8" +"ngramDistanceUTF8" +"ngramMinHash" +"ngramMinHashArg" +"ngramMinHashArgCaseInsensitive" +"ngramMinHashArgCaseInsensitiveUTF8" +"ngramMinHashArgUTF8" +"ngramMinHashCaseInsensitive" +"ngramMinHashCaseInsensitiveUTF8" +"ngramMinHashUTF8" +"ngramSearch" +"ngramSearchCaseInsensitive" +"ngramSearchCaseInsensitiveUTF8" +"ngramSearchUTF8" +"ngramSimHash" +"ngramSimHashCaseInsensitive" +"ngramSimHashCaseInsensitiveUTF8" +"ngramSimHashUTF8" +"ngrams" +"nonNegativeDerivative" +"nonNegativeDerivativeArgMax" +"nonNegativeDerivativeArgMin" +"nonNegativeDerivativeArray" +"nonNegativeDerivativeDistinct" +"nonNegativeDerivativeForEach" +"nonNegativeDerivativeIf" +"nonNegativeDerivativeMap" +"nonNegativeDerivativeMerge" +"nonNegativeDerivativeNull" +"nonNegativeDerivativeOrDefault" +"nonNegativeDerivativeOrNull" +"nonNegativeDerivativeResample" +"nonNegativeDerivativeSimpleState" +"nonNegativeDerivativeState" +"normL1" +"normL2" +"normL2Squared" +"normLinf" +"normLp" +"normalizeL1" +"normalizeL2" +"normalizeLinf" +"normalizeLp" +"normalizeQuery" +"normalizeQueryKeepNames" +"normalizeUTF8NFC" +"normalizeUTF8NFD" +"normalizeUTF8NFKC" +"normalizeUTF8NFKD" +"normalizedQueryHash" +"normalizedQueryHashKeepNames" +"not" +"notEmpty" +"notEquals" +"notILike" +"notIn" +"notInIgnoreSet" +"notLike" +"notNullIn" +"notNullInIgnoreSet" +"nothing" +"nothingArgMax" +"nothingArgMin" +"nothingArray" +"nothingDistinct" +"nothingForEach" +"nothingIf" +"nothingMap" +"nothingMerge" +"nothingNull" +"nothingNull" +"nothingNullArgMax" +"nothingNullArgMin" +"nothingNullArray" +"nothingNullDistinct" +"nothingNullForEach" +"nothingNullIf" +"nothingNullMap" +"nothingNullMerge" +"nothingNullNull" +"nothingNullOrDefault" +"nothingNullOrNull" +"nothingNullResample" +"nothingNullSimpleState" +"nothingNullState" +"nothingOrDefault" +"nothingOrNull" +"nothingResample" +"nothingSimpleState" +"nothingState" +"nothingUInt64" +"nothingUInt64ArgMax" +"nothingUInt64ArgMin" +"nothingUInt64Array" +"nothingUInt64Distinct" +"nothingUInt64ForEach" +"nothingUInt64If" +"nothingUInt64Map" +"nothingUInt64Merge" +"nothingUInt64Null" +"nothingUInt64OrDefault" +"nothingUInt64OrNull" +"nothingUInt64Resample" +"nothingUInt64SimpleState" +"nothingUInt64State" +"now" +"now64" +"nowInBlock" +"nth_value" +"nth_valueArgMax" +"nth_valueArgMin" +"nth_valueArray" +"nth_valueDistinct" +"nth_valueForEach" +"nth_valueIf" +"nth_valueMap" +"nth_valueMerge" +"nth_valueNull" +"nth_valueOrDefault" +"nth_valueOrNull" +"nth_valueResample" +"nth_valueSimpleState" +"nth_valueState" +"ntile" +"ntileArgMax" +"ntileArgMin" +"ntileArray" +"ntileDistinct" +"ntileForEach" +"ntileIf" +"ntileMap" +"ntileMerge" +"ntileNull" +"ntileOrDefault" +"ntileOrNull" +"ntileResample" +"ntileSimpleState" +"ntileState" +"nullIf" +"nullIn" +"nullInIgnoreSet" +"or" +"parseDateTime" +"parseDateTime32BestEffort" +"parseDateTime32BestEffortOrNull" +"parseDateTime32BestEffortOrZero" +"parseDateTime64BestEffort" +"parseDateTime64BestEffortOrNull" +"parseDateTime64BestEffortOrZero" +"parseDateTime64BestEffortUS" +"parseDateTime64BestEffortUSOrNull" +"parseDateTime64BestEffortUSOrZero" +"parseDateTimeBestEffort" +"parseDateTimeBestEffortOrNull" +"parseDateTimeBestEffortOrZero" +"parseDateTimeBestEffortUS" +"parseDateTimeBestEffortUSOrNull" +"parseDateTimeBestEffortUSOrZero" +"parseDateTimeInJodaSyntax" +"parseDateTimeInJodaSyntaxOrNull" +"parseDateTimeInJodaSyntaxOrZero" +"parseDateTimeOrNull" +"parseDateTimeOrZero" +"parseReadableSize" +"parseReadableSizeOrNull" +"parseReadableSizeOrZero" +"parseTimeDelta" +"partitionID" +"partitionId" +"path" +"pathFull" +"percentRank" +"percentRankArgMax" +"percentRankArgMin" +"percentRankArray" +"percentRankDistinct" +"percentRankForEach" +"percentRankIf" +"percentRankMap" +"percentRankMerge" +"percentRankNull" +"percentRankOrDefault" +"percentRankOrNull" +"percentRankResample" +"percentRankSimpleState" +"percentRankState" +"percent_rank" +"percent_rankArgMax" +"percent_rankArgMin" +"percent_rankArray" +"percent_rankDistinct" +"percent_rankForEach" +"percent_rankIf" +"percent_rankMap" +"percent_rankMerge" +"percent_rankNull" +"percent_rankOrDefault" +"percent_rankOrNull" +"percent_rankResample" +"percent_rankSimpleState" +"percent_rankState" +"pi" +"plus" +"pmod" +"pointInEllipses" +"pointInPolygon" +"polygonAreaCartesian" +"polygonAreaSpherical" +"polygonConvexHullCartesian" +"polygonPerimeterCartesian" +"polygonPerimeterSpherical" +"polygonsDistanceCartesian" +"polygonsDistanceSpherical" +"polygonsEqualsCartesian" +"polygonsIntersectionCartesian" +"polygonsIntersectionSpherical" +"polygonsSymDifferenceCartesian" +"polygonsSymDifferenceSpherical" +"polygonsUnionCartesian" +"polygonsUnionSpherical" +"polygonsWithinCartesian" +"polygonsWithinSpherical" +"port" +"portRFC" +"position" +"positionCaseInsensitive" +"positionCaseInsensitiveUTF8" +"positionUTF8" +"positiveModulo" +"positive_modulo" +"pow" +"power" +"printf" +"proportionsZTest" +"protocol" +"punycodeDecode" +"punycodeEncode" +"quantile" +"quantileArgMax" +"quantileArgMin" +"quantileArray" +"quantileBFloat16" +"quantileBFloat16ArgMax" +"quantileBFloat16ArgMin" +"quantileBFloat16Array" +"quantileBFloat16Distinct" +"quantileBFloat16ForEach" +"quantileBFloat16If" +"quantileBFloat16Map" +"quantileBFloat16Merge" +"quantileBFloat16Null" +"quantileBFloat16OrDefault" +"quantileBFloat16OrNull" +"quantileBFloat16Resample" +"quantileBFloat16SimpleState" +"quantileBFloat16State" +"quantileBFloat16Weighted" +"quantileBFloat16WeightedArgMax" +"quantileBFloat16WeightedArgMin" +"quantileBFloat16WeightedArray" +"quantileBFloat16WeightedDistinct" +"quantileBFloat16WeightedForEach" +"quantileBFloat16WeightedIf" +"quantileBFloat16WeightedMap" +"quantileBFloat16WeightedMerge" +"quantileBFloat16WeightedNull" +"quantileBFloat16WeightedOrDefault" +"quantileBFloat16WeightedOrNull" +"quantileBFloat16WeightedResample" +"quantileBFloat16WeightedSimpleState" +"quantileBFloat16WeightedState" +"quantileDD" +"quantileDDArgMax" +"quantileDDArgMin" +"quantileDDArray" +"quantileDDDistinct" +"quantileDDForEach" +"quantileDDIf" +"quantileDDMap" +"quantileDDMerge" +"quantileDDNull" +"quantileDDOrDefault" +"quantileDDOrNull" +"quantileDDResample" +"quantileDDSimpleState" +"quantileDDState" +"quantileDeterministic" +"quantileDeterministicArgMax" +"quantileDeterministicArgMin" +"quantileDeterministicArray" +"quantileDeterministicDistinct" +"quantileDeterministicForEach" +"quantileDeterministicIf" +"quantileDeterministicMap" +"quantileDeterministicMerge" +"quantileDeterministicNull" +"quantileDeterministicOrDefault" +"quantileDeterministicOrNull" +"quantileDeterministicResample" +"quantileDeterministicSimpleState" +"quantileDeterministicState" +"quantileDistinct" +"quantileExact" +"quantileExactArgMax" +"quantileExactArgMin" +"quantileExactArray" +"quantileExactDistinct" +"quantileExactExclusive" +"quantileExactExclusiveArgMax" +"quantileExactExclusiveArgMin" +"quantileExactExclusiveArray" +"quantileExactExclusiveDistinct" +"quantileExactExclusiveForEach" +"quantileExactExclusiveIf" +"quantileExactExclusiveMap" +"quantileExactExclusiveMerge" +"quantileExactExclusiveNull" +"quantileExactExclusiveOrDefault" +"quantileExactExclusiveOrNull" +"quantileExactExclusiveResample" +"quantileExactExclusiveSimpleState" +"quantileExactExclusiveState" +"quantileExactForEach" +"quantileExactHigh" +"quantileExactHighArgMax" +"quantileExactHighArgMin" +"quantileExactHighArray" +"quantileExactHighDistinct" +"quantileExactHighForEach" +"quantileExactHighIf" +"quantileExactHighMap" +"quantileExactHighMerge" +"quantileExactHighNull" +"quantileExactHighOrDefault" +"quantileExactHighOrNull" +"quantileExactHighResample" +"quantileExactHighSimpleState" +"quantileExactHighState" +"quantileExactIf" +"quantileExactInclusive" +"quantileExactInclusiveArgMax" +"quantileExactInclusiveArgMin" +"quantileExactInclusiveArray" +"quantileExactInclusiveDistinct" +"quantileExactInclusiveForEach" +"quantileExactInclusiveIf" +"quantileExactInclusiveMap" +"quantileExactInclusiveMerge" +"quantileExactInclusiveNull" +"quantileExactInclusiveOrDefault" +"quantileExactInclusiveOrNull" +"quantileExactInclusiveResample" +"quantileExactInclusiveSimpleState" +"quantileExactInclusiveState" +"quantileExactLow" +"quantileExactLowArgMax" +"quantileExactLowArgMin" +"quantileExactLowArray" +"quantileExactLowDistinct" +"quantileExactLowForEach" +"quantileExactLowIf" +"quantileExactLowMap" +"quantileExactLowMerge" +"quantileExactLowNull" +"quantileExactLowOrDefault" +"quantileExactLowOrNull" +"quantileExactLowResample" +"quantileExactLowSimpleState" +"quantileExactLowState" +"quantileExactMap" +"quantileExactMerge" +"quantileExactNull" +"quantileExactOrDefault" +"quantileExactOrNull" +"quantileExactResample" +"quantileExactSimpleState" +"quantileExactState" +"quantileExactWeighted" +"quantileExactWeightedArgMax" +"quantileExactWeightedArgMin" +"quantileExactWeightedArray" +"quantileExactWeightedDistinct" +"quantileExactWeightedForEach" +"quantileExactWeightedIf" +"quantileExactWeightedMap" +"quantileExactWeightedMerge" +"quantileExactWeightedNull" +"quantileExactWeightedOrDefault" +"quantileExactWeightedOrNull" +"quantileExactWeightedResample" +"quantileExactWeightedSimpleState" +"quantileExactWeightedState" +"quantileForEach" +"quantileGK" +"quantileGKArgMax" +"quantileGKArgMin" +"quantileGKArray" +"quantileGKDistinct" +"quantileGKForEach" +"quantileGKIf" +"quantileGKMap" +"quantileGKMerge" +"quantileGKNull" +"quantileGKOrDefault" +"quantileGKOrNull" +"quantileGKResample" +"quantileGKSimpleState" +"quantileGKState" +"quantileIf" +"quantileInterpolatedWeighted" +"quantileInterpolatedWeightedArgMax" +"quantileInterpolatedWeightedArgMin" +"quantileInterpolatedWeightedArray" +"quantileInterpolatedWeightedDistinct" +"quantileInterpolatedWeightedForEach" +"quantileInterpolatedWeightedIf" +"quantileInterpolatedWeightedMap" +"quantileInterpolatedWeightedMerge" +"quantileInterpolatedWeightedNull" +"quantileInterpolatedWeightedOrDefault" +"quantileInterpolatedWeightedOrNull" +"quantileInterpolatedWeightedResample" +"quantileInterpolatedWeightedSimpleState" +"quantileInterpolatedWeightedState" +"quantileMap" +"quantileMerge" +"quantileNull" +"quantileOrDefault" +"quantileOrNull" +"quantileResample" +"quantileSimpleState" +"quantileState" +"quantileTDigest" +"quantileTDigestArgMax" +"quantileTDigestArgMin" +"quantileTDigestArray" +"quantileTDigestDistinct" +"quantileTDigestForEach" +"quantileTDigestIf" +"quantileTDigestMap" +"quantileTDigestMerge" +"quantileTDigestNull" +"quantileTDigestOrDefault" +"quantileTDigestOrNull" +"quantileTDigestResample" +"quantileTDigestSimpleState" +"quantileTDigestState" +"quantileTDigestWeighted" +"quantileTDigestWeightedArgMax" +"quantileTDigestWeightedArgMin" +"quantileTDigestWeightedArray" +"quantileTDigestWeightedDistinct" +"quantileTDigestWeightedForEach" +"quantileTDigestWeightedIf" +"quantileTDigestWeightedMap" +"quantileTDigestWeightedMerge" +"quantileTDigestWeightedNull" +"quantileTDigestWeightedOrDefault" +"quantileTDigestWeightedOrNull" +"quantileTDigestWeightedResample" +"quantileTDigestWeightedSimpleState" +"quantileTDigestWeightedState" +"quantileTiming" +"quantileTimingArgMax" +"quantileTimingArgMin" +"quantileTimingArray" +"quantileTimingDistinct" +"quantileTimingForEach" +"quantileTimingIf" +"quantileTimingMap" +"quantileTimingMerge" +"quantileTimingNull" +"quantileTimingOrDefault" +"quantileTimingOrNull" +"quantileTimingResample" +"quantileTimingSimpleState" +"quantileTimingState" +"quantileTimingWeighted" +"quantileTimingWeightedArgMax" +"quantileTimingWeightedArgMin" +"quantileTimingWeightedArray" +"quantileTimingWeightedDistinct" +"quantileTimingWeightedForEach" +"quantileTimingWeightedIf" +"quantileTimingWeightedMap" +"quantileTimingWeightedMerge" +"quantileTimingWeightedNull" +"quantileTimingWeightedOrDefault" +"quantileTimingWeightedOrNull" +"quantileTimingWeightedResample" +"quantileTimingWeightedSimpleState" +"quantileTimingWeightedState" +"quantiles" +"quantilesArgMax" +"quantilesArgMin" +"quantilesArray" +"quantilesBFloat16" +"quantilesBFloat16ArgMax" +"quantilesBFloat16ArgMin" +"quantilesBFloat16Array" +"quantilesBFloat16Distinct" +"quantilesBFloat16ForEach" +"quantilesBFloat16If" +"quantilesBFloat16Map" +"quantilesBFloat16Merge" +"quantilesBFloat16Null" +"quantilesBFloat16OrDefault" +"quantilesBFloat16OrNull" +"quantilesBFloat16Resample" +"quantilesBFloat16SimpleState" +"quantilesBFloat16State" +"quantilesBFloat16Weighted" +"quantilesBFloat16WeightedArgMax" +"quantilesBFloat16WeightedArgMin" +"quantilesBFloat16WeightedArray" +"quantilesBFloat16WeightedDistinct" +"quantilesBFloat16WeightedForEach" +"quantilesBFloat16WeightedIf" +"quantilesBFloat16WeightedMap" +"quantilesBFloat16WeightedMerge" +"quantilesBFloat16WeightedNull" +"quantilesBFloat16WeightedOrDefault" +"quantilesBFloat16WeightedOrNull" +"quantilesBFloat16WeightedResample" +"quantilesBFloat16WeightedSimpleState" +"quantilesBFloat16WeightedState" +"quantilesDD" +"quantilesDDArgMax" +"quantilesDDArgMin" +"quantilesDDArray" +"quantilesDDDistinct" +"quantilesDDForEach" +"quantilesDDIf" +"quantilesDDMap" +"quantilesDDMerge" +"quantilesDDNull" +"quantilesDDOrDefault" +"quantilesDDOrNull" +"quantilesDDResample" +"quantilesDDSimpleState" +"quantilesDDState" +"quantilesDeterministic" +"quantilesDeterministicArgMax" +"quantilesDeterministicArgMin" +"quantilesDeterministicArray" +"quantilesDeterministicDistinct" +"quantilesDeterministicForEach" +"quantilesDeterministicIf" +"quantilesDeterministicMap" +"quantilesDeterministicMerge" +"quantilesDeterministicNull" +"quantilesDeterministicOrDefault" +"quantilesDeterministicOrNull" +"quantilesDeterministicResample" +"quantilesDeterministicSimpleState" +"quantilesDeterministicState" +"quantilesDistinct" +"quantilesExact" +"quantilesExactArgMax" +"quantilesExactArgMin" +"quantilesExactArray" +"quantilesExactDistinct" +"quantilesExactExclusive" +"quantilesExactExclusiveArgMax" +"quantilesExactExclusiveArgMin" +"quantilesExactExclusiveArray" +"quantilesExactExclusiveDistinct" +"quantilesExactExclusiveForEach" +"quantilesExactExclusiveIf" +"quantilesExactExclusiveMap" +"quantilesExactExclusiveMerge" +"quantilesExactExclusiveNull" +"quantilesExactExclusiveOrDefault" +"quantilesExactExclusiveOrNull" +"quantilesExactExclusiveResample" +"quantilesExactExclusiveSimpleState" +"quantilesExactExclusiveState" +"quantilesExactForEach" +"quantilesExactHigh" +"quantilesExactHighArgMax" +"quantilesExactHighArgMin" +"quantilesExactHighArray" +"quantilesExactHighDistinct" +"quantilesExactHighForEach" +"quantilesExactHighIf" +"quantilesExactHighMap" +"quantilesExactHighMerge" +"quantilesExactHighNull" +"quantilesExactHighOrDefault" +"quantilesExactHighOrNull" +"quantilesExactHighResample" +"quantilesExactHighSimpleState" +"quantilesExactHighState" +"quantilesExactIf" +"quantilesExactInclusive" +"quantilesExactInclusiveArgMax" +"quantilesExactInclusiveArgMin" +"quantilesExactInclusiveArray" +"quantilesExactInclusiveDistinct" +"quantilesExactInclusiveForEach" +"quantilesExactInclusiveIf" +"quantilesExactInclusiveMap" +"quantilesExactInclusiveMerge" +"quantilesExactInclusiveNull" +"quantilesExactInclusiveOrDefault" +"quantilesExactInclusiveOrNull" +"quantilesExactInclusiveResample" +"quantilesExactInclusiveSimpleState" +"quantilesExactInclusiveState" +"quantilesExactLow" +"quantilesExactLowArgMax" +"quantilesExactLowArgMin" +"quantilesExactLowArray" +"quantilesExactLowDistinct" +"quantilesExactLowForEach" +"quantilesExactLowIf" +"quantilesExactLowMap" +"quantilesExactLowMerge" +"quantilesExactLowNull" +"quantilesExactLowOrDefault" +"quantilesExactLowOrNull" +"quantilesExactLowResample" +"quantilesExactLowSimpleState" +"quantilesExactLowState" +"quantilesExactMap" +"quantilesExactMerge" +"quantilesExactNull" +"quantilesExactOrDefault" +"quantilesExactOrNull" +"quantilesExactResample" +"quantilesExactSimpleState" +"quantilesExactState" +"quantilesExactWeighted" +"quantilesExactWeightedArgMax" +"quantilesExactWeightedArgMin" +"quantilesExactWeightedArray" +"quantilesExactWeightedDistinct" +"quantilesExactWeightedForEach" +"quantilesExactWeightedIf" +"quantilesExactWeightedMap" +"quantilesExactWeightedMerge" +"quantilesExactWeightedNull" +"quantilesExactWeightedOrDefault" +"quantilesExactWeightedOrNull" +"quantilesExactWeightedResample" +"quantilesExactWeightedSimpleState" +"quantilesExactWeightedState" +"quantilesForEach" +"quantilesGK" +"quantilesGKArgMax" +"quantilesGKArgMin" +"quantilesGKArray" +"quantilesGKDistinct" +"quantilesGKForEach" +"quantilesGKIf" +"quantilesGKMap" +"quantilesGKMerge" +"quantilesGKNull" +"quantilesGKOrDefault" +"quantilesGKOrNull" +"quantilesGKResample" +"quantilesGKSimpleState" +"quantilesGKState" +"quantilesIf" +"quantilesInterpolatedWeighted" +"quantilesInterpolatedWeightedArgMax" +"quantilesInterpolatedWeightedArgMin" +"quantilesInterpolatedWeightedArray" +"quantilesInterpolatedWeightedDistinct" +"quantilesInterpolatedWeightedForEach" +"quantilesInterpolatedWeightedIf" +"quantilesInterpolatedWeightedMap" +"quantilesInterpolatedWeightedMerge" +"quantilesInterpolatedWeightedNull" +"quantilesInterpolatedWeightedOrDefault" +"quantilesInterpolatedWeightedOrNull" +"quantilesInterpolatedWeightedResample" +"quantilesInterpolatedWeightedSimpleState" +"quantilesInterpolatedWeightedState" +"quantilesMap" +"quantilesMerge" +"quantilesNull" +"quantilesOrDefault" +"quantilesOrNull" +"quantilesResample" +"quantilesSimpleState" +"quantilesState" +"quantilesTDigest" +"quantilesTDigestArgMax" +"quantilesTDigestArgMin" +"quantilesTDigestArray" +"quantilesTDigestDistinct" +"quantilesTDigestForEach" +"quantilesTDigestIf" +"quantilesTDigestMap" +"quantilesTDigestMerge" +"quantilesTDigestNull" +"quantilesTDigestOrDefault" +"quantilesTDigestOrNull" +"quantilesTDigestResample" +"quantilesTDigestSimpleState" +"quantilesTDigestState" +"quantilesTDigestWeighted" +"quantilesTDigestWeightedArgMax" +"quantilesTDigestWeightedArgMin" +"quantilesTDigestWeightedArray" +"quantilesTDigestWeightedDistinct" +"quantilesTDigestWeightedForEach" +"quantilesTDigestWeightedIf" +"quantilesTDigestWeightedMap" +"quantilesTDigestWeightedMerge" +"quantilesTDigestWeightedNull" +"quantilesTDigestWeightedOrDefault" +"quantilesTDigestWeightedOrNull" +"quantilesTDigestWeightedResample" +"quantilesTDigestWeightedSimpleState" +"quantilesTDigestWeightedState" +"quantilesTiming" +"quantilesTimingArgMax" +"quantilesTimingArgMin" +"quantilesTimingArray" +"quantilesTimingDistinct" +"quantilesTimingForEach" +"quantilesTimingIf" +"quantilesTimingMap" +"quantilesTimingMerge" +"quantilesTimingNull" +"quantilesTimingOrDefault" +"quantilesTimingOrNull" +"quantilesTimingResample" +"quantilesTimingSimpleState" +"quantilesTimingState" +"quantilesTimingWeighted" +"quantilesTimingWeightedArgMax" +"quantilesTimingWeightedArgMin" +"quantilesTimingWeightedArray" +"quantilesTimingWeightedDistinct" +"quantilesTimingWeightedForEach" +"quantilesTimingWeightedIf" +"quantilesTimingWeightedMap" +"quantilesTimingWeightedMerge" +"quantilesTimingWeightedNull" +"quantilesTimingWeightedOrDefault" +"quantilesTimingWeightedOrNull" +"quantilesTimingWeightedResample" +"quantilesTimingWeightedSimpleState" +"quantilesTimingWeightedState" +"queryID" +"queryString" +"queryStringAndFragment" +"query_id" +"radians" +"rand" +"rand32" +"rand64" +"randBernoulli" +"randBinomial" +"randCanonical" +"randChiSquared" +"randConstant" +"randExponential" +"randFisherF" +"randLogNormal" +"randNegativeBinomial" +"randNormal" +"randPoisson" +"randStudentT" +"randUniform" +"randomFixedString" +"randomPrintableASCII" +"randomString" +"randomStringUTF8" +"range" +"rank" +"rankArgMax" +"rankArgMin" +"rankArray" +"rankCorr" +"rankCorrArgMax" +"rankCorrArgMin" +"rankCorrArray" +"rankCorrDistinct" +"rankCorrForEach" +"rankCorrIf" +"rankCorrMap" +"rankCorrMerge" +"rankCorrNull" +"rankCorrOrDefault" +"rankCorrOrNull" +"rankCorrResample" +"rankCorrSimpleState" +"rankCorrState" +"rankDistinct" +"rankForEach" +"rankIf" +"rankMap" +"rankMerge" +"rankNull" +"rankOrDefault" +"rankOrNull" +"rankResample" +"rankSimpleState" +"rankState" +"readWKTLineString" +"readWKTMultiLineString" +"readWKTMultiPolygon" +"readWKTPoint" +"readWKTPolygon" +"readWKTRing" +"regexpExtract" +"regexpQuoteMeta" +"regionHierarchy" +"regionIn" +"regionToArea" +"regionToCity" +"regionToContinent" +"regionToCountry" +"regionToDistrict" +"regionToName" +"regionToPopulation" +"regionToTopContinent" +"reinterpret" +"reinterpretAsDate" +"reinterpretAsDateTime" +"reinterpretAsFixedString" +"reinterpretAsFloat32" +"reinterpretAsFloat64" +"reinterpretAsInt128" +"reinterpretAsInt16" +"reinterpretAsInt256" +"reinterpretAsInt32" +"reinterpretAsInt64" +"reinterpretAsInt8" +"reinterpretAsString" +"reinterpretAsUInt128" +"reinterpretAsUInt16" +"reinterpretAsUInt256" +"reinterpretAsUInt32" +"reinterpretAsUInt64" +"reinterpretAsUInt8" +"reinterpretAsUUID" +"repeat" +"replace" +"replaceAll" +"replaceOne" +"replaceRegexpAll" +"replaceRegexpOne" +"replicate" +"retention" +"retentionArgMax" +"retentionArgMin" +"retentionArray" +"retentionDistinct" +"retentionForEach" +"retentionIf" +"retentionMap" +"retentionMerge" +"retentionNull" +"retentionOrDefault" +"retentionOrNull" +"retentionResample" +"retentionSimpleState" +"retentionState" +"reverse" +"reverseUTF8" +"revision" +"right" +"rightPad" +"rightPadUTF8" +"rightUTF8" +"round" +"roundAge" +"roundBankers" +"roundDown" +"roundDuration" +"roundToExp2" +"rowNumberInAllBlocks" +"rowNumberInBlock" +"row_number" +"row_numberArgMax" +"row_numberArgMin" +"row_numberArray" +"row_numberDistinct" +"row_numberForEach" +"row_numberIf" +"row_numberMap" +"row_numberMerge" +"row_numberNull" +"row_numberOrDefault" +"row_numberOrNull" +"row_numberResample" +"row_numberSimpleState" +"row_numberState" +"rpad" +"rtrim" +"runningAccumulate" +"runningConcurrency" +"runningDifference" +"runningDifferenceStartingWithFirstValue" +"s2CapContains" +"s2CapUnion" +"s2CellsIntersect" +"s2GetNeighbors" +"s2RectAdd" +"s2RectContains" +"s2RectIntersection" +"s2RectUnion" +"s2ToGeo" +"scalarProduct" +"sequenceCount" +"sequenceCountArgMax" +"sequenceCountArgMin" +"sequenceCountArray" +"sequenceCountDistinct" +"sequenceCountForEach" +"sequenceCountIf" +"sequenceCountMap" +"sequenceCountMerge" +"sequenceCountNull" +"sequenceCountOrDefault" +"sequenceCountOrNull" +"sequenceCountResample" +"sequenceCountSimpleState" +"sequenceCountState" +"sequenceMatch" +"sequenceMatchArgMax" +"sequenceMatchArgMin" +"sequenceMatchArray" +"sequenceMatchDistinct" +"sequenceMatchForEach" +"sequenceMatchIf" +"sequenceMatchMap" +"sequenceMatchMerge" +"sequenceMatchNull" +"sequenceMatchOrDefault" +"sequenceMatchOrNull" +"sequenceMatchResample" +"sequenceMatchSimpleState" +"sequenceMatchState" +"sequenceNextNode" +"sequenceNextNodeArgMax" +"sequenceNextNodeArgMin" +"sequenceNextNodeArray" +"sequenceNextNodeDistinct" +"sequenceNextNodeForEach" +"sequenceNextNodeIf" +"sequenceNextNodeMap" +"sequenceNextNodeMerge" +"sequenceNextNodeNull" +"sequenceNextNodeOrDefault" +"sequenceNextNodeOrNull" +"sequenceNextNodeResample" +"sequenceNextNodeSimpleState" +"sequenceNextNodeState" +"seriesDecomposeSTL" +"seriesOutliersDetectTukey" +"seriesPeriodDetectFFT" +"serverTimeZone" +"serverTimezone" +"serverUUID" +"shardCount" +"shardNum" +"showCertificate" +"sigmoid" +"sign" +"simpleJSONExtractBool" +"simpleJSONExtractFloat" +"simpleJSONExtractInt" +"simpleJSONExtractRaw" +"simpleJSONExtractString" +"simpleJSONExtractUInt" +"simpleJSONHas" +"simpleLinearRegression" +"simpleLinearRegressionArgMax" +"simpleLinearRegressionArgMin" +"simpleLinearRegressionArray" +"simpleLinearRegressionDistinct" +"simpleLinearRegressionForEach" +"simpleLinearRegressionIf" +"simpleLinearRegressionMap" +"simpleLinearRegressionMerge" +"simpleLinearRegressionNull" +"simpleLinearRegressionOrDefault" +"simpleLinearRegressionOrNull" +"simpleLinearRegressionResample" +"simpleLinearRegressionSimpleState" +"simpleLinearRegressionState" +"sin" +"singleValueOrNull" +"singleValueOrNullArgMax" +"singleValueOrNullArgMin" +"singleValueOrNullArray" +"singleValueOrNullDistinct" +"singleValueOrNullForEach" +"singleValueOrNullIf" +"singleValueOrNullMap" +"singleValueOrNullMerge" +"singleValueOrNullNull" +"singleValueOrNullOrDefault" +"singleValueOrNullOrNull" +"singleValueOrNullResample" +"singleValueOrNullSimpleState" +"singleValueOrNullState" +"sinh" +"sipHash128" +"sipHash128Keyed" +"sipHash128Reference" +"sipHash128ReferenceKeyed" +"sipHash64" +"sipHash64Keyed" +"skewPop" +"skewPopArgMax" +"skewPopArgMin" +"skewPopArray" +"skewPopDistinct" +"skewPopForEach" +"skewPopIf" +"skewPopMap" +"skewPopMerge" +"skewPopNull" +"skewPopOrDefault" +"skewPopOrNull" +"skewPopResample" +"skewPopSimpleState" +"skewPopState" +"skewSamp" +"skewSampArgMax" +"skewSampArgMin" +"skewSampArray" +"skewSampDistinct" +"skewSampForEach" +"skewSampIf" +"skewSampMap" +"skewSampMerge" +"skewSampNull" +"skewSampOrDefault" +"skewSampOrNull" +"skewSampResample" +"skewSampSimpleState" +"skewSampState" +"sleep" +"sleepEachRow" +"snowflakeIDToDateTime" +"snowflakeIDToDateTime64" +"snowflakeToDateTime" +"snowflakeToDateTime64" +"soundex" +"space" +"sparkBar" +"sparkBarArgMax" +"sparkBarArgMin" +"sparkBarArray" +"sparkBarDistinct" +"sparkBarForEach" +"sparkBarIf" +"sparkBarMap" +"sparkBarMerge" +"sparkBarNull" +"sparkBarOrDefault" +"sparkBarOrNull" +"sparkBarResample" +"sparkBarSimpleState" +"sparkBarState" +"sparkbar" +"sparkbarArgMax" +"sparkbarArgMin" +"sparkbarArray" +"sparkbarDistinct" +"sparkbarForEach" +"sparkbarIf" +"sparkbarMap" +"sparkbarMerge" +"sparkbarNull" +"sparkbarOrDefault" +"sparkbarOrNull" +"sparkbarResample" +"sparkbarSimpleState" +"sparkbarState" +"splitByAlpha" +"splitByChar" +"splitByNonAlpha" +"splitByRegexp" +"splitByString" +"splitByWhitespace" +"sqid" +"sqidDecode" +"sqidEncode" +"sqrt" +"startsWith" +"startsWithUTF8" +"stddevPop" +"stddevPopArgMax" +"stddevPopArgMin" +"stddevPopArray" +"stddevPopDistinct" +"stddevPopForEach" +"stddevPopIf" +"stddevPopMap" +"stddevPopMerge" +"stddevPopNull" +"stddevPopOrDefault" +"stddevPopOrNull" +"stddevPopResample" +"stddevPopSimpleState" +"stddevPopStable" +"stddevPopStableArgMax" +"stddevPopStableArgMin" +"stddevPopStableArray" +"stddevPopStableDistinct" +"stddevPopStableForEach" +"stddevPopStableIf" +"stddevPopStableMap" +"stddevPopStableMerge" +"stddevPopStableNull" +"stddevPopStableOrDefault" +"stddevPopStableOrNull" +"stddevPopStableResample" +"stddevPopStableSimpleState" +"stddevPopStableState" +"stddevPopState" +"stddevSamp" +"stddevSampArgMax" +"stddevSampArgMin" +"stddevSampArray" +"stddevSampDistinct" +"stddevSampForEach" +"stddevSampIf" +"stddevSampMap" +"stddevSampMerge" +"stddevSampNull" +"stddevSampOrDefault" +"stddevSampOrNull" +"stddevSampResample" +"stddevSampSimpleState" +"stddevSampStable" +"stddevSampStableArgMax" +"stddevSampStableArgMin" +"stddevSampStableArray" +"stddevSampStableDistinct" +"stddevSampStableForEach" +"stddevSampStableIf" +"stddevSampStableMap" +"stddevSampStableMerge" +"stddevSampStableNull" +"stddevSampStableOrDefault" +"stddevSampStableOrNull" +"stddevSampStableResample" +"stddevSampStableSimpleState" +"stddevSampStableState" +"stddevSampState" +"stem" +"stochasticLinearRegression" +"stochasticLinearRegressionArgMax" +"stochasticLinearRegressionArgMin" +"stochasticLinearRegressionArray" +"stochasticLinearRegressionDistinct" +"stochasticLinearRegressionForEach" +"stochasticLinearRegressionIf" +"stochasticLinearRegressionMap" +"stochasticLinearRegressionMerge" +"stochasticLinearRegressionNull" +"stochasticLinearRegressionOrDefault" +"stochasticLinearRegressionOrNull" +"stochasticLinearRegressionResample" +"stochasticLinearRegressionSimpleState" +"stochasticLinearRegressionState" +"stochasticLogisticRegression" +"stochasticLogisticRegressionArgMax" +"stochasticLogisticRegressionArgMin" +"stochasticLogisticRegressionArray" +"stochasticLogisticRegressionDistinct" +"stochasticLogisticRegressionForEach" +"stochasticLogisticRegressionIf" +"stochasticLogisticRegressionMap" +"stochasticLogisticRegressionMerge" +"stochasticLogisticRegressionNull" +"stochasticLogisticRegressionOrDefault" +"stochasticLogisticRegressionOrNull" +"stochasticLogisticRegressionResample" +"stochasticLogisticRegressionSimpleState" +"stochasticLogisticRegressionState" +"str_to_date" +"str_to_map" +"stringJaccardIndex" +"stringJaccardIndexUTF8" +"stringToH3" +"structureToCapnProtoSchema" +"structureToProtobufSchema" +"studentTTest" +"studentTTestArgMax" +"studentTTestArgMin" +"studentTTestArray" +"studentTTestDistinct" +"studentTTestForEach" +"studentTTestIf" +"studentTTestMap" +"studentTTestMerge" +"studentTTestNull" +"studentTTestOrDefault" +"studentTTestOrNull" +"studentTTestResample" +"studentTTestSimpleState" +"studentTTestState" +"subBitmap" +"subDate" +"substr" +"substring" +"substringIndex" +"substringIndexUTF8" +"substringUTF8" +"subtractDays" +"subtractHours" +"subtractInterval" +"subtractMicroseconds" +"subtractMilliseconds" +"subtractMinutes" +"subtractMonths" +"subtractNanoseconds" +"subtractQuarters" +"subtractSeconds" +"subtractTupleOfIntervals" +"subtractWeeks" +"subtractYears" +"sum" +"sumArgMax" +"sumArgMin" +"sumArray" +"sumCount" +"sumCountArgMax" +"sumCountArgMin" +"sumCountArray" +"sumCountDistinct" +"sumCountForEach" +"sumCountIf" +"sumCountMap" +"sumCountMerge" +"sumCountNull" +"sumCountOrDefault" +"sumCountOrNull" +"sumCountResample" +"sumCountSimpleState" +"sumCountState" +"sumDistinct" +"sumForEach" +"sumIf" +"sumKahan" +"sumKahanArgMax" +"sumKahanArgMin" +"sumKahanArray" +"sumKahanDistinct" +"sumKahanForEach" +"sumKahanIf" +"sumKahanMap" +"sumKahanMerge" +"sumKahanNull" +"sumKahanOrDefault" +"sumKahanOrNull" +"sumKahanResample" +"sumKahanSimpleState" +"sumKahanState" +"sumMap" +"sumMapFiltered" +"sumMapFilteredArgMax" +"sumMapFilteredArgMin" +"sumMapFilteredArray" +"sumMapFilteredDistinct" +"sumMapFilteredForEach" +"sumMapFilteredIf" +"sumMapFilteredMap" +"sumMapFilteredMerge" +"sumMapFilteredNull" +"sumMapFilteredOrDefault" +"sumMapFilteredOrNull" +"sumMapFilteredResample" +"sumMapFilteredSimpleState" +"sumMapFilteredState" +"sumMapFilteredWithOverflow" +"sumMapFilteredWithOverflowArgMax" +"sumMapFilteredWithOverflowArgMin" +"sumMapFilteredWithOverflowArray" +"sumMapFilteredWithOverflowDistinct" +"sumMapFilteredWithOverflowForEach" +"sumMapFilteredWithOverflowIf" +"sumMapFilteredWithOverflowMap" +"sumMapFilteredWithOverflowMerge" +"sumMapFilteredWithOverflowNull" +"sumMapFilteredWithOverflowOrDefault" +"sumMapFilteredWithOverflowOrNull" +"sumMapFilteredWithOverflowResample" +"sumMapFilteredWithOverflowSimpleState" +"sumMapFilteredWithOverflowState" +"sumMapWithOverflow" +"sumMapWithOverflowArgMax" +"sumMapWithOverflowArgMin" +"sumMapWithOverflowArray" +"sumMapWithOverflowDistinct" +"sumMapWithOverflowForEach" +"sumMapWithOverflowIf" +"sumMapWithOverflowMap" +"sumMapWithOverflowMerge" +"sumMapWithOverflowNull" +"sumMapWithOverflowOrDefault" +"sumMapWithOverflowOrNull" +"sumMapWithOverflowResample" +"sumMapWithOverflowSimpleState" +"sumMapWithOverflowState" +"sumMappedArrays" +"sumMappedArraysArgMax" +"sumMappedArraysArgMin" +"sumMappedArraysArray" +"sumMappedArraysDistinct" +"sumMappedArraysForEach" +"sumMappedArraysIf" +"sumMappedArraysMap" +"sumMappedArraysMerge" +"sumMappedArraysNull" +"sumMappedArraysOrDefault" +"sumMappedArraysOrNull" +"sumMappedArraysResample" +"sumMappedArraysSimpleState" +"sumMappedArraysState" +"sumMerge" +"sumNull" +"sumOrDefault" +"sumOrNull" +"sumResample" +"sumSimpleState" +"sumState" +"sumWithOverflow" +"sumWithOverflowArgMax" +"sumWithOverflowArgMin" +"sumWithOverflowArray" +"sumWithOverflowDistinct" +"sumWithOverflowForEach" +"sumWithOverflowIf" +"sumWithOverflowMap" +"sumWithOverflowMerge" +"sumWithOverflowNull" +"sumWithOverflowOrDefault" +"sumWithOverflowOrNull" +"sumWithOverflowResample" +"sumWithOverflowSimpleState" +"sumWithOverflowState" +"svg" +"synonyms" +"tan" +"tanh" +"tcpPort" +"tgamma" +"theilsU" +"theilsUArgMax" +"theilsUArgMin" +"theilsUArray" +"theilsUDistinct" +"theilsUForEach" +"theilsUIf" +"theilsUMap" +"theilsUMerge" +"theilsUNull" +"theilsUOrDefault" +"theilsUOrNull" +"theilsUResample" +"theilsUSimpleState" +"theilsUState" +"throwIf" +"tid" +"timeDiff" +"timeSlot" +"timeSlots" +"timeZone" +"timeZoneOf" +"timeZoneOffset" +"timestamp" +"timestampDiff" +"timestamp_diff" +"timezone" +"timezoneOf" +"timezoneOffset" +"toBool" +"toColumnTypeName" +"toDate" +"toDate32" +"toDate32OrDefault" +"toDate32OrNull" +"toDate32OrZero" +"toDateOrDefault" +"toDateOrNull" +"toDateOrZero" +"toDateTime" +"toDateTime32" +"toDateTime64" +"toDateTime64OrDefault" +"toDateTime64OrNull" +"toDateTime64OrZero" +"toDateTimeOrDefault" +"toDateTimeOrNull" +"toDateTimeOrZero" +"toDayOfMonth" +"toDayOfWeek" +"toDayOfYear" +"toDaysSinceYearZero" +"toDecimal128" +"toDecimal128OrDefault" +"toDecimal128OrNull" +"toDecimal128OrZero" +"toDecimal256" +"toDecimal256OrDefault" +"toDecimal256OrNull" +"toDecimal256OrZero" +"toDecimal32" +"toDecimal32OrDefault" +"toDecimal32OrNull" +"toDecimal32OrZero" +"toDecimal64" +"toDecimal64OrDefault" +"toDecimal64OrNull" +"toDecimal64OrZero" +"toDecimalString" +"toFixedString" +"toFloat32" +"toFloat32OrDefault" +"toFloat32OrNull" +"toFloat32OrZero" +"toFloat64" +"toFloat64OrDefault" +"toFloat64OrNull" +"toFloat64OrZero" +"toHour" +"toIPv4" +"toIPv4OrDefault" +"toIPv4OrNull" +"toIPv4OrZero" +"toIPv6" +"toIPv6OrDefault" +"toIPv6OrNull" +"toIPv6OrZero" +"toISOWeek" +"toISOYear" +"toInt128" +"toInt128OrDefault" +"toInt128OrNull" +"toInt128OrZero" +"toInt16" +"toInt16OrDefault" +"toInt16OrNull" +"toInt16OrZero" +"toInt256" +"toInt256OrDefault" +"toInt256OrNull" +"toInt256OrZero" +"toInt32" +"toInt32OrDefault" +"toInt32OrNull" +"toInt32OrZero" +"toInt64" +"toInt64OrDefault" +"toInt64OrNull" +"toInt64OrZero" +"toInt8" +"toInt8OrDefault" +"toInt8OrNull" +"toInt8OrZero" +"toIntervalDay" +"toIntervalHour" +"toIntervalMicrosecond" +"toIntervalMillisecond" +"toIntervalMinute" +"toIntervalMonth" +"toIntervalNanosecond" +"toIntervalQuarter" +"toIntervalSecond" +"toIntervalWeek" +"toIntervalYear" +"toJSONString" +"toLastDayOfMonth" +"toLastDayOfWeek" +"toLowCardinality" +"toMillisecond" +"toMinute" +"toModifiedJulianDay" +"toModifiedJulianDayOrNull" +"toMonday" +"toMonth" +"toNullable" +"toQuarter" +"toRelativeDayNum" +"toRelativeHourNum" +"toRelativeMinuteNum" +"toRelativeMonthNum" +"toRelativeQuarterNum" +"toRelativeSecondNum" +"toRelativeWeekNum" +"toRelativeYearNum" +"toSecond" +"toStartOfDay" +"toStartOfFifteenMinutes" +"toStartOfFiveMinute" +"toStartOfFiveMinutes" +"toStartOfHour" +"toStartOfISOYear" +"toStartOfInterval" +"toStartOfMicrosecond" +"toStartOfMillisecond" +"toStartOfMinute" +"toStartOfMonth" +"toStartOfNanosecond" +"toStartOfQuarter" +"toStartOfSecond" +"toStartOfTenMinutes" +"toStartOfWeek" +"toStartOfYear" +"toString" +"toStringCutToZero" +"toTime" +"toTimeZone" +"toTimezone" +"toTypeName" +"toUInt128" +"toUInt128OrDefault" +"toUInt128OrNull" +"toUInt128OrZero" +"toUInt16" +"toUInt16OrDefault" +"toUInt16OrNull" +"toUInt16OrZero" +"toUInt256" +"toUInt256OrDefault" +"toUInt256OrNull" +"toUInt256OrZero" +"toUInt32" +"toUInt32OrDefault" +"toUInt32OrNull" +"toUInt32OrZero" +"toUInt64" +"toUInt64OrDefault" +"toUInt64OrNull" +"toUInt64OrZero" +"toUInt8" +"toUInt8OrDefault" +"toUInt8OrNull" +"toUInt8OrZero" +"toUTCTimestamp" +"toUUID" +"toUUIDOrDefault" +"toUUIDOrNull" +"toUUIDOrZero" +"toUnixTimestamp" +"toUnixTimestamp64Micro" +"toUnixTimestamp64Milli" +"toUnixTimestamp64Nano" +"toValidUTF8" +"toWeek" +"toYYYYMM" +"toYYYYMMDD" +"toYYYYMMDDhhmmss" +"toYear" +"toYearWeek" +"to_utc_timestamp" +"today" +"tokens" +"topK" +"topKArgMax" +"topKArgMin" +"topKArray" +"topKDistinct" +"topKForEach" +"topKIf" +"topKMap" +"topKMerge" +"topKNull" +"topKOrDefault" +"topKOrNull" +"topKResample" +"topKSimpleState" +"topKState" +"topKWeighted" +"topKWeightedArgMax" +"topKWeightedArgMin" +"topKWeightedArray" +"topKWeightedDistinct" +"topKWeightedForEach" +"topKWeightedIf" +"topKWeightedMap" +"topKWeightedMerge" +"topKWeightedNull" +"topKWeightedOrDefault" +"topKWeightedOrNull" +"topKWeightedResample" +"topKWeightedSimpleState" +"topKWeightedState" +"topLevelDomain" +"topLevelDomainRFC" +"transactionID" +"transactionLatestSnapshot" +"transactionOldestSnapshot" +"transform" +"translate" +"translateUTF8" +"trim" +"trimBoth" +"trimLeft" +"trimRight" +"trunc" +"truncate" +"tryBase58Decode" +"tryBase64Decode" +"tryBase64URLDecode" +"tryDecrypt" +"tryIdnaEncode" +"tryPunycodeDecode" +"tumble" +"tumbleEnd" +"tumbleStart" +"tuple" +"tupleConcat" +"tupleDivide" +"tupleDivideByNumber" +"tupleElement" +"tupleHammingDistance" +"tupleIntDiv" +"tupleIntDivByNumber" +"tupleIntDivOrZero" +"tupleIntDivOrZeroByNumber" +"tupleMinus" +"tupleModulo" +"tupleModuloByNumber" +"tupleMultiply" +"tupleMultiplyByNumber" +"tupleNames" +"tupleNegate" +"tuplePlus" +"tupleToNameValuePairs" +"ucase" +"unbin" +"unhex" +"uniq" +"uniqArgMax" +"uniqArgMin" +"uniqArray" +"uniqCombined" +"uniqCombined64" +"uniqCombined64ArgMax" +"uniqCombined64ArgMin" +"uniqCombined64Array" +"uniqCombined64Distinct" +"uniqCombined64ForEach" +"uniqCombined64If" +"uniqCombined64Map" +"uniqCombined64Merge" +"uniqCombined64Null" +"uniqCombined64OrDefault" +"uniqCombined64OrNull" +"uniqCombined64Resample" +"uniqCombined64SimpleState" +"uniqCombined64State" +"uniqCombinedArgMax" +"uniqCombinedArgMin" +"uniqCombinedArray" +"uniqCombinedDistinct" +"uniqCombinedForEach" +"uniqCombinedIf" +"uniqCombinedMap" +"uniqCombinedMerge" +"uniqCombinedNull" +"uniqCombinedOrDefault" +"uniqCombinedOrNull" +"uniqCombinedResample" +"uniqCombinedSimpleState" +"uniqCombinedState" +"uniqDistinct" +"uniqExact" +"uniqExactArgMax" +"uniqExactArgMin" +"uniqExactArray" +"uniqExactDistinct" +"uniqExactForEach" +"uniqExactIf" +"uniqExactMap" +"uniqExactMerge" +"uniqExactNull" +"uniqExactOrDefault" +"uniqExactOrNull" +"uniqExactResample" +"uniqExactSimpleState" +"uniqExactState" +"uniqForEach" +"uniqHLL12" +"uniqHLL12ArgMax" +"uniqHLL12ArgMin" +"uniqHLL12Array" +"uniqHLL12Distinct" +"uniqHLL12ForEach" +"uniqHLL12If" +"uniqHLL12Map" +"uniqHLL12Merge" +"uniqHLL12Null" +"uniqHLL12OrDefault" +"uniqHLL12OrNull" +"uniqHLL12Resample" +"uniqHLL12SimpleState" +"uniqHLL12State" +"uniqIf" +"uniqMap" +"uniqMerge" +"uniqNull" +"uniqOrDefault" +"uniqOrNull" +"uniqResample" +"uniqSimpleState" +"uniqState" +"uniqTheta" +"uniqThetaArgMax" +"uniqThetaArgMin" +"uniqThetaArray" +"uniqThetaDistinct" +"uniqThetaForEach" +"uniqThetaIf" +"uniqThetaIntersect" +"uniqThetaMap" +"uniqThetaMerge" +"uniqThetaNot" +"uniqThetaNull" +"uniqThetaOrDefault" +"uniqThetaOrNull" +"uniqThetaResample" +"uniqThetaSimpleState" +"uniqThetaState" +"uniqThetaUnion" +"uniqUpTo" +"uniqUpToArgMax" +"uniqUpToArgMin" +"uniqUpToArray" +"uniqUpToDistinct" +"uniqUpToForEach" +"uniqUpToIf" +"uniqUpToMap" +"uniqUpToMerge" +"uniqUpToNull" +"uniqUpToOrDefault" +"uniqUpToOrNull" +"uniqUpToResample" +"uniqUpToSimpleState" +"uniqUpToState" +"upper" +"upperUTF8" +"uptime" +"user" +"validateNestedArraySizes" +"varPop" +"varPopArgMax" +"varPopArgMin" +"varPopArray" +"varPopDistinct" +"varPopForEach" +"varPopIf" +"varPopMap" +"varPopMerge" +"varPopNull" +"varPopOrDefault" +"varPopOrNull" +"varPopResample" +"varPopSimpleState" +"varPopStable" +"varPopStableArgMax" +"varPopStableArgMin" +"varPopStableArray" +"varPopStableDistinct" +"varPopStableForEach" +"varPopStableIf" +"varPopStableMap" +"varPopStableMerge" +"varPopStableNull" +"varPopStableOrDefault" +"varPopStableOrNull" +"varPopStableResample" +"varPopStableSimpleState" +"varPopStableState" +"varPopState" +"varSamp" +"varSampArgMax" +"varSampArgMin" +"varSampArray" +"varSampDistinct" +"varSampForEach" +"varSampIf" +"varSampMap" +"varSampMerge" +"varSampNull" +"varSampOrDefault" +"varSampOrNull" +"varSampResample" +"varSampSimpleState" +"varSampStable" +"varSampStableArgMax" +"varSampStableArgMin" +"varSampStableArray" +"varSampStableDistinct" +"varSampStableForEach" +"varSampStableIf" +"varSampStableMap" +"varSampStableMerge" +"varSampStableNull" +"varSampStableOrDefault" +"varSampStableOrNull" +"varSampStableResample" +"varSampStableSimpleState" +"varSampStableState" +"varSampState" +"variantElement" +"variantType" +"vectorDifference" +"vectorSum" +"version" +"visibleWidth" +"visitParamExtractBool" +"visitParamExtractFloat" +"visitParamExtractInt" +"visitParamExtractRaw" +"visitParamExtractString" +"visitParamExtractUInt" +"visitParamHas" +"week" +"welchTTest" +"welchTTestArgMax" +"welchTTestArgMin" +"welchTTestArray" +"welchTTestDistinct" +"welchTTestForEach" +"welchTTestIf" +"welchTTestMap" +"welchTTestMerge" +"welchTTestNull" +"welchTTestOrDefault" +"welchTTestOrNull" +"welchTTestResample" +"welchTTestSimpleState" +"welchTTestState" +"widthBucket" +"width_bucket" +"windowFunnel" +"windowFunnelArgMax" +"windowFunnelArgMin" +"windowFunnelArray" +"windowFunnelDistinct" +"windowFunnelForEach" +"windowFunnelIf" +"windowFunnelMap" +"windowFunnelMerge" +"windowFunnelNull" +"windowFunnelOrDefault" +"windowFunnelOrNull" +"windowFunnelResample" +"windowFunnelSimpleState" +"windowFunnelState" +"windowID" +"wkt" +"wordShingleMinHash" +"wordShingleMinHashArg" +"wordShingleMinHashArgCaseInsensitive" +"wordShingleMinHashArgCaseInsensitiveUTF8" +"wordShingleMinHashArgUTF8" +"wordShingleMinHashCaseInsensitive" +"wordShingleMinHashCaseInsensitiveUTF8" +"wordShingleMinHashUTF8" +"wordShingleSimHash" +"wordShingleSimHashCaseInsensitive" +"wordShingleSimHashCaseInsensitiveUTF8" +"wordShingleSimHashUTF8" +"wyHash64" +"xor" +"xxHash32" +"xxHash64" +"xxh3" +"yandexConsistentHash" +"yearweek" +"yesterday" +"zookeeperSessionUptime" From 04286bc270f9f473a07bc4ae27ae61d96256f775 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Aug 2024 14:45:05 +0000 Subject: [PATCH 0890/1722] Add status to PipelineExecutor. Verify status of pusing pipeline. --- programs/obfuscator/Obfuscator.cpp | 2 +- src/Client/LocalConnection.cpp | 8 +++- src/Interpreters/SystemLog.cpp | 2 +- src/Processors/Executors/ExecutingGraph.cpp | 19 ++++----- src/Processors/Executors/ExecutingGraph.h | 11 ++++- src/Processors/Executors/PipelineExecutor.cpp | 31 +++++++++----- src/Processors/Executors/PipelineExecutor.h | 20 +++++++++- .../PushingAsyncPipelineExecutor.cpp | 21 +++++++--- .../Executors/PushingAsyncPipelineExecutor.h | 7 +++- .../Executors/PushingPipelineExecutor.cpp | 40 ++++++++++++++----- .../Executors/PushingPipelineExecutor.h | 9 +++-- .../Transforms/CreatingSetsTransform.cpp | 3 +- src/Server/GRPCServer.cpp | 3 +- src/Server/TCPHandler.cpp | 13 ++++-- src/Storages/Distributed/DistributedSink.cpp | 6 ++- src/Storages/StorageBuffer.cpp | 3 +- src/Storages/tests/gtest_storage_log.cpp | 2 +- ...221_insert_timeout_overflow_mode.reference | 2 + .../03221_insert_timeout_overflow_mode.sh | 8 ++++ 19 files changed, 153 insertions(+), 57 deletions(-) create mode 100644 tests/queries/0_stateless/03221_insert_timeout_overflow_mode.reference create mode 100755 tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 688ae1a1143..7c13215e350 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1462,7 +1462,7 @@ try while (in_executor.pull(block)) { Columns columns = obfuscator.generate(block.getColumns()); - out_executor.push(header.cloneWithColumns(columns)); + std::ignore = out_executor.push(header.cloneWithColumns(columns)); processed_rows += block.rows(); if (!silent) std::cerr << "Processed " << processed_rows << " rows\n"; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 072184e0a66..8f1e0958002 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -287,13 +287,17 @@ void LocalConnection::sendData(const Block & block, const String &, bool) if (!block) return; + bool inserted = false; if (state->pushing_async_executor) - state->pushing_async_executor->push(block); + inserted = state->pushing_async_executor->push(block); else if (state->pushing_executor) - state->pushing_executor->push(block); + inserted = state->pushing_executor->push(block); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown executor"); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); + if (send_profile_events) sendProfileEvents(); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 572481e6b12..0cad56af00a 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -556,7 +556,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, PushingPipelineExecutor executor(io.pipeline); executor.start(); - executor.push(block); + std::ignore = executor.push(block); executor.finish(); } catch (...) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 6d5b60d8159..10470325bb8 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -96,7 +96,7 @@ bool ExecutingGraph::addEdges(uint64_t node) return was_edge_added; } -bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) +ExecutingGraph::UpdateNodeStatus ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) { auto & cur_node = *nodes[pid]; Processors new_processors; @@ -108,7 +108,7 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) catch (...) { cur_node.exception = std::current_exception(); - return false; + return UpdateNodeStatus::Exception; } { @@ -118,7 +118,7 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) { for (auto & processor : new_processors) processor->cancel(); - return false; + return UpdateNodeStatus::Cancelled; } processors->insert(processors->end(), new_processors.begin(), new_processors.end()); @@ -178,7 +178,7 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) } } - return true; + return UpdateNodeStatus::Done; } void ExecutingGraph::initializeExecution(Queue & queue) @@ -213,7 +213,7 @@ void ExecutingGraph::initializeExecution(Queue & queue) } -bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue) +ExecutingGraph::UpdateNodeStatus ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue) { std::stack updated_edges; std::stack updated_processors; @@ -309,7 +309,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue catch (...) { node.exception = std::current_exception(); - return false; + return UpdateNodeStatus::Exception; } #ifndef NDEBUG @@ -386,8 +386,9 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue read_lock.unlock(); { std::unique_lock lock(nodes_mutex); - if (!expandPipeline(updated_processors, pid)) - return false; + auto status = expandPipeline(updated_processors, pid); + if (status != UpdateNodeStatus::Done) + return status; } read_lock.lock(); @@ -397,7 +398,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue } } - return true; + return UpdateNodeStatus::Done; } void ExecutingGraph::cancel(bool cancel_all_processors) diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 71dcd360a2c..e1a6ac96203 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -138,10 +138,17 @@ public: /// Traverse graph the first time to update all the childless nodes. void initializeExecution(Queue & queue); + enum class UpdateNodeStatus + { + Done, + Exception, + Cancelled, + }; + /// Update processor with pid number (call IProcessor::prepare). /// Check parents and children of current processor and push them to stacks if they also need to be updated. /// If processor wants to be expanded, lock will be upgraded to get write access to pipeline. - bool updateNode(uint64_t pid, Queue & queue, Queue & async_queue); + UpdateNodeStatus updateNode(uint64_t pid, Queue & queue, Queue & async_queue); void cancel(bool cancel_all_processors = true); @@ -155,7 +162,7 @@ private: /// Update graph after processor (pid) returned ExpandPipeline status. /// All new nodes and nodes with updated ports are pushed into stack. - bool expandPipeline(std::stack & stack, uint64_t pid); + UpdateNodeStatus expandPipeline(std::stack & stack, uint64_t pid); std::shared_ptr processors; std::vector source_processors; diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 82cad471a29..23b3a6d9f5f 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -77,9 +77,9 @@ const Processors & PipelineExecutor::getProcessors() const return graph->getProcessors(); } -void PipelineExecutor::cancel() +void PipelineExecutor::cancel(ExecutionStatus reason) { - cancelled = true; + tryUpdateExecutionStatus(ExecutionStatus::Executing, reason); finish(); graph->cancel(); } @@ -98,6 +98,11 @@ void PipelineExecutor::finish() tasks.finish(); } +bool PipelineExecutor::tryUpdateExecutionStatus(ExecutionStatus expected, ExecutionStatus desired) +{ + return execution_status.compare_exchange_strong(expected, desired); +} + void PipelineExecutor::execute(size_t num_threads, bool concurrency_control) { checkTimeLimit(); @@ -120,7 +125,7 @@ void PipelineExecutor::execute(size_t num_threads, bool concurrency_control) } catch (...) { - span.addAttribute(ExecutionStatus::fromCurrentException()); + span.addAttribute(DB::ExecutionStatus::fromCurrentException()); #ifndef NDEBUG LOG_TRACE(log, "Exception while executing query. Current state:\n{}", dumpPipeline()); @@ -169,7 +174,7 @@ bool PipelineExecutor::checkTimeLimitSoft() // We call cancel here so that all processors are notified and tasks waken up // so that the "break" is faster and doesn't wait for long events if (!continuing) - cancel(); + cancel(ExecutionStatus::CancelledByTimeout); return continuing; } @@ -195,7 +200,8 @@ void PipelineExecutor::finalizeExecution() { checkTimeLimit(); - if (cancelled) + auto status = execution_status.load(); + if (status == ExecutionStatus::CancelledByTimeout || status == ExecutionStatus::CancelledByUser) return; bool all_processors_finished = true; @@ -271,7 +277,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie break; if (!context.executeTask()) - cancel(); + cancel(ExecutionStatus::Exception); if (tasks.isFinished()) break; @@ -289,11 +295,13 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie Queue async_queue; /// Prepare processor after execution. - if (!graph->updateNode(context.getProcessorID(), queue, async_queue)) - cancel(); + auto status = graph->updateNode(context.getProcessorID(), queue, async_queue); + if (status == ExecutingGraph::UpdateNodeStatus::Exception) + cancel(ExecutionStatus::Exception); /// Push other tasks to global queue. - tasks.pushTasks(queue, async_queue, context); + if (status == ExecutingGraph::UpdateNodeStatus::Done) + tasks.pushTasks(queue, async_queue, context); } #ifndef NDEBUG @@ -309,7 +317,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie { /// spawnThreads can throw an exception, for example CANNOT_SCHEDULE_TASK. /// We should cancel execution properly before rethrow. - cancel(); + cancel(ExecutionStatus::Exception); throw; } @@ -328,6 +336,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie void PipelineExecutor::initializeExecution(size_t num_threads, bool concurrency_control) { is_execution_initialized = true; + tryUpdateExecutionStatus(ExecutionStatus::NotStarted, ExecutionStatus::Executing); size_t use_threads = num_threads; @@ -393,7 +402,7 @@ void PipelineExecutor::executeImpl(size_t num_threads, bool concurrency_control) { /// If finished_flag is not set, there was an exception. /// Cancel execution in this case. - cancel(); + cancel(ExecutionStatus::Exception); if (pool) pool->wait(); } diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index ae119355cb5..79d0a29d4e1 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -48,8 +48,20 @@ public: const Processors & getProcessors() const; + enum class ExecutionStatus + { + NotStarted, + Executing, + Finished, + Exception, + CancelledByUser, + CancelledByTimeout, + }; + /// Cancel execution. May be called from another thread. - void cancel(); + void cancel() { cancel(ExecutionStatus::CancelledByUser); } + + ExecutionStatus getExecutionStatus() const { return execution_status.load(); } /// Cancel processors which only read data from source. May be called from another thread. void cancelReading(); @@ -81,7 +93,7 @@ private: /// system.opentelemetry_span_log bool trace_processors = false; - std::atomic_bool cancelled = false; + std::atomic execution_status = ExecutionStatus::NotStarted; std::atomic_bool cancelled_reading = false; LoggerPtr log = getLogger("PipelineExecutor"); @@ -105,6 +117,10 @@ private: void executeStepImpl(size_t thread_num, std::atomic_bool * yield_flag = nullptr); void executeSingleThread(size_t thread_num); void finish(); + void cancel(ExecutionStatus reason); + + /// If execution_status == from, change it to desired. + bool tryUpdateExecutionStatus(ExecutionStatus expected, ExecutionStatus desired); String dumpPipeline() const; }; diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 830a96533ed..db5cf451c9e 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -176,7 +176,17 @@ void PushingAsyncPipelineExecutor::start() data->thread = ThreadFromGlobalPool(std::move(func)); } -void PushingAsyncPipelineExecutor::push(Chunk chunk) +static void checkExecutionStatus(PipelineExecutor::ExecutionStatus status) +{ + if (status == PipelineExecutor::ExecutionStatus::CancelledByTimeout + || status == PipelineExecutor::ExecutionStatus::CancelledByUser) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); +} + +bool PushingAsyncPipelineExecutor::push(Chunk chunk) { if (!started) start(); @@ -185,13 +195,14 @@ void PushingAsyncPipelineExecutor::push(Chunk chunk) data->rethrowExceptionIfHas(); if (!is_pushed) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Pipeline for PushingAsyncPipelineExecutor was finished before all data was inserted"); + checkExecutionStatus(data->executor->getExecutionStatus()); + + return is_pushed; } -void PushingAsyncPipelineExecutor::push(Block block) +bool PushingAsyncPipelineExecutor::push(Block block) { - push(Chunk(block.getColumns(), block.rows())); + return push(Chunk(block.getColumns(), block.rows())); } void PushingAsyncPipelineExecutor::finish() diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.h b/src/Processors/Executors/PushingAsyncPipelineExecutor.h index f976cd4c339..7835aaf596f 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.h @@ -36,8 +36,11 @@ public: void start(); - void push(Chunk chunk); - void push(Block block); + /// Return 'true' if push was successful. + /// Return 'false' if pipline was cancelled without exception. + /// This may happen in case of timeout_overflow_mode = 'break' OR internal bug. + [[nodiscard]] bool push(Chunk chunk); + [[nodiscard]] bool push(Block block); void finish(); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 696932932df..3133cfd9a1e 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -80,36 +80,56 @@ const Block & PushingPipelineExecutor::getHeader() const return pushing_source->getPort().getHeader(); } +static void checkExecutionStatus(PipelineExecutor::ExecutionStatus status) +{ + if (status == PipelineExecutor::ExecutionStatus::CancelledByTimeout + || status == PipelineExecutor::ExecutionStatus::CancelledByUser) + return; -void PushingPipelineExecutor::start() + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); +} + +bool PushingPipelineExecutor::start() { if (started) - return; + return true; started = true; executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); if (!executor->executeStep(&input_wait_flag)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); + { + checkExecutionStatus(executor->getExecutionStatus()); + return false; + } + + return true; } -void PushingPipelineExecutor::push(Chunk chunk) +bool PushingPipelineExecutor::push(Chunk chunk) { if (!started) - start(); + { + if (!start()) + return false; + } pushing_source->setData(std::move(chunk)); if (!executor->executeStep(&input_wait_flag)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); + { + checkExecutionStatus(executor->getExecutionStatus()); + return false; + } + + return true; } -void PushingPipelineExecutor::push(Block block) +bool PushingPipelineExecutor::push(Block block) { - push(Chunk(block.getColumns(), block.rows())); + return push(Chunk(block.getColumns(), block.rows())); } void PushingPipelineExecutor::finish() diff --git a/src/Processors/Executors/PushingPipelineExecutor.h b/src/Processors/Executors/PushingPipelineExecutor.h index f549c9482db..4021f61fb6b 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.h +++ b/src/Processors/Executors/PushingPipelineExecutor.h @@ -35,10 +35,13 @@ public: /// Get structure of returned block or chunk. const Block & getHeader() const; - void start(); + bool start(); - void push(Chunk chunk); - void push(Block block); + /// Return 'true' if push was successful. + /// Return 'false' if pipline was cancelled without exception. + /// This may happen in case of timeout_overflow_mode = 'break' OR internal bug. + [[nodiscard]] bool push(Chunk chunk); + [[nodiscard]] bool push(Block block); void finish(); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index eeb8f4a6060..857233ac028 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -215,7 +215,8 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_table) { block = materializeBlock(block); - executor->push(block); + if (!executor->push(block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert into a table"); rows_to_transfer += block.rows(); bytes_to_transfer += block.bytes(); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index d8a4d7f0e1f..c261d76ef33 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1012,7 +1012,8 @@ namespace while (pipeline_executor->pull(block)) { if (block) - executor.push(block); + if (!executor.push(block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); } if (isQueryCancelled()) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 448dfafbd9d..283b60b533c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -932,12 +932,18 @@ void TCPHandler::processInsertQuery() executor.start(); if (processed_data) - executor.push(std::move(processed_data)); + { + if (!executor.push(std::move(processed_data))) + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); + } else startInsertQuery(); while (readDataNext()) - executor.push(std::move(state.block_for_insert)); + { + if (!executor.push(std::move(state.block_for_insert))) + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); + } if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) executor.cancel(); @@ -2034,7 +2040,8 @@ bool TCPHandler::receiveData(bool scalar) QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context, /*async_insert=*/false)); PushingPipelineExecutor executor(temporary_table_out); executor.start(); - executor.push(block); + if (!executor.push(block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert into temporary table"); executor.finish(); } else if (state.need_receive_data_for_input) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index e3e73e42096..69f9e5b9380 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -89,7 +89,8 @@ static void writeBlockConvert(PushingPipelineExecutor & executor, const Block & { Block adopted_block = adoptBlock(executor.getHeader(), block, log); for (size_t i = 0; i < repeats; ++i) - executor.push(adopted_block); + if (!executor.push(adopted_block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); } @@ -408,7 +409,8 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; Block adopted_shard_block = adoptBlock(job.executor->getHeader(), shard_block, log); - job.executor->push(adopted_shard_block); + if (!job.executor->push(adopted_shard_block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); } else // local { diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f753d369d2d..3223a2813a3 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1069,7 +1069,8 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto block_io = interpreter.execute(); PushingPipelineExecutor executor(block_io.pipeline); executor.start(); - executor.push(std::move(block_to_write)); + if (!executor.push(std::move(block_to_write))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageBuffer could not write data to destination table"); executor.finish(); } diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index d75f3616f21..60890337cb4 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -98,7 +98,7 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr con QueryPipeline pipeline(table->write({}, metadata_snapshot, context, /*async_insert=*/false)); PushingPipelineExecutor executor(pipeline); - executor.push(block); + std::ignore = executor.push(block); executor.finish(); return data; diff --git a/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.reference b/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.reference new file mode 100644 index 00000000000..68538c3f75b --- /dev/null +++ b/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.reference @@ -0,0 +1,2 @@ +QUERY_WAS_CANCELLED +QUERY_WAS_CANCELLED diff --git a/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh b/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh new file mode 100755 index 00000000000..030c5211b2d --- /dev/null +++ b/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query "create table null_t (number UInt64) engine = Null;" +${CLICKHOUSE_CLIENT} --query "select sleep(0.1) from system.numbers settings max_block_size = 1 format Native" 2>/dev/null | ${CLICKHOUSE_CLIENT} --max_execution_time = 0.3 --timeout_overflow_mode = 'break' --query "insert into null_t format Native" 2>&1 | grep -o "QUERY_WAS_CANCELLED" From 5a6090ad05117c76a4b37071a6362f30f395b235 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 13 Aug 2024 16:25:07 +0200 Subject: [PATCH 0891/1722] Fix --- src/Processors/Sources/PostgreSQLSource.cpp | 12 ++++++------ src/Processors/Sources/PostgreSQLSource.h | 14 +++++++++----- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Processors/Sources/PostgreSQLSource.cpp b/src/Processors/Sources/PostgreSQLSource.cpp index a3d6fd691d8..b9bda46bd10 100644 --- a/src/Processors/Sources/PostgreSQLSource.cpp +++ b/src/Processors/Sources/PostgreSQLSource.cpp @@ -35,9 +35,9 @@ PostgreSQLSource::PostgreSQLSource( const Block & sample_block, UInt64 max_block_size_) : ISource(sample_block.cloneEmpty()) - , query_str(query_str_) , max_block_size(max_block_size_) , connection_holder(std::move(connection_holder_)) + , query_str(query_str_) { init(sample_block); } @@ -51,10 +51,10 @@ PostgreSQLSource::PostgreSQLSource( UInt64 max_block_size_, bool auto_commit_) : ISource(sample_block.cloneEmpty()) - , query_str(query_str_) - , tx(std::move(tx_)) , max_block_size(max_block_size_) , auto_commit(auto_commit_) + , query_str(query_str_) + , tx(std::move(tx_)) { init(sample_block); } @@ -204,15 +204,15 @@ PostgreSQLSource::~PostgreSQLSource() */ stream->close(); } - - stream.reset(); - tx.reset(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } + stream.reset(); + tx.reset(); + if (connection_holder) connection_holder->setBroken(); } diff --git a/src/Processors/Sources/PostgreSQLSource.h b/src/Processors/Sources/PostgreSQLSource.h index 8a648ae8bb5..319c5d8d7c2 100644 --- a/src/Processors/Sources/PostgreSQLSource.h +++ b/src/Processors/Sources/PostgreSQLSource.h @@ -38,14 +38,12 @@ protected: UInt64 max_block_size_, bool auto_commit_); - String query_str; - std::shared_ptr tx; - std::unique_ptr stream; - Status prepare() override; - void onStart(); Chunk generate() override; + + void onStart(); + void onFinish(); private: @@ -61,6 +59,12 @@ private: postgres::ConnectionHolderPtr connection_holder; std::unordered_map array_info; + +protected: + String query_str; + /// tx and stream must be destroyed before connection_holder. + std::shared_ptr tx; + std::unique_ptr stream; }; From be92986eda3298f7f44d64509bda83689713a2a0 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 14:48:29 +0000 Subject: [PATCH 0892/1722] Possibly fix broken tests and build --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h | 2 +- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h | 2 +- src/TableFunctions/registerDataLakeTableFunctions.cpp | 2 -- tests/integration/helpers/s3_tools.py | 2 +- 6 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c896a760597..949f994fa09 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -3,7 +3,7 @@ #include "config.h" #include -#if USE_AWS_S3 && USE_PARQUET +#if USE_PARQUET #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index c8603fccb86..ae0e3066ffb 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -2,7 +2,7 @@ #include "config.h" -#if USE_AWS_S3 && USE_AVRO +#if USE_AVRO #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 6d18b13df01..9b9d92e282c 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -1,6 +1,6 @@ #include "config.h" -#if USE_AWS_S3 && USE_AVRO +#if USE_AVRO #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index 9476ac6e7d9..7b0deab91c3 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -1,6 +1,6 @@ #pragma once -#if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. #include #include diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 8de852f4fba..8361d8a7977 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -75,10 +75,8 @@ void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AVRO -# if USE_AWS_S3 registerTableFunctionIceberg(factory); #endif -#endif #if USE_AWS_S3 # if USE_PARQUET registerTableFunctionDeltaLake(factory); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/s3_tools.py index 39c56e0eb85..5b727060e69 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/s3_tools.py @@ -85,7 +85,7 @@ class AzureUploader(CloudUploader): def upload_directory(minio_client, bucket, local_path, remote_path): - S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( + return S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( local_path, remote_path ) From 7ebb6efb2e413c44a82f2ac25d6dcb8e0da2f59a Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 15:10:56 +0000 Subject: [PATCH 0893/1722] Style check --- tests/queries/0_stateless/00652_mergetree_mutations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_mergetree_mutations.sh b/tests/queries/0_stateless/00652_mergetree_mutations.sh index 6be0ebf882f..edb306d3883 100755 --- a/tests/queries/0_stateless/00652_mergetree_mutations.sh +++ b/tests/queries/0_stateless/00652_mergetree_mutations.sh @@ -73,7 +73,7 @@ sleep 0.1 for i in {1..10} do - if [ $(${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner'") -eq 2 ]; then + if [ "$(${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner'")" -eq 2 ]; then break fi From ae614648a3397c4738b85ab8d138419387c562ed Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Aug 2024 15:13:42 +0000 Subject: [PATCH 0894/1722] trigger sync From c5ae139c972d46d1e0bfa6ab5f165a049b6786f5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Aug 2024 15:18:07 +0000 Subject: [PATCH 0895/1722] Cleanup. --- programs/obfuscator/Obfuscator.cpp | 2 +- src/Client/LocalConnection.cpp | 8 ++--- src/Interpreters/SystemLog.cpp | 2 +- .../PushingAsyncPipelineExecutor.cpp | 15 ++++---- .../Executors/PushingAsyncPipelineExecutor.h | 7 ++-- .../Executors/PushingPipelineExecutor.cpp | 34 ++++++------------- .../Executors/PushingPipelineExecutor.h | 9 ++--- .../Transforms/CreatingSetsTransform.cpp | 3 +- src/Server/GRPCServer.cpp | 3 +- src/Server/TCPHandler.cpp | 13 ++----- src/Storages/Distributed/DistributedSink.cpp | 6 ++-- src/Storages/StorageBuffer.cpp | 3 +- src/Storages/tests/gtest_storage_log.cpp | 2 +- 13 files changed, 36 insertions(+), 71 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 7c13215e350..688ae1a1143 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1462,7 +1462,7 @@ try while (in_executor.pull(block)) { Columns columns = obfuscator.generate(block.getColumns()); - std::ignore = out_executor.push(header.cloneWithColumns(columns)); + out_executor.push(header.cloneWithColumns(columns)); processed_rows += block.rows(); if (!silent) std::cerr << "Processed " << processed_rows << " rows\n"; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 8f1e0958002..072184e0a66 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -287,17 +287,13 @@ void LocalConnection::sendData(const Block & block, const String &, bool) if (!block) return; - bool inserted = false; if (state->pushing_async_executor) - inserted = state->pushing_async_executor->push(block); + state->pushing_async_executor->push(block); else if (state->pushing_executor) - inserted = state->pushing_executor->push(block); + state->pushing_executor->push(block); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown executor"); - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); - if (send_profile_events) sendProfileEvents(); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 0cad56af00a..572481e6b12 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -556,7 +556,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, PushingPipelineExecutor executor(io.pipeline); executor.start(); - std::ignore = executor.push(block); + executor.push(block); executor.finish(); } catch (...) diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index db5cf451c9e..866d224a08d 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; } class PushingAsyncSource : public ISource @@ -176,17 +177,17 @@ void PushingAsyncPipelineExecutor::start() data->thread = ThreadFromGlobalPool(std::move(func)); } -static void checkExecutionStatus(PipelineExecutor::ExecutionStatus status) +[[noreturn]] static void throwOnExecutionStatus(PipelineExecutor::ExecutionStatus status) { if (status == PipelineExecutor::ExecutionStatus::CancelledByTimeout || status == PipelineExecutor::ExecutionStatus::CancelledByUser) - return; + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); } -bool PushingAsyncPipelineExecutor::push(Chunk chunk) +void PushingAsyncPipelineExecutor::push(Chunk chunk) { if (!started) start(); @@ -195,14 +196,12 @@ bool PushingAsyncPipelineExecutor::push(Chunk chunk) data->rethrowExceptionIfHas(); if (!is_pushed) - checkExecutionStatus(data->executor->getExecutionStatus()); - - return is_pushed; + throwOnExecutionStatus(data->executor->getExecutionStatus()); } -bool PushingAsyncPipelineExecutor::push(Block block) +void PushingAsyncPipelineExecutor::push(Block block) { - return push(Chunk(block.getColumns(), block.rows())); + push(Chunk(block.getColumns(), block.rows())); } void PushingAsyncPipelineExecutor::finish() diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.h b/src/Processors/Executors/PushingAsyncPipelineExecutor.h index 7835aaf596f..f976cd4c339 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.h @@ -36,11 +36,8 @@ public: void start(); - /// Return 'true' if push was successful. - /// Return 'false' if pipline was cancelled without exception. - /// This may happen in case of timeout_overflow_mode = 'break' OR internal bug. - [[nodiscard]] bool push(Chunk chunk); - [[nodiscard]] bool push(Block block); + void push(Chunk chunk); + void push(Block block); void finish(); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 3133cfd9a1e..7a1c0111a3a 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; } class PushingSource : public ISource @@ -80,56 +81,43 @@ const Block & PushingPipelineExecutor::getHeader() const return pushing_source->getPort().getHeader(); } -static void checkExecutionStatus(PipelineExecutor::ExecutionStatus status) +[[noreturn]] static void throwOnExecutionStatus(PipelineExecutor::ExecutionStatus status) { if (status == PipelineExecutor::ExecutionStatus::CancelledByTimeout || status == PipelineExecutor::ExecutionStatus::CancelledByUser) - return; + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); } -bool PushingPipelineExecutor::start() +void PushingPipelineExecutor::start() { if (started) - return true; + return; started = true; executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); if (!executor->executeStep(&input_wait_flag)) - { - checkExecutionStatus(executor->getExecutionStatus()); - return false; - } - - return true; + throwOnExecutionStatus(executor->getExecutionStatus()); } -bool PushingPipelineExecutor::push(Chunk chunk) +void PushingPipelineExecutor::push(Chunk chunk) { if (!started) - { - if (!start()) - return false; - } + start(); pushing_source->setData(std::move(chunk)); if (!executor->executeStep(&input_wait_flag)) - { - checkExecutionStatus(executor->getExecutionStatus()); - return false; - } - - return true; + throwOnExecutionStatus(executor->getExecutionStatus()); } -bool PushingPipelineExecutor::push(Block block) +void PushingPipelineExecutor::push(Block block) { - return push(Chunk(block.getColumns(), block.rows())); + push(Chunk(block.getColumns(), block.rows())); } void PushingPipelineExecutor::finish() diff --git a/src/Processors/Executors/PushingPipelineExecutor.h b/src/Processors/Executors/PushingPipelineExecutor.h index 4021f61fb6b..f549c9482db 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.h +++ b/src/Processors/Executors/PushingPipelineExecutor.h @@ -35,13 +35,10 @@ public: /// Get structure of returned block or chunk. const Block & getHeader() const; - bool start(); + void start(); - /// Return 'true' if push was successful. - /// Return 'false' if pipline was cancelled without exception. - /// This may happen in case of timeout_overflow_mode = 'break' OR internal bug. - [[nodiscard]] bool push(Chunk chunk); - [[nodiscard]] bool push(Block block); + void push(Chunk chunk); + void push(Block block); void finish(); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 857233ac028..eeb8f4a6060 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -215,8 +215,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_table) { block = materializeBlock(block); - if (!executor->push(block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert into a table"); + executor->push(block); rows_to_transfer += block.rows(); bytes_to_transfer += block.bytes(); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index c261d76ef33..d8a4d7f0e1f 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1012,8 +1012,7 @@ namespace while (pipeline_executor->pull(block)) { if (block) - if (!executor.push(block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); + executor.push(block); } if (isQueryCancelled()) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 283b60b533c..448dfafbd9d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -932,18 +932,12 @@ void TCPHandler::processInsertQuery() executor.start(); if (processed_data) - { - if (!executor.push(std::move(processed_data))) - throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); - } + executor.push(std::move(processed_data)); else startInsertQuery(); while (readDataNext()) - { - if (!executor.push(std::move(state.block_for_insert))) - throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); - } + executor.push(std::move(state.block_for_insert)); if (state.cancellation_status == CancellationStatus::FULLY_CANCELLED) executor.cancel(); @@ -2040,8 +2034,7 @@ bool TCPHandler::receiveData(bool scalar) QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context, /*async_insert=*/false)); PushingPipelineExecutor executor(temporary_table_out); executor.start(); - if (!executor.push(block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert into temporary table"); + executor.push(block); executor.finish(); } else if (state.need_receive_data_for_input) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 69f9e5b9380..e3e73e42096 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -89,8 +89,7 @@ static void writeBlockConvert(PushingPipelineExecutor & executor, const Block & { Block adopted_block = adoptBlock(executor.getHeader(), block, log); for (size_t i = 0; i < repeats; ++i) - if (!executor.push(adopted_block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); + executor.push(adopted_block); } @@ -409,8 +408,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; Block adopted_shard_block = adoptBlock(job.executor->getHeader(), shard_block, log); - if (!job.executor->push(adopted_shard_block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send data"); + job.executor->push(adopted_shard_block); } else // local { diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 3223a2813a3..f753d369d2d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1069,8 +1069,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto block_io = interpreter.execute(); PushingPipelineExecutor executor(block_io.pipeline); executor.start(); - if (!executor.push(std::move(block_to_write))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageBuffer could not write data to destination table"); + executor.push(std::move(block_to_write)); executor.finish(); } diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 60890337cb4..d75f3616f21 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -98,7 +98,7 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr con QueryPipeline pipeline(table->write({}, metadata_snapshot, context, /*async_insert=*/false)); PushingPipelineExecutor executor(pipeline); - std::ignore = executor.push(block); + executor.push(block); executor.finish(); return data; From 94cc37a39f53f884cc7c6cd81a76c41bb2ea8565 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 11:49:42 +0000 Subject: [PATCH 0896/1722] Remove robin-map submodule At some point, usearch stopped to use robin-map. --- .gitmodules | 3 --- contrib/CMakeLists.txt | 3 +-- contrib/robin-map | 1 - contrib/robin-map-cmake/CMakeLists.txt | 1 - contrib/usearch-cmake/CMakeLists.txt | 2 -- 5 files changed, 1 insertion(+), 9 deletions(-) delete mode 160000 contrib/robin-map delete mode 100644 contrib/robin-map-cmake/CMakeLists.txt diff --git a/.gitmodules b/.gitmodules index 0a66031de8d..cdee6a43ad8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -345,9 +345,6 @@ [submodule "contrib/FP16"] path = contrib/FP16 url = https://github.com/Maratyszcza/FP16.git -[submodule "contrib/robin-map"] - path = contrib/robin-map - url = https://github.com/Tessil/robin-map.git [submodule "contrib/aklomp-base64"] path = contrib/aklomp-base64 url = https://github.com/aklomp/base64.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index dc2ad2a3150..d7489bc5c0e 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -209,9 +209,8 @@ endif() option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES}) if (ENABLE_USEARCH) add_contrib (FP16-cmake FP16) - add_contrib (robin-map-cmake robin-map) add_contrib (SimSIMD-cmake SimSIMD) - add_contrib (usearch-cmake usearch) # requires: FP16, robin-map, SimdSIMD + add_contrib (usearch-cmake usearch) # requires: FP16, SimdSIMD else () message(STATUS "Not using USearch") endif () diff --git a/contrib/robin-map b/contrib/robin-map deleted file mode 160000 index 851a59e0e30..00000000000 --- a/contrib/robin-map +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 851a59e0e3063ee0e23089062090a73fd3de482d diff --git a/contrib/robin-map-cmake/CMakeLists.txt b/contrib/robin-map-cmake/CMakeLists.txt deleted file mode 100644 index f82ad705dcc..00000000000 --- a/contrib/robin-map-cmake/CMakeLists.txt +++ /dev/null @@ -1 +0,0 @@ -# See contrib/usearch-cmake/CMakeLists.txt diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 6be622275ae..83221e3810f 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -1,5 +1,4 @@ set(FP16_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/FP16") -set(ROBIN_MAP_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/robin-map") set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") @@ -7,7 +6,6 @@ add_library(_usearch INTERFACE) target_include_directories(_usearch SYSTEM INTERFACE ${FP16_PROJECT_DIR}/include - ${ROBIN_MAP_PROJECT_DIR}/include ${SIMSIMD_PROJECT_DIR}/include ${USEARCH_PROJECT_DIR}/include) From 5ca85674e6fd51633c8fa636bed71002dd2cd281 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 19:29:04 +0000 Subject: [PATCH 0897/1722] Bump usearch to 2.3.2 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index de2cb75b9e9..c98e4635f3c 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit de2cb75b9e9e3389d5e1e51fd9f8ed151f3c17cf +Subproject commit c98e4635f3cca9e33918fe1bdca23571162e0c28 diff --git a/contrib/usearch b/contrib/usearch index 30810452bec..65b5d178f05 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 30810452bec5d3d3aa0931bb5d761e2f09aa6356 +Subproject commit 65b5d178f053d21480796d214b6ca04172d854a4 From 98c18eb341481d3daf610282c58960eb89188960 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 19:50:53 +0000 Subject: [PATCH 0898/1722] Bump usearch to 2.4.1 --- contrib/usearch | 2 +- .../MergeTree/MergeTreeIndexVectorSimilarity.cpp | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/contrib/usearch b/contrib/usearch index 65b5d178f05..e811aa8c1d0 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 65b5d178f053d21480796d214b6ca04172d854a4 +Subproject commit e811aa8c1d07dfb3725e05fedb550f91fe44a324 diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 5b0793fa0c8..083311a6602 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -95,9 +95,14 @@ USearchIndexWithSerialization::USearchIndexWithSerialization( unum::usearch::metric_kind_t metric_kind, unum::usearch::scalar_kind_t scalar_kind, UsearchHnswParams usearch_hnsw_params) - : Base(Base::make(unum::usearch::metric_punned_t(dimensions, metric_kind, scalar_kind), - unum::usearch::index_dense_config_t(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search))) { + unum::usearch::metric_punned_t metric(dimensions, metric_kind, scalar_kind); + + unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search); + config.enable_key_lookups = false; /// we don't do row-to-vector lookups + + USearchIndex usearch_index = USearchIndex::make(metric, config); + swap(usearch_index); } void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const From dbe66e6092f6a6e232108f5b586253d0a017f66a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 09:47:17 +0000 Subject: [PATCH 0899/1722] Bump usearch to 2.5.1 --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index e811aa8c1d0..f2b4bff52b7 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit e811aa8c1d07dfb3725e05fedb550f91fe44a324 +Subproject commit f2b4bff52b74a0bf33067bc034ba68bb785753ee From 383d2816e66c1ad67c9f4b962b52efd2cb8d1b53 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 09:56:11 +0000 Subject: [PATCH 0900/1722] Bump usearch to 2.6.1 --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index f2b4bff52b7..a7bc711dfb9 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit f2b4bff52b74a0bf33067bc034ba68bb785753ee +Subproject commit a7bc711dfb9e5665a1aee89d3a0297a211f2b97d From 88f2d2e67df5acbca8c4df45be00ab23d42e090c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 10:00:12 +0000 Subject: [PATCH 0901/1722] Bump usearch to v2.7.8 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index c98e4635f3c..8f2c8881e44 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit c98e4635f3cca9e33918fe1bdca23571162e0c28 +Subproject commit 8f2c8881e440a55cfea246996984662623b4d5dd diff --git a/contrib/usearch b/contrib/usearch index a7bc711dfb9..b58cdb4025b 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit a7bc711dfb9e5665a1aee89d3a0297a211f2b97d +Subproject commit b58cdb4025b68b55800dcc9f36fa33b43c003a7e diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 083311a6602..346f69140bb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -364,7 +364,7 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members); ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, result.computed_distances); - std::vector neighbors(result.size()); /// indexes of dots which were closest to the reference vector + std::vector neighbors(result.size()); /// indexes of dots which were closest to the reference vector std::vector distances(result.size()); result.dump_to(neighbors.data(), distances.data()); From fe7da4e7d1f9f76c9bb1d13fc0baa01433f069a6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 10:23:12 +0000 Subject: [PATCH 0902/1722] Bump usearch to 2.8.16 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index 8f2c8881e44..fed0b4f8ec6 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit 8f2c8881e440a55cfea246996984662623b4d5dd +Subproject commit fed0b4f8ec6c1fb75d47e554ae8ca9188fc068f4 diff --git a/contrib/usearch b/contrib/usearch index b58cdb4025b..81edcb7936b 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit b58cdb4025b68b55800dcc9f36fa33b43c003a7e +Subproject commit 81edcb7936b3aba701997ae6b1af59a61df280e1 From bd09e948ba710a9013fdfa477288162936bc6f85 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 12:21:43 +0000 Subject: [PATCH 0903/1722] Bump usearch to 2.9.2 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index fed0b4f8ec6..02665027985 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit fed0b4f8ec6c1fb75d47e554ae8ca9188fc068f4 +Subproject commit 02665027985a578bd91514011c31a0bbe302304d diff --git a/contrib/usearch b/contrib/usearch index 81edcb7936b..3ba2661f46f 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 81edcb7936b3aba701997ae6b1af59a61df280e1 +Subproject commit 3ba2661f46fbc0065113e11f29404020210ebb53 From 92aed17e7cfdd78340df66075520713b2dee5c66 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 12:24:01 +0000 Subject: [PATCH 0904/1722] Bump usearch to 2.10.5 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index 02665027985..127ead1da7c 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit 02665027985a578bd91514011c31a0bbe302304d +Subproject commit 127ead1da7c39957b30a50dd85e74814edb022d6 diff --git a/contrib/usearch b/contrib/usearch index 3ba2661f46f..fa1019941fe 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 3ba2661f46fbc0065113e11f29404020210ebb53 +Subproject commit fa1019941fe71f359516543ff4ec9f6fa8f0cb80 From 72efc8308c8cda9a2015fd1c5a1057c5b4a5675a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 12:26:49 +0000 Subject: [PATCH 0905/1722] Bump usearch to 2.11.7 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index 127ead1da7c..18d17686124 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit 127ead1da7c39957b30a50dd85e74814edb022d6 +Subproject commit 18d17686124ddebd9fe55eee56b2e0273a613d4b diff --git a/contrib/usearch b/contrib/usearch index fa1019941fe..bc83df3c0b7 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit fa1019941fe71f359516543ff4ec9f6fa8f0cb80 +Subproject commit bc83df3c0b7da8376574a3ca2b48f0738365c205 From 58d76fabf68c48c08e25fca8d4f6318f86010625 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 12:28:09 +0000 Subject: [PATCH 0906/1722] Bump usearch to 2.12.0 --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index bc83df3c0b7..e6c81f78c64 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit bc83df3c0b7da8376574a3ca2b48f0738365c205 +Subproject commit e6c81f78c64c0d8119f854691a06e60660638a25 From dcf96fa9f4363f7607e9b5ed82056d94c49a6ee3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Aug 2024 17:57:06 +0200 Subject: [PATCH 0907/1722] Update 03221_insert_timeout_overflow_mode.sh --- tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh b/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh index 030c5211b2d..db943a665cb 100755 --- a/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh +++ b/tests/queries/0_stateless/03221_insert_timeout_overflow_mode.sh @@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query "create table null_t (number UInt64) engine = Null;" -${CLICKHOUSE_CLIENT} --query "select sleep(0.1) from system.numbers settings max_block_size = 1 format Native" 2>/dev/null | ${CLICKHOUSE_CLIENT} --max_execution_time = 0.3 --timeout_overflow_mode = 'break' --query "insert into null_t format Native" 2>&1 | grep -o "QUERY_WAS_CANCELLED" +${CLICKHOUSE_CLIENT} --query "select sleep(0.1) from system.numbers settings max_block_size = 1 format Native" 2>/dev/null | ${CLICKHOUSE_CLIENT} --max_execution_time 0.3 --timeout_overflow_mode break --query "insert into null_t format Native" 2>&1 | grep -o "QUERY_WAS_CANCELLED" From 6dfed409f460311f133e30e70f839f9865d71861 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 16:09:45 +0000 Subject: [PATCH 0908/1722] Fix seraching for query params --- tests/clickhouse-test | 35 ++++++++++++++++++++++------------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5bde4686d3a..515b519af3e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1978,13 +1978,20 @@ class TestSuite: def is_shebang(line: str) -> bool: return line.startswith("#!") - def find_tag_line(file): - line = file.readline() - while line != "": - line = line.strip() - if line and not is_shebang(line): + def find_tag_line(lines, comment_sign): + for line in lines: + if line.startswith(comment_sign) and line[ + len(comment_sign) : + ].lstrip().startswith("Tags:"): + return line + return "" + + def find_random_settings_limits_line(lines, comment_sign): + for line in lines: + if line.startswith(comment_sign) and line[ + len(comment_sign) : + ].lstrip().startswith("Random settings limits:"): return line - line = file.readline() return "" def load_tags_and_random_settings_limits_from_file(filepath): @@ -1992,13 +1999,16 @@ class TestSuite: need_query_params = False with open(filepath, "r", encoding="utf-8") as file: try: - tag_line = find_tag_line(file) - next_line = file.readline() + lines = file.readlines() + tag_line = find_tag_line(lines, comment_sign) + random_settings_limits_line = find_random_settings_limits_line( + lines, comment_sign + ) except UnicodeDecodeError: return [], {} try: if filepath.endswith(".sql"): - for line in file: + for line in lines: if "{CLICKHOUSE_DATABASE" in line: need_query_params = True except UnicodeDecodeError: @@ -2006,7 +2016,6 @@ class TestSuite: parsed_tags = parse_tags_from_line(tag_line, comment_sign) if need_query_params: parsed_tags.add("need-query-parameters") - random_settings_limits_line = next_line if parsed_tags else tag_line random_settings_limits = parse_random_settings_limits_from_line( random_settings_limits_line, comment_sign ) @@ -2068,9 +2077,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( - all_tags_and_random_settings_limits[1] - ) + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From 7e209ebdf686e374fdf764cb6acb3e7de83e927f Mon Sep 17 00:00:00 2001 From: Austin Bruch Date: Tue, 13 Aug 2024 12:30:17 -0400 Subject: [PATCH 0909/1722] Remove trailing colon in header for consistency --- docs/en/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 7908a3cb934..3243e6cf569 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -103,7 +103,7 @@ LIMIT 2; └─────────┴─────────┴─────────┘ ``` -### Inserting data from a file into a table: +### Inserting data from a file into a table ``` sql INSERT INTO FUNCTION From 6d2f17ea3b018dd54d2010cf3fbac3d6aa46142e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 17:12:38 +0000 Subject: [PATCH 0910/1722] Add more tests --- src/Columns/ColumnObject.cpp | 1 - .../03211_nested_json_merges.reference.j2 | 84 +++++++++++++++---- .../03211_nested_json_merges.sql.j2 | 19 ++++- .../0_stateless/03222_json_squashing.sql | 16 ++-- ...ested_json_in_shared_data_merges.reference | 81 ++++++++++++++++++ ...3223_nested_json_in_shared_data_merges.sql | 26 ++++++ 6 files changed, 203 insertions(+), 24 deletions(-) create mode 100644 tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.reference create mode 100644 tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.sql diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 0645c10340d..e1a3fbf3b18 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { diff --git a/tests/queries/0_stateless/03211_nested_json_merges.reference.j2 b/tests/queries/0_stateless/03211_nested_json_merges.reference.j2 index e05e9ced0b1..9b6ed82abed 100644 --- a/tests/queries/0_stateless/03211_nested_json_merges.reference.j2 +++ b/tests/queries/0_stateless/03211_nested_json_merges.reference.j2 @@ -1,3 +1,4 @@ +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; Dynamic paths 300000 c 150000 d @@ -19,26 +20,20 @@ Shared data paths 150000 d 150000 e Dynamic paths -300000 c -150000 d -Shared data paths -Dynamic paths -300000 c -Shared data paths -150000 d -Dynamic paths -600000 f -300000 c -150000 e -Shared data paths -150000 d -Dynamic paths 600000 f +450000 c Shared data paths 300000 c 150000 d 150000 e Dynamic paths +750000 c +Shared data paths +600000 f +150000 d +150000 e +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; +Dynamic paths 300000 c 150000 d Shared data paths @@ -59,6 +54,20 @@ Shared data paths 150000 d 150000 e Dynamic paths +600000 f +450000 c +Shared data paths +300000 c +150000 d +150000 e +Dynamic paths +750000 c +Shared data paths +600000 f +150000 d +150000 e +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; +Dynamic paths 300000 c 150000 d Shared data paths @@ -78,3 +87,50 @@ Shared data paths 300000 c 150000 d 150000 e +Dynamic paths +600000 f +450000 c +Shared data paths +300000 c +150000 d +150000 e +Dynamic paths +750000 c +Shared data paths +600000 f +150000 d +150000 e +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; +Dynamic paths +300000 c +150000 d +Shared data paths +Dynamic paths +300000 c +Shared data paths +150000 d +Dynamic paths +600000 f +300000 c +150000 e +Shared data paths +150000 d +Dynamic paths +600000 f +Shared data paths +300000 c +150000 d +150000 e +Dynamic paths +600000 f +450000 c +Shared data paths +300000 c +150000 d +150000 e +Dynamic paths +750000 c +Shared data paths +600000 f +150000 d +150000 e diff --git a/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 b/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 index 4ab5a5da0ef..0af998e22bb 100644 --- a/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 +++ b/tests/queries/0_stateless/03211_nested_json_merges.sql.j2 @@ -1,4 +1,4 @@ --- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan +-- Tags: no-fasttest, long, no-tsan, no-asan, no-msan, no-ubsan set allow_experimental_json_type = 1; @@ -9,6 +9,8 @@ drop table if exists test; '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;'] -%} +select '{{ create_command }}'; + {{ create_command }} system stop merges test; @@ -27,6 +29,7 @@ select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from te select 'Shared data paths'; select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path; +system stop merges test; 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); @@ -41,6 +44,20 @@ select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from te select 'Shared data paths'; select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path; +system stop merges test; +insert into test select number, toJSONString(map('b', arrayMap(x -> map('c', x), range(number % 5 + 1)))) from numbers(150000); + +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 -%} diff --git a/tests/queries/0_stateless/03222_json_squashing.sql b/tests/queries/0_stateless/03222_json_squashing.sql index b6bda7a702f..53090c5cb88 100644 --- a/tests/queries/0_stateless/03222_json_squashing.sql +++ b/tests/queries/0_stateless/03222_json_squashing.sql @@ -6,7 +6,7 @@ set max_block_size = 1000; drop table if exists test; create table test (json JSON) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, '{}'::JSON, number < 3000, '{"a" : 42, "b" : "Hello"}'::JSON, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON) from numbers(1000000); +insert into test select multiIf(number < 1000, '{}'::JSON, number < 3000, '{"a" : 42, "b" : "Hello"}'::JSON, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; select 'Dynamic paths'; @@ -15,7 +15,7 @@ select 'Shared data paths'; select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; truncate table test; -insert into test select multiIf(number < 1000, '{"a" : 42, "b" : "Hello"}'::JSON, number < 3000, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON, '{"e" : 43, "f" : ["s1", "s2", "s3"]}'::JSON) from numbers(1000000); +insert into test select multiIf(number < 1000, '{"a" : 42, "b" : "Hello"}'::JSON, number < 3000, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON, '{"e" : 43, "f" : ["s1", "s2", "s3"]}'::JSON) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; select 'Dynamic paths'; @@ -25,7 +25,7 @@ select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by drop table test; create table test (json JSON(max_dynamic_paths=2)) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, '{}'::JSON(max_dynamic_paths=2), number < 3000, '{"a" : 42, "b" : "Hello"}'::JSON(max_dynamic_paths=2), '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2)) from numbers(1000000); +insert into test select multiIf(number < 1000, '{}'::JSON(max_dynamic_paths=2), number < 3000, '{"a" : 42, "b" : "Hello"}'::JSON(max_dynamic_paths=2), '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2)) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; select 'Dynamic paths'; @@ -34,7 +34,7 @@ select 'Shared data paths'; select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; truncate table test; -insert into test select multiIf(number < 1000, '{"a" : 42, "b" : "Hello"}'::JSON(max_dynamic_paths=2), number < 3000, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2), '{"e" : 43, "f" : ["s1", "s2", "s3"]}'::JSON(max_dynamic_paths=2)) from numbers(1000000); +insert into test select multiIf(number < 1000, '{"a" : 42, "b" : "Hello"}'::JSON(max_dynamic_paths=2), number < 3000, '{"c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2), '{"e" : 43, "f" : ["s1", "s2", "s3"]}'::JSON(max_dynamic_paths=2)) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; select 'Dynamic paths'; @@ -43,7 +43,7 @@ select 'Shared data paths'; select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by path; truncate table test; -insert into test select multiIf(number < 1000, '{"a" : 42}'::JSON(max_dynamic_paths=2), number < 3000, '{"b" : "Hello", "c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2), '{"e" : 43}'::JSON(max_dynamic_paths=2)) from numbers(1000000); +insert into test select multiIf(number < 1000, '{"a" : 42}'::JSON(max_dynamic_paths=2), number < 3000, '{"b" : "Hello", "c" : [1, 2, 3], "d" : "2020-01-01"}'::JSON(max_dynamic_paths=2), '{"e" : 43}'::JSON(max_dynamic_paths=2)) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(json)) as path from test order by path; select 'Dynamic paths'; @@ -53,7 +53,7 @@ select distinct arrayJoin(JSONSharedDataPaths(json)) as path from test order by drop table test; create table test (json JSON(max_dynamic_paths=8)) engine=MergeTree order by tuple(); -insert into test select multiIf(number < 1000, '{}'::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"b" : 42, "c" : [1, 2, 3]}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8)) from numbers(1000000); +insert into test select multiIf(number < 1000, '{}'::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"b" : 42, "c" : [1, 2, 3]}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8)) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(arrayJoin(json.a[]))) as path from test order by path; select 'Dynamic paths'; @@ -62,7 +62,7 @@ select 'Shared data paths'; select distinct arrayJoin(JSONSharedDataPaths(arrayJoin(json.a[]))) as path from test order by path; truncate table test; -insert into test select multiIf(number < 1000, materialize('{"a" : [{"b" : 42, "c" : [1, 2, 3]}]}')::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"f" : "2020-01-01 00:00:00", "g" : "Hello2"}]}')::JSON(max_dynamic_paths=8)) from numbers(1000000); +insert into test select multiIf(number < 1000, materialize('{"a" : [{"b" : 42, "c" : [1, 2, 3]}]}')::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"f" : "2020-01-01 00:00:00", "g" : "Hello2"}]}')::JSON(max_dynamic_paths=8)) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(arrayJoin(json.a[]))) as path from test order by path; select 'Dynamic paths'; @@ -71,7 +71,7 @@ select 'Shared data paths'; select distinct arrayJoin(JSONSharedDataPaths(arrayJoin(json.a[]))) as path from test order by path; truncate table test; -insert into test select multiIf(number < 1000, materialize('{"a" : [{"b" : 42}]}')::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"f" : "2020-01-01 00:00:00"}]}')::JSON(max_dynamic_paths=8)) from numbers(1000000); +insert into test select multiIf(number < 1000, materialize('{"a" : [{"b" : 42}]}')::JSON(max_dynamic_paths=8), number < 3000, materialize('{"a" : [{"d" : "2020-01-01", "e" : "Hello"}]}')::JSON(max_dynamic_paths=8), materialize('{"a" : [{"f" : "2020-01-01 00:00:00"}]}')::JSON(max_dynamic_paths=8)) from numbers(20000); select 'All paths'; select distinct arrayJoin(JSONAllPaths(arrayJoin(json.a[]))) as path from test order by path; select 'Dynamic paths'; diff --git a/tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.reference b/tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.reference new file mode 100644 index 00000000000..6c01506e800 --- /dev/null +++ b/tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.reference @@ -0,0 +1,81 @@ +All paths +['b'] +['b'] +['b'] +['b'] +['b'] +['c'] +['c'] +['c'] +['c'] +['c'] +Dynamic paths +['b'] +['b'] +['b'] +['b'] +['b'] +[] +[] +[] +[] +[] +Shared data paths +[] +[] +[] +[] +[] +['c'] +['c'] +['c'] +['c'] +['c'] +All paths +['b'] +['b'] +['b'] +['b'] +['b'] +['c'] +['c'] +['c'] +['c'] +['c'] +['b'] +['b'] +['b'] +['b'] +['b'] +Dynamic paths +['b'] +['b'] +['b'] +['b'] +['b'] +['c'] +['c'] +['c'] +['c'] +['c'] +['b'] +['b'] +['b'] +['b'] +['b'] +Shared data paths +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] diff --git a/tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.sql b/tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.sql new file mode 100644 index 00000000000..311eba37772 --- /dev/null +++ b/tests/queries/0_stateless/03223_nested_json_in_shared_data_merges.sql @@ -0,0 +1,26 @@ +set allow_experimental_json_type = 1; + +drop table if exists test; +create table test (json JSON(max_dynamic_paths=8)) engine=MergeTree order by tuple() settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1; +insert into test select materialize('{"a" : [{"b" : 42}]}')::JSON(max_dynamic_paths=8) from numbers(5); +insert into test select materialize('{"aa1" : 42, "aa2" : 42, "aa3" : 42, "aa4" : 42, "aa5" : 42, "aa6" : 42, "aa7" : 42, "aa8" : 42, "a" : [{"c" : 42}]}') from numbers(5); +optimize table test final; + +select 'All paths'; +select JSONAllPaths(arrayJoin(json.a[])) from test; +select 'Dynamic paths'; +select JSONDynamicPaths(arrayJoin(json.a[])) from test; +select 'Shared data paths'; +select JSONSharedDataPaths(arrayJoin(json.a[])) from test; + +insert into test select materialize('{"a" : [{"b" : 42}]}')::JSON(max_dynamic_paths=8) from numbers(5); +optimize table test final; + +select 'All paths'; +select JSONAllPaths(arrayJoin(json.a[])) from test; +select 'Dynamic paths'; +select JSONDynamicPaths(arrayJoin(json.a[])) from test; +select 'Shared data paths'; +select JSONSharedDataPaths(arrayJoin(json.a[])) from test; + +drop table test; From 22b31ef81f036ad8a65e7172e6be2270674ac09f Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 17:42:47 +0000 Subject: [PATCH 0911/1722] Add more tests --- .../03209_json_type_horizontal_merges.sql.j2 | 4 +- .../03209_json_type_merges_small.reference.j2 | 272 ++++++++++++++++++ .../03209_json_type_merges_small.sql.j2 | 76 +++++ .../03209_json_type_vertical_merges.sql.j2 | 4 +- ...3211_nested_json_merges_small.reference.j2 | 136 +++++++++ .../03211_nested_json_merges_small.sql.j2 | 63 ++++ 6 files changed, 551 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03209_json_type_merges_small.reference.j2 create mode 100644 tests/queries/0_stateless/03209_json_type_merges_small.sql.j2 create mode 100644 tests/queries/0_stateless/03211_nested_json_merges_small.reference.j2 create mode 100644 tests/queries/0_stateless/03211_nested_json_merges_small.sql.j2 diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 index 05b5d8d6095..cc143e4ceef 100644 --- a/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.sql.j2 @@ -4,8 +4,8 @@ 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;'] -%} +{% 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;', + '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;'] -%} select '{{ create_command }}'; diff --git a/tests/queries/0_stateless/03209_json_type_merges_small.reference.j2 b/tests/queries/0_stateless/03209_json_type_merges_small.reference.j2 new file mode 100644 index 00000000000..f953dee10fe --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_merges_small.reference.j2 @@ -0,0 +1,272 @@ +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; +Dynamic paths +10 a +9 b +8 c +7 d +6 e +Shared data paths +Dynamic paths +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +1 g +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +10 a +9 b +4 c +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +12 c +10 a +Shared data paths +9 b +7 d +6 e +1 g +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; +Dynamic paths +10 a +9 b +8 c +7 d +6 e +Shared data paths +Dynamic paths +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +1 g +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +10 a +9 b +4 c +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +12 c +10 a +Shared data paths +9 b +7 d +6 e +1 g +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; +Dynamic paths +10 a +9 b +8 c +7 d +6 e +Shared data paths +Dynamic paths +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +1 g +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +10 a +9 b +4 c +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +12 c +10 a +Shared data paths +9 b +7 d +6 e +1 g +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; +Dynamic paths +10 a +9 b +8 c +7 d +6 e +Shared data paths +Dynamic paths +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +8 c +Shared data paths +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +1 g +Shared data paths +8 c +7 d +6 e +Dynamic paths +20 f +10 a +9 b +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +10 a +9 b +4 c +Shared data paths +8 c +7 d +6 e +1 g +Dynamic paths +20 f +12 c +10 a +Shared data paths +9 b +7 d +6 e +1 g diff --git a/tests/queries/0_stateless/03209_json_type_merges_small.sql.j2 b/tests/queries/0_stateless/03209_json_type_merges_small.sql.j2 new file mode 100644 index 00000000000..e4b64ac7561 --- /dev/null +++ b/tests/queries/0_stateless/03209_json_type_merges_small.sql.j2 @@ -0,0 +1,76 @@ +-- 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;', + '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;', + '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;', + '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;'] -%} + +select '{{ create_command }}'; + +{{ create_command }} + +system stop merges test; +insert into test select number, toJSONString(map('a', number)) from numbers(10); +insert into test select number, toJSONString(map('b', number)) from numbers(9); +insert into test select number, toJSONString(map('c', number)) from numbers(8); +insert into test select number, toJSONString(map('d', number)) from numbers(7); +insert into test select number, toJSONString(map('e', number)) from numbers(6); +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(20); +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(1); +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('c', number)) from numbers(4); +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 -%} diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 b/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 index d17d8a48537..e427db7677f 100644 --- a/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.sql.j2 @@ -4,8 +4,8 @@ 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;'] -%} +{% 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;', + '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;'] -%} select '{{ create_command }}'; diff --git a/tests/queries/0_stateless/03211_nested_json_merges_small.reference.j2 b/tests/queries/0_stateless/03211_nested_json_merges_small.reference.j2 new file mode 100644 index 00000000000..76339dba3e3 --- /dev/null +++ b/tests/queries/0_stateless/03211_nested_json_merges_small.reference.j2 @@ -0,0 +1,136 @@ +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; +Dynamic paths +30 c +15 d +Shared data paths +Dynamic paths +30 c +Shared data paths +15 d +Dynamic paths +60 f +30 c +15 e +Shared data paths +15 d +Dynamic paths +60 f +Shared data paths +30 c +15 d +15 e +Dynamic paths +60 f +45 c +Shared data paths +30 c +15 d +15 e +Dynamic paths +75 c +Shared data paths +60 f +15 d +15 e +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; +Dynamic paths +30 c +15 d +Shared data paths +Dynamic paths +30 c +Shared data paths +15 d +Dynamic paths +60 f +30 c +15 e +Shared data paths +15 d +Dynamic paths +60 f +Shared data paths +30 c +15 d +15 e +Dynamic paths +60 f +45 c +Shared data paths +30 c +15 d +15 e +Dynamic paths +75 c +Shared data paths +60 f +15 d +15 e +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; +Dynamic paths +30 c +15 d +Shared data paths +Dynamic paths +30 c +Shared data paths +15 d +Dynamic paths +60 f +30 c +15 e +Shared data paths +15 d +Dynamic paths +60 f +Shared data paths +30 c +15 d +15 e +Dynamic paths +60 f +45 c +Shared data paths +30 c +15 d +15 e +Dynamic paths +75 c +Shared data paths +60 f +15 d +15 e +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; +Dynamic paths +30 c +15 d +Shared data paths +Dynamic paths +30 c +Shared data paths +15 d +Dynamic paths +60 f +30 c +15 e +Shared data paths +15 d +Dynamic paths +60 f +Shared data paths +30 c +15 d +15 e +Dynamic paths +60 f +45 c +Shared data paths +30 c +15 d +15 e +Dynamic paths +75 c +Shared data paths +60 f +15 d +15 e diff --git a/tests/queries/0_stateless/03211_nested_json_merges_small.sql.j2 b/tests/queries/0_stateless/03211_nested_json_merges_small.sql.j2 new file mode 100644 index 00000000000..86e5a6c71c9 --- /dev/null +++ b/tests/queries/0_stateless/03211_nested_json_merges_small.sql.j2 @@ -0,0 +1,63 @@ +-- Tags: no-fasttest, long, 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;'] -%} + +select '{{ create_command }}'; + +{{ create_command }} + +system stop merges test; +insert into test select number, toJSONString(map('a', number)) from numbers(10); +insert into test select number, toJSONString(map('b', arrayMap(x -> map('c', x), range(number % 5 + 1)))) from numbers(10); +insert into test select number, toJSONString(map('b', arrayMap(x -> map('d', x), range(number % 5 + 1)))) from numbers(5); + +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; + +system stop merges test; +insert into test select number, toJSONString(map('b', arrayMap(x -> map('e', x), range(number % 5 + 1)))) from numbers(5); +insert into test select number, toJSONString(map('b', arrayMap(x -> map('f', x), range(number % 5 + 1)))) from numbers(20); + +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; + +system stop merges test; +insert into test select number, toJSONString(map('b', arrayMap(x -> map('c', x), range(number % 5 + 1)))) from numbers(15); + +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 -%} From 9833ef0bed218afdc1927181c11bac306fe21dda Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 13 Aug 2024 17:50:13 +0000 Subject: [PATCH 0912/1722] slightly better --- .../Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h | 2 +- src/Processors/Merges/IMergingTransform.h | 6 +++--- src/Storages/MergeTree/MergeTask.cpp | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 39171c5a978..c34028b1cba 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -50,7 +50,7 @@ public: void consume(Input & input, size_t source_num) override; Status merge() override; - MergedStats getMergedStats() const override { return {.bytes = accumulated_bytes, .rows = accumulated_rows, .blocks = chunk_num}; } + MergedStats getMergedStats() const override { return {.bytes = accumulated_bytes, .rows = accumulated_rows, .blocks = chunk_num}; } private: Chunk prepareToMerge(); diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index fba5b038618..e5cd3bdde46 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -113,7 +113,7 @@ public: void work() override { - Stopwatch watch; + Stopwatch watch{CLOCK_MONOTONIC_COARSE}; if (!state.init_chunks.empty()) algorithm.initialize(std::move(state.init_chunks)); @@ -180,12 +180,12 @@ protected: if (seconds == 0.0) { - LOG_DEBUG(log, "{}: {} blocks, {} rows, {} bytes in 0 sec.", + LOG_DEBUG(log, "{}, {} blocks, {} rows, {} bytes in 0 sec.", transform_message, stats.blocks, stats.rows, stats.bytes); } else { - LOG_DEBUG(log, "{}: {} blocks, {} rows, {} bytes in {} sec., {} rows/sec., {}/sec.", + LOG_DEBUG(log, "{}, {} blocks, {} rows, {} bytes in {} sec., {} rows/sec., {}/sec.", transform_message, stats.blocks, stats.rows, stats.bytes, seconds, stats.rows / seconds, ReadableSize(stats.bytes / seconds)); } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3aa4d764685..95e00773bae 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -945,7 +945,7 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const MergeTask::StageRuntimeContextPtr MergeTask::MergeProjectionsStage::getContextForNextStage() { /// Do not increment for projection stage because time is already accounted in main task. - /// The projection stage has its own empty projection stage which may add a drift of severals milliseconds. + /// The projection stage has its own empty projection stage which may add a drift of several milliseconds. if (global_ctx->parent_part == nullptr) { ProfileEvents::increment(ProfileEvents::MergeExecuteMilliseconds, ctx->elapsed_execute_ns / 1000000UL); From c75112827c2d73620c0a5c579b04dfe4049deb56 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 18:05:27 +0000 Subject: [PATCH 0913/1722] Fis style --- src/Columns/ColumnObject.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index e1a3fbf3b18..d4f752cda95 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1212,10 +1212,10 @@ void ColumnObject::prepareForSquashing(const std::vector & source_col typed_paths_source_columns.reserve(typed_paths.size()); std::unordered_map dynamic_paths_source_columns; dynamic_paths_source_columns.reserve(dynamic_paths.size()); - + for (const auto & [path, column] : typed_paths) typed_paths_source_columns[path].reserve(source_columns.size()); - + for (const auto & [path, column] : dynamic_paths) dynamic_paths_source_columns[path].reserve(source_columns.size()); @@ -1225,10 +1225,10 @@ void ColumnObject::prepareForSquashing(const std::vector & source_col const auto & source_object_column = assert_cast(*source_column); total_size += source_object_column.size(); shared_data_source_columns.push_back(source_object_column.shared_data); - + for (const auto & [path, column] : source_object_column.typed_paths) typed_paths_source_columns.at(path).push_back(column); - + for (const auto & [path, column] : source_object_column.dynamic_paths) { if (dynamic_paths.contains(path)) @@ -1237,7 +1237,7 @@ void ColumnObject::prepareForSquashing(const std::vector & source_col } shared_data->prepareForSquashing(shared_data_source_columns); - + for (const auto & [path, source_typed_columns] : typed_paths_source_columns) typed_paths[path]->prepareForSquashing(source_typed_columns); From 3cfb921befa895e445e8d7b98e639015e1e41aa0 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 18:41:53 +0000 Subject: [PATCH 0914/1722] Fix using schema_inference_make_columns_nullable=0 --- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Formats/SchemaInferenceUtils.cpp | 4 +++ src/Processors/Formats/ISchemaReader.cpp | 2 +- .../Formats/Impl/ArrowBlockInputFormat.cpp | 7 +++-- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 9 ++++--- .../Formats/Impl/ArrowColumnToCHColumn.h | 3 ++- .../Impl/NativeORCBlockInputFormat.cpp | 2 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 7 +++-- .../Formats/Impl/ParquetBlockInputFormat.cpp | 7 +++-- .../03036_parquet_arrow_nullable.reference | 26 +++++++++++++++++++ .../03036_parquet_arrow_nullable.sh | 7 +++++ 13 files changed, 65 insertions(+), 15 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0808e8eb49f..bc9c6daab1b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1116,7 +1116,7 @@ class IColumn; M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \ M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ - M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ + M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHosue will use information about nullability from the data..", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a78836ff63c..8d8257b9abc 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -255,7 +255,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.max_bytes_to_read_for_schema_inference = settings.input_format_max_bytes_to_read_for_schema_inference; format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference; format_settings.schema_inference_hints = settings.schema_inference_hints; - format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable; + format_settings.schema_inference_make_columns_nullable = settings.schema_inference_make_columns_nullable.valueOr(2); format_settings.mysql_dump.table_name = settings.input_format_mysql_dump_table_name; format_settings.mysql_dump.map_column_names = settings.input_format_mysql_dump_map_column_names; format_settings.sql_insert.max_batch_size = settings.output_format_sql_insert_max_batch_size; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index f0359218775..479b1a89adf 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -75,7 +75,7 @@ struct FormatSettings Raw }; - bool schema_inference_make_columns_nullable = true; + UInt64 schema_inference_make_columns_nullable = true; DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 3c374ada9e6..c04682e8765 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1262,7 +1262,11 @@ namespace if (checkCharCaseInsensitive('n', buf)) { if (checkStringCaseInsensitive("ull", buf)) + { + if (settings.schema_inference_make_columns_nullable == 0) + return std::make_shared(); return makeNullable(std::make_shared()); + } else if (checkStringCaseInsensitive("an", buf)) return std::make_shared(); } diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 45523700a5d..569d4bb39e7 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -54,7 +54,7 @@ void checkFinalInferredType( type = default_type; } - if (settings.schema_inference_make_columns_nullable) + if (settings.schema_inference_make_columns_nullable == 1) type = makeNullableRecursively(type); /// In case when data for some column could contain nulls and regular values, /// resulting inferred type is Nullable. diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 72a93002669..cf079e52db0 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -204,8 +204,11 @@ NamesAndTypesList ArrowSchemaReader::readSchema() schema = file_reader->schema(); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, stream ? "ArrowStream" : "Arrow", format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) + *schema, + stream ? "ArrowStream" : "Arrow", + format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference, + format_settings.schema_inference_make_columns_nullable != 0); + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index ed91913de4d..bcc8bfecdc6 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -712,6 +712,7 @@ struct ReadColumnFromArrowColumnSettings FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior; bool allow_arrow_null_type; bool skip_columns_with_unsupported_types; + bool allow_inferring_nullable_columns; }; static ColumnWithTypeAndName readColumnFromArrowColumn( @@ -1085,7 +1086,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( bool is_map_nested_column, const ReadColumnFromArrowColumnSettings & settings) { - bool read_as_nullable_column = arrow_column->null_count() || is_nullable_column || (type_hint && type_hint->isNullable()); + bool read_as_nullable_column = (arrow_column->null_count() || is_nullable_column || (type_hint && type_hint->isNullable())) && settings.allow_inferring_nullable_columns; if (read_as_nullable_column && arrow_column->type()->id() != arrow::Type::LIST && arrow_column->type()->id() != arrow::Type::LARGE_LIST && @@ -1149,14 +1150,16 @@ static std::shared_ptr createArrowColumn(const std::shared_ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader( const arrow::Schema & schema, const std::string & format_name, - bool skip_columns_with_unsupported_types) + bool skip_columns_with_unsupported_types, + bool allow_inferring_nullable_columns) { ReadColumnFromArrowColumnSettings settings { .format_name = format_name, .date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore, .allow_arrow_null_type = false, - .skip_columns_with_unsupported_types = skip_columns_with_unsupported_types + .skip_columns_with_unsupported_types = skip_columns_with_unsupported_types, + .allow_inferring_nullable_columns = allow_inferring_nullable_columns, }; ColumnsWithTypeAndName sample_columns; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 27e9afdf763..8521cd2f410 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -34,7 +34,8 @@ public: static Block arrowSchemaToCHHeader( const arrow::Schema & schema, const std::string & format_name, - bool skip_columns_with_unsupported_types = false); + bool skip_columns_with_unsupported_types = false, + bool allow_inferring_nullable_columns = true); struct DictionaryInfo { diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 58bec8120f1..b0fd6789d1a 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1002,7 +1002,7 @@ NamesAndTypesList NativeORCSchemaReader::readSchema() header.insert(ColumnWithTypeAndName{type, name}); } - if (format_settings.schema_inference_make_columns_nullable) + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index a3c218fa26e..2266c0b488c 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -160,8 +160,11 @@ NamesAndTypesList ORCSchemaReader::readSchema() { initializeIfNeeded(); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, "ORC", format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) + *schema, + "ORC", + format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference, + format_settings.schema_inference_make_columns_nullable != 0); + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index bc5e8292192..b116070b8df 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -810,8 +810,11 @@ NamesAndTypesList ParquetSchemaReader::readSchema() THROW_ARROW_NOT_OK(parquet::arrow::FromParquetSchema(metadata->schema(), &schema)); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, "Parquet", format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) + *schema, + "Parquet", + format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference, + format_settings.schema_inference_make_columns_nullable != 0); + if (format_settings.schema_inference_make_columns_nullable == 1) return getNamesAndRecursivelyNullableTypes(header); return header.getNamesAndTypesList(); } diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference index 985f8192f26..d15f0d8365d 100644 --- a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference @@ -1,40 +1,66 @@ Parquet a UInt64 a_nullable Nullable(UInt64) +a UInt64 +a_nullable UInt64 Arrow a UInt64 a_nullable Nullable(UInt64) +a UInt64 +a_nullable UInt64 Parquet b Array(UInt64) b_nullable Array(Nullable(UInt64)) +b Array(UInt64) +b_nullable Array(UInt64) Arrow b Array(Nullable(UInt64)) b_nullable Array(Nullable(UInt64)) +b Array(UInt64) +b_nullable Array(UInt64) Parquet c Tuple(\n a UInt64,\n b String) c_nullable Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +c Tuple(\n a UInt64,\n b String) +c_nullable Tuple(\n a UInt64,\n b String) Arrow c Tuple(\n a UInt64,\n b String) c_nullable Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +c Tuple(\n a UInt64,\n b String) +c_nullable Tuple(\n a UInt64,\n b String) Parquet d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) +d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String))) Arrow d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) +d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String))) Parquet e Map(UInt64, String) e_nullable Map(UInt64, Nullable(String)) +e Map(UInt64, String) +e_nullable Map(UInt64, String) Arrow e Map(UInt64, Nullable(String)) e_nullable Map(UInt64, Nullable(String)) +e Map(UInt64, String) +e_nullable Map(UInt64, String) Parquet f Map(UInt64, Map(UInt64, String)) f_nullables Map(UInt64, Map(UInt64, Nullable(String))) +f Map(UInt64, Map(UInt64, String)) +f_nullables Map(UInt64, Map(UInt64, String)) Arrow f Map(UInt64, Map(UInt64, Nullable(String))) f_nullables Map(UInt64, Map(UInt64, Nullable(String))) +f Map(UInt64, Map(UInt64, String)) +f_nullables Map(UInt64, Map(UInt64, String)) Parquet g String g_nullable Nullable(String) +g String +g_nullable String Arrow g LowCardinality(String) g_nullable LowCardinality(String) +g LowCardinality(String) +g_nullable LowCardinality(String) diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh b/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh index bdd641e2b94..379756f78f3 100755 --- a/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh @@ -14,6 +14,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, a_nullable Nullable(UInt64)', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -21,6 +22,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('b Array(UInt64), b_nullable Array(Nullable(UInt64))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -28,6 +30,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('c Tuple(a UInt64, b String), c_nullable Tuple(a Nullable(UInt64), b Nullable(String))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -35,6 +38,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('d Tuple(a UInt64, b Tuple(a UInt64, b String), d_nullable Tuple(a UInt64, b Tuple(a Nullable(UInt64), b Nullable(String))))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -42,6 +46,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('e Map(UInt64, String), e_nullable Map(UInt64, Nullable(String))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -49,6 +54,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('f Map(UInt64, Map(UInt64, String)), f_nullables Map(UInt64, Map(UInt64, Nullable(String)))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done @@ -56,6 +62,7 @@ for format in $formats do echo $format $CLICKHOUSE_LOCAL -q "select * from generateRandom('g LowCardinality(String), g_nullable LowCardinality(Nullable(String))', 42) limit 10 settings output_format_arrow_low_cardinality_as_dictionary=1, allow_suspicious_low_cardinality_types=1 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 'auto'" $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" done From 70708fd5dcf633d4d3147240195554587f4fb14f Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 19:19:02 +0000 Subject: [PATCH 0915/1722] Update docs, make better --- docs/en/interfaces/schema-inference.md | 7 ++----- docs/en/operations/settings/settings-formats.md | 4 ++-- src/Processors/Formats/ISchemaReader.cpp | 5 ----- 3 files changed, 4 insertions(+), 12 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 05fae994cbe..5b3cd179e21 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1385,7 +1385,7 @@ DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : nul #### schema_inference_make_columns_nullable Controls making inferred types `Nullable` in schema inference for formats without information about nullability. -If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will be `Nullable` only if `input_format_null_as_default` is disabled and the column contains `NULL` in a sample that is parsed during schema inference. +If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will never be `Nullable`, if set to `auto`, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference or file metadata contains information about column nullability. Enabled by default. @@ -1408,15 +1408,13 @@ DESC format(JSONEachRow, $$ └─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` ```sql -SET schema_inference_make_columns_nullable = 0; -SET input_format_null_as_default = 0; +SET schema_inference_make_columns_nullable = 'auto'; DESC format(JSONEachRow, $$ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]} {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]} $$) ``` ```response - ┌─name────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠│ id │ Int64 │ │ │ │ │ │ │ age │ Int64 │ │ │ │ │ │ @@ -1428,7 +1426,6 @@ DESC format(JSONEachRow, $$ ```sql SET schema_inference_make_columns_nullable = 0; -SET input_format_null_as_default = 1; DESC format(JSONEachRow, $$ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]} {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index f8b40cd81ac..57812ef0e03 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -171,8 +171,8 @@ If the `schema_inference_hints` is not formated properly, or if there is a typo ## schema_inference_make_columns_nullable {#schema_inference_make_columns_nullable} -Controls making inferred types `Nullable` in schema inference for formats without information about nullability. -If the setting is enabled, the inferred type will be `Nullable` only if column contains `NULL` in a sample that is parsed during schema inference. +Controls making inferred types `Nullable` in schema inference. +If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will never be `Nullable`, if set to `auto`, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference or file metadata contains information about column nullability. Default value: `true`. diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 569d4bb39e7..e002e64b7e5 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -56,11 +56,6 @@ void checkFinalInferredType( if (settings.schema_inference_make_columns_nullable == 1) type = makeNullableRecursively(type); - /// In case when data for some column could contain nulls and regular values, - /// resulting inferred type is Nullable. - /// If input_format_null_as_default is enabled, we should remove Nullable type. - else if (settings.null_as_default) - type = removeNullable(type); } void ISchemaReader::transformTypesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) From 6af5fedf420c667e2a7866c89dfe0bd1d2ff37dd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Aug 2024 19:26:35 +0000 Subject: [PATCH 0916/1722] Update autogenerated version to 24.9.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++++----- .../StorageSystemContributors.generated.cpp | 17 +++++++++++++++++ 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index d69646d3694..c82038804fe 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54489) +SET(VERSION_REVISION 54490) SET(VERSION_MAJOR 24) -SET(VERSION_MINOR 8) +SET(VERSION_MINOR 9) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 3f8b27d7accd2b5ec4afe7d0dd459115323304af) -SET(VERSION_DESCRIBE v24.8.1.1-testing) -SET(VERSION_STRING 24.8.1.1) +SET(VERSION_GITHASH e02b434d2fc0c4fbee29ca675deab7474d274608) +SET(VERSION_DESCRIBE v24.9.1.1-testing) +SET(VERSION_STRING 24.9.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 35b9c0008c6..eb6f0382d15 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -457,6 +457,7 @@ const char * auto_contributors[] { "Gleb-Tretyakov", "GoGoWen2021", "Gosha Letov", + "Graham Campbell", "Gregory", "Grigorii Sokolik", "Grigory", @@ -472,6 +473,7 @@ const char * auto_contributors[] { "Habibullah Oladepo", "HaiBo Li", "Hakob Saghatelyan", + "Halersson Paris", "Hamoon", "Han Fei", "Han Shukai", @@ -541,6 +543,7 @@ const char * auto_contributors[] { "JackyWoo", "Jacob Hayes", "Jacob Herrington", + "Jacob Reckhard", "Jai Jhala", "Jake Bamrah", "Jake Liu", @@ -661,6 +664,7 @@ const char * auto_contributors[] { "LaurieLY", "Lee sungju", "Lemore", + "Lennard Eijsackers", "Leonardo Cecchi", "Leonardo Maciel", "Leonid Krylov", @@ -804,6 +808,7 @@ const char * auto_contributors[] { "Mingliang Pan", "Misko Lee", "Misz606", + "MiÑhael Stetsyuk", "MochiXu", "Mohamad Fadhil", "Mohammad Arab Anvari", @@ -922,6 +927,7 @@ const char * auto_contributors[] { "Pervakov Grigorii", "Pervakov Grigory", "Peter", + "Peter Nguyen", "Petr Vasilev", "Pham Anh Tuan", "Philip Hallstrom", @@ -981,6 +987,7 @@ const char * auto_contributors[] { "Ronald Bradford", "Rory Crispin", "Roy Bellingan", + "Ruihang Xia", "Ruslan", "Ruslan Mardugalliamov", "Ruslan Savchenko", @@ -1000,9 +1007,11 @@ const char * auto_contributors[] { "Sami Kerola", "Samuel Chou", "Samuel Colvin", + "Samuele Guerrini", "San", "Sanjam Panda", "Sariel", + "Sasha Sheikin", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -1202,6 +1211,7 @@ const char * auto_contributors[] { "Vladimir Makarov", "Vladimir Mihailenco", "Vladimir Smirnov", + "Vladimir Varankin", "Vladislav Rassokhin", "Vladislav Smirnov", "Vladislav V", @@ -1275,6 +1285,7 @@ const char * auto_contributors[] { "Zhichun Wu", "Zhiguo Zhou", "Zhipeng", + "Zhukova, Maria", "Zhuo Qiu", "Zijie Lu", "Zimu Li", @@ -1502,6 +1513,7 @@ const char * auto_contributors[] { "hchen9", "hcz", "hdhoang", + "heguangnan", "heleihelei", "helifu", "hendrik-m", @@ -1572,6 +1584,7 @@ const char * auto_contributors[] { "kevinyhzou", "kgurjev", "khamadiev", + "khodyrevyurii", "kigerzhang", "kirillikoff", "kmeaw", @@ -1787,6 +1800,7 @@ const char * auto_contributors[] { "ruslandoga", "ryzuo", "s-kat", + "sakulali", "sanjam", "santaux", "santrancisco", @@ -1804,6 +1818,7 @@ const char * auto_contributors[] { "shabroo", "shangshujie", "shedx", + "shiyer7474", "shuai-xu", "shuchaome", "shuyang", @@ -1901,6 +1916,7 @@ const char * auto_contributors[] { "wzl", "xPoSx", "xbthink", + "xc0derx", "xiao", "xiaolei565", "xiebin", @@ -1964,6 +1980,7 @@ const char * auto_contributors[] { "zkun", "zlx19950903", "zombee0", + "zoomxi", "zvonand", "zvrr", "zvvr", From a9226f49e7e052d2c392214afe32f4d6de1d6d62 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 13 Aug 2024 20:24:40 +0000 Subject: [PATCH 0917/1722] remove name with cyrillic letter --- src/Storages/System/StorageSystemContributors.generated.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index eb6f0382d15..67dfe3bfe86 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -808,7 +808,6 @@ const char * auto_contributors[] { "Mingliang Pan", "Misko Lee", "Misz606", - "MiÑhael Stetsyuk", "MochiXu", "Mohamad Fadhil", "Mohammad Arab Anvari", From 24ddea69453588bb5c126cf9951c3b0bc6fcafcf Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 20:49:16 +0000 Subject: [PATCH 0918/1722] Fix tests --- src/Parsers/ParserDataType.cpp | 11 +++++------ .../02553_new_type_json_attach_partition.sql | 1 + 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index e941c05eba1..6e6e8758c51 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -242,10 +242,12 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// Allow mixed lists of nested and normal types. /// Parameters are either: - /// - Nested table elements; + /// - Nested table element; + /// - Tuple element /// - Enum element in form of 'a' = 1; /// - literal; - /// - Dynamic type arguments; + /// - Dynamic type argument; + /// - JSON type argument; /// - another data type (or identifier); size_t arg_num = 0; @@ -271,7 +273,7 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ObjectArgumentParser parser; parser.parse(pos, arg, expected); } - else if (type_name == "Nested") + else if (boost::to_lower_copy(type_name) == "nested") { ParserNameTypePair name_and_type_parser; name_and_type_parser.parse(pos, arg, expected); @@ -334,9 +336,6 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++arg_num; } - if (pos->type == TokenType::Comma) - // ignore trailing comma inside Nested structures like Tuple(Int, Tuple(Int, String),) - ++pos; if (pos->type != TokenType::ClosingRoundBracket) return false; ++pos; diff --git a/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql b/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql index 8a5abd31cb4..c7d4c0b5d55 100644 --- a/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql +++ b/tests/queries/0_stateless/02553_new_type_json_attach_partition.sql @@ -5,6 +5,7 @@ DROP TABLE IF EXISTS t_json_attach_partition; CREATE TABLE t_json_attach_partition(b UInt64, c JSON) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": 1}}; + ALTER TABLE t_json_attach_partition DETACH PARTITION tuple(); INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": [1, 2]}}; From 0133806e6c991325b5ace29469d8f450969c3ec0 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 13 Aug 2024 20:55:51 +0000 Subject: [PATCH 0919/1722] Fix --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 9 ++++++++ .../Formats/Impl/ParquetBlockInputFormat.cpp | 23 ++++++++----------- ...arquet_big_integer_compatibility.reference | 1 + ...02786_parquet_big_integer_compatibility.sh | 3 +++ 4 files changed, 23 insertions(+), 13 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index ed91913de4d..5e7f763dfbc 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -743,6 +743,15 @@ static ColumnWithTypeAndName readNonNullableColumnFromArrowColumn( case TypeIndex::IPv6: return readIPv6ColumnFromBinaryData(arrow_column, column_name); /// ORC format outputs big integers as binary column, because there is no fixed binary in ORC. + /// + /// When ORC/Parquet file says the type is "byte array" or "fixed len byte array", + /// but the clickhouse query says to interpret the column as e.g. Int128, it + /// may mean one of two things: + /// * The byte array is the 16 bytes of Int128, little-endian. + /// * The byte array is an ASCII string containing the Int128 formatted in base 10. + /// There's no reliable way to distinguish these cases. We just guess: if the + /// byte array is variable-length, and the length is different from sizeof(type), + /// we parse as text, otherwise as binary. case TypeIndex::Int128: return readColumnWithBigNumberFromBinaryData(arrow_column, column_name, type_hint); case TypeIndex::UInt128: diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3f2e701afc2..c6167e572df 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -224,21 +224,18 @@ static Field decodePlainParquetValueSlow(const std::string & data, parquet::Type if (data.empty()) return Field(); - /// Long integers. - auto reinterpret_fixed_string = [&](auto x) - { - if (data.size() != sizeof(x)) - throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected {} size: {}", fieldTypeToString(Field::TypeToEnum::value), data.size()); - memcpy(&x, data.data(), data.size()); - return Field(x); - }; + /// Long integers, encoded either as text or as little-endian bytes. + /// The parquet file doesn't know that it's numbers, so the min/max are produced by comparing + /// strings lexicographically. So these min and max are mostly useless to us. + /// There's one case where they're not useless: min == max; currently we don't make use of this. switch (type_hint) { - case TypeIndex::UInt128: return reinterpret_fixed_string(UInt128(0)); - case TypeIndex::UInt256: return reinterpret_fixed_string(UInt256(0)); - case TypeIndex::Int128: return reinterpret_fixed_string(Int128(0)); - case TypeIndex::Int256: return reinterpret_fixed_string(Int256(0)); - case TypeIndex::IPv6: return reinterpret_fixed_string(IPv6(0)); + case TypeIndex::UInt128: + case TypeIndex::UInt256: + case TypeIndex::Int128: + case TypeIndex::Int256: + case TypeIndex::IPv6: + return Field(); default: break; } diff --git a/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference index 7764974255b..877bb5f390f 100644 --- a/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference +++ b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference @@ -1 +1,2 @@ 424242424242424242424242424242424242424242424242424242 +22707864971053448441042714569797161695738549521977760418632926980540162388532 diff --git a/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh index 8865b2e7aab..0f590027f19 100755 --- a/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh +++ b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh @@ -5,5 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# This is parsed as text. $CLICKHOUSE_LOCAL -q "select toString(424242424242424242424242424242424242424242424242424242::UInt256) as x format Parquet" | $CLICKHOUSE_LOCAL --input-format=Parquet --structure='x UInt256' -q "select * from table" +# But this is parsed as binary because text length happens to be 32 bytes. Not ideal. +$CLICKHOUSE_LOCAL -q "select toString(42424242424242424242424242424242::UInt256) as x format Parquet" | $CLICKHOUSE_LOCAL --input-format=Parquet --structure='x UInt256' -q "select * from table" From b9ffa929ba418d54e5e140470f21d4347ac0eab9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Aug 2024 21:08:53 +0000 Subject: [PATCH 0920/1722] Fix: min marks to read overflow with parallel replicas --- .../MergeTree/MergeTreeIndexGranularity.cpp | 12 ++++++++++-- ..._replicas_min_marks_to_read_overflow.reference | 10 ++++++++++ ...rallel_replicas_min_marks_to_read_overflow.sql | 15 +++++++++++++++ 3 files changed, 35 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference create mode 100644 tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 2a45ab1d927..2f9a4a47b11 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -103,8 +103,16 @@ size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t num /// This is a heuristic to respect min_marks_to_read which is ignored by MergeTreeReadPool in case of remote disk. /// See comment in IMergeTreeSelectAlgorithm. - if (min_marks_to_read && from_mark + 2 * min_marks_to_read <= to_mark) - to_mark = from_mark + min_marks_to_read; + if (min_marks_to_read) + { + // check that ... + bool overflow = ((1ULL << 63) & min_marks_to_read); // further multiplication by 2 will not overflow + if (!overflow) + overflow = (std::numeric_limits::max() - from_mark) < 2 * min_marks_to_read; // further addition will not overflow + + if (!overflow && from_mark + 2 * min_marks_to_read <= to_mark) + to_mark = from_mark + min_marks_to_read; + } return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows; } diff --git a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference new file mode 100644 index 00000000000..7fafd4d13ea --- /dev/null +++ b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference @@ -0,0 +1,10 @@ +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 diff --git a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql new file mode 100644 index 00000000000..112373e5db2 --- /dev/null +++ b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test__fuzz_22 SYNC; + +CREATE TABLE test__fuzz_22 (k Float32, v String) ENGINE = ReplicatedMergeTree('/clickhouse/03222/{database}/test__fuzz_22', 'r1') ORDER BY k SETTINGS index_granularity = 1; + +INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(10_000); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT k, v +FROM test__fuzz_22 +ORDER BY k +LIMIT 100, 10 +SETTINGS merge_tree_min_rows_for_concurrent_read = 9223372036854775806; + +DROP TABLE test__fuzz_22 SYNC; From 0c9e1a061f825e5b9c5d623d90d4d898cd05e44c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 13 Aug 2024 18:49:18 +0200 Subject: [PATCH 0921/1722] CI: Create new release branch workflow updates --- .github/workflows/create_release.yml | 2 +- tests/ci/ci_utils.py | 5 + tests/ci/create_release.py | 240 ++++++++++++++++----------- tests/ci/docker_server.py | 2 +- tests/ci/version_helper.py | 17 +- 5 files changed, 161 insertions(+), 105 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index d4993b373df..73613c65266 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -129,9 +129,9 @@ jobs: if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash run: | - python3 ./tests/ci/create_release.py --set-progress-completed git reset --hard HEAD git checkout "$GITHUB_REF_NAME" + python3 ./tests/ci/create_release.py --set-progress-completed - name: Create GH Release if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} shell: bash diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index d807f5be09f..b8778e0cc50 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -167,6 +167,11 @@ class GH: latest_branch = Shell.get_output( 'gh pr list --label release --repo ClickHouse/ClickHouse --search "sort:created" -L1 --json headRefName' ) + if latest_branch: + latest_branch = json.loads(latest_branch)[0]["headRefName"] + print( + f"Latest branch [{latest_branch}], release branch [{branch}], release latest [{latest_branch == branch}]" + ) return latest_branch == branch diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 27eba273ce0..b5ea61e1952 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -61,6 +61,7 @@ class ReleaseContextManager: # create initial release info self.release_info = ReleaseInfo( release_branch="NA", + release_type="NA", commit_sha=args.ref, release_tag="NA", version="NA", @@ -93,6 +94,7 @@ class ReleaseContextManager: @dataclasses.dataclass class ReleaseInfo: version: str + release_type: str release_tag: str release_branch: str commit_sha: str @@ -131,7 +133,7 @@ class ReleaseInfo: return self def prepare( - self, commit_ref: str, release_type: str, skip_tag_check: bool + self, commit_ref: str, release_type: str, _skip_tag_check: bool ) -> "ReleaseInfo": version = None release_branch = None @@ -143,17 +145,18 @@ class ReleaseInfo: assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch - Shell.check( - f"git merge-base --is-ancestor {commit_ref} origin/master", - strict=True, - verbose=True, - ) + if commit_ref != "master": + Shell.check( + f"git merge-base --is-ancestor {commit_ref} origin/master", + strict=True, + verbose=True, + ) with checkout(commit_ref): commit_sha = Shell.get_output_or_raise(f"git rev-list -n1 {commit_ref}") # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) - release_branch = "master" + release_branch = f"{version.major}.{version.minor}" expected_prev_tag = f"v{version.major}.{version.minor}.1.1-new" version.bump().with_description(VersionType.NEW) assert ( @@ -204,10 +207,11 @@ class ReleaseInfo: expected_tag_prefix ) and git.latest_tag.endswith(expected_tag_suffix): pass - elif not skip_tag_check: - assert ( - False - ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]. Already Released?" + # TODO: uncomment and check with dry-run + # elif not skip_tag_check: + # assert ( + # False + # ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]. Already Released?" previous_release_sha = Shell.get_output_or_raise( f"git rev-list -n1 {previous_release_tag}" @@ -238,6 +242,7 @@ class ReleaseInfo: self.release_progress = ReleaseProgress.STARTED self.progress_status = ReleaseProgressDescription.OK self.latest = latest_release + self.release_type = release_type return self def push_release_tag(self, dry_run: bool) -> None: @@ -262,16 +267,15 @@ class ReleaseInfo: @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: cmd = f"gh api repos/{CI.Envs.GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" - Shell.check(cmd, dry_run=dry_run, strict=True) + res = Shell.check(cmd, dry_run=dry_run, verbose=True) + if not res: + # not a critical error - do not fail. branch might be created already (recovery case) + print("WARNING: failed to create backport labels for the new branch") def push_new_release_branch(self, dry_run: bool) -> None: - assert ( - self.release_branch == "master" - ), "New release branch can be created only for release type [new]" git = Git() version = get_version_from_repo(git=git) - new_release_branch = f"{version.major}.{version.minor}" - stable_release_type = version.get_stable_release_type() + new_release_branch = self.release_branch version_after_release = copy(version) version_after_release.bump() assert ( @@ -285,11 +289,8 @@ class ReleaseInfo: print( f"Create and push new release branch [{new_release_branch}], commit [{self.commit_sha}]" ) - with checkout(self.release_branch): + with checkout("master"): with checkout_new(new_release_branch): - pr_labels = f"--label {CI.Labels.RELEASE}" - if stable_release_type == VersionType.LTS: - pr_labels += f" --label {CI.Labels.RELEASE_LTS}" cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) @@ -302,67 +303,108 @@ class ReleaseInfo: ReleaseInfo._create_gh_label( f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) - Shell.check( - f"""gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' - --head {new_release_branch} {pr_labels} - --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' - """, - dry_run=dry_run, - strict=True, - verbose=True, - ) def get_version_bump_branch(self): return f"bump_version_{self.version}" def update_version_and_contributors_list(self, dry_run: bool) -> None: - # Bump version, update contributors list, create PR - branch_upd_version_contributors = self.get_version_bump_branch() + # Bump version, update contributors list, create on release branch with checkout(self.commit_sha): git = Git() version = get_version_from_repo(git=git) - if self.release_branch == "master": + if self.release_type == "patch": + assert ( + version.string == self.version + ), f"BUG: version in release info does not match version in git commit, expected [{self.version}], got [{version.string}]" + version.bump_patch() + else: + version.reset_tweak() + version.with_description(version.get_stable_release_type()) + + with checkout(self.release_branch): + update_cmake_version(version) + update_contributors(raise_error=True) + cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" + cmd_push_branch = f"{GIT_PREFIX} push" + Shell.check( + cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True + ) + Shell.check(cmd_push_branch, strict=True, dry_run=dry_run, verbose=True) + if dry_run: + Shell.check( + f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + Shell.check( + f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + + # TODO: move to new GH step? + if self.release_type == "new": + print("Update version on master branch") + branch_upd_version_contributors = self.get_version_bump_branch() + with checkout(self.commit_sha): + git = Git() + version = get_version_from_repo(git=git) version.bump() version.with_description(VersionType.TESTING) - else: - version.with_description(version.get_stable_release_type()) - assert ( - version.string == self.version - ), f"BUG: version in release info does not match version in git commit, expected [{self.version}], got [{version.string}]" - with checkout(self.release_branch): - with checkout_new(branch_upd_version_contributors): - update_cmake_version(version) - update_contributors(raise_error=True) - cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" - cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" - actor = os.getenv("GITHUB_ACTOR", "") or "me" - body = f"Automatic version bump after release {self.release_tag}\n### Changelog category (leave one):\n- Not for changelog (changelog entry is not required)\n" - cmd_create_pr = f"gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body \"{body}\" --assignee {actor}" + with checkout("master"): + with checkout_new(branch_upd_version_contributors): + update_cmake_version(version) + update_contributors(raise_error=True) + cmd_commit_version_upd = f"{GIT_PREFIX} commit '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}' -m 'Update autogenerated version to {self.version} and contributors'" + cmd_push_branch = f"{GIT_PREFIX} push --set-upstream origin {branch_upd_version_contributors}" + actor = os.getenv("GITHUB_ACTOR", "") or "me" + body = f"Automatic version bump after release {self.release_tag}\n### Changelog category (leave one):\n- Not for changelog (changelog entry is not required)\n" + cmd_create_pr = f"gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base master --body \"{body}\" --assignee {actor}" + Shell.check( + cmd_commit_version_upd, + strict=True, + dry_run=dry_run, + verbose=True, + ) + Shell.check( + cmd_push_branch, strict=True, dry_run=dry_run, verbose=True + ) + Shell.check( + cmd_create_pr, strict=True, dry_run=dry_run, verbose=True + ) + if dry_run: + Shell.check( + f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + Shell.check( + f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", + verbose=True, + ) + self.version_bump_pr = "dry-run" + else: + self.version_bump_pr = GH.get_pr_url_by_branch( + branch=branch_upd_version_contributors + ) + + # TODO: move to new GH step? + print("Create Release PR") + with checkout(self.release_branch): + pr_labels = f"--label {CI.Labels.RELEASE}" + if version.get_stable_release_type() == VersionType.LTS: + pr_labels += f" --label {CI.Labels.RELEASE_LTS}" Shell.check( - cmd_commit_version_upd, strict=True, dry_run=dry_run, verbose=True + f"""gh pr create --repo {CI.Envs.GITHUB_REPOSITORY} --title 'Release pull request for branch {self.release_branch}' \ + --head {self.release_branch} {pr_labels} \ + --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.'""", + dry_run=dry_run, + strict=True, + verbose=True, ) - Shell.check(cmd_push_branch, strict=True, dry_run=dry_run, verbose=True) - Shell.check(cmd_create_pr, strict=True, dry_run=dry_run, verbose=True) - if dry_run: - Shell.check( - f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", - verbose=True, - ) - Shell.check( - f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'", - verbose=True, - ) - self.version_bump_pr = "dry-run" - else: - self.version_bump_pr = GH.get_pr_url_by_branch( - branch=branch_upd_version_contributors - ) def get_change_log_branch(self): return f"auto/{self.release_tag}" def update_release_info(self, dry_run: bool) -> "ReleaseInfo": - if self.release_branch != "master": + if self.release_type == "patch": if not self.changelog_pr: branch = self.get_change_log_branch() if not dry_run: @@ -371,21 +413,22 @@ class ReleaseInfo: url = "dry-run" print(f"ChangeLog PR url [{url}]") self.changelog_pr = url - - if not self.version_bump_pr: - branch = self.get_version_bump_branch() - if not dry_run: - url = GH.get_pr_url_by_branch(branch=branch) - else: - url = "dry-run" - print(f"Version bump PR url [{url}]") - self.version_bump_pr = url - - self.release_url = f"https://github.com/{CI.Envs.GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" - print(f"Release url [{self.release_url}]") - self.docker = f"docker run --rm clickhouse/clickhouse:{self.version} clickhouse --version" + else: + # new release branch - find version bump pr on a master branch + branch = self.get_version_bump_branch() + if not dry_run: + url = GH.get_pr_url_by_branch(branch=branch) + else: + url = "dry-run" + print(f"Version bump PR url [{url}]") + self.version_bump_pr = url + + self.release_url = f"https://github.com/{CI.Envs.GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + print(f"Release url [{self.release_url}]") + self.dump() + return self def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: @@ -410,35 +453,40 @@ class ReleaseInfo: def merge_prs(self, dry_run: bool) -> None: repo = CI.Envs.GITHUB_REPOSITORY - assert self.version_bump_pr - if dry_run: - version_bump_pr_num = 12345 - else: - version_bump_pr_num = int(self.version_bump_pr.split("/")[-1]) - print("Merging Version bump PR") - res_1 = Shell.check( - f"gh pr merge {version_bump_pr_num} --repo {repo} --merge --auto", - verbose=True, - dry_run=dry_run, - ) - - res_2 = True - if not self.release_tag.endswith("-new"): + if self.release_type == "patch": assert self.changelog_pr print("Merging ChangeLog PR") if dry_run: changelog_pr_num = 23456 else: changelog_pr_num = int(self.changelog_pr.split("/")[-1]) - res_2 = Shell.check( + res = Shell.check( f"gh pr merge {changelog_pr_num} --repo {repo} --merge --auto", verbose=True, dry_run=dry_run, ) else: - assert not self.changelog_pr + if not dry_run: + assert not self.changelog_pr + res = True - self.prs_merged = res_1 and res_2 + if self.release_type == "new": + assert self.version_bump_pr + print("Merging Version Bump PR") + if dry_run: + version_bump_pr = 23456 + else: + version_bump_pr = int(self.version_bump_pr.split("/")[-1]) + res = res and Shell.check( + f"gh pr merge {version_bump_pr} --repo {repo} --merge --auto", + verbose=True, + dry_run=dry_run, + ) + else: + if not dry_run: + assert not self.changelog_pr + + self.prs_merged = res class RepoTypes: @@ -759,7 +807,7 @@ if __name__ == "__main__": release_info.prepare( commit_ref=args.ref, release_type=args.release_type, - skip_tag_check=args.skip_tag_check, + _skip_tag_check=args.skip_tag_check, ) if args.download_packages: diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 3251ec5644e..34439c19f0a 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -70,7 +70,7 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--tag-type", type=str, - choices=("head", "release", "latest-release"), + choices=("head", "release", "release-latest"), default="head", help="defines required tags for resulting docker image. " "head - for master image (tag: head) " diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 07a7a9601c0..b20b2bb25cf 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -85,6 +85,16 @@ class ClickHouseVersion: self._tweak = 1 return self + def bump_patch(self) -> "ClickHouseVersion": + self._revision += 1 + self._patch += 1 + self._tweak = 1 + return self + + def reset_tweak(self) -> "ClickHouseVersion": + self._tweak = 1 + return self + def major_update(self) -> "ClickHouseVersion": if self._git is not None: self._git.update() @@ -104,13 +114,6 @@ class ClickHouseVersion: self.major, self.minor, self.patch + 1, self.revision, self._git ) - def reset_tweak(self) -> "ClickHouseVersion": - if self._git is not None: - self._git.update() - return ClickHouseVersion( - self.major, self.minor, self.patch, self.revision, self._git, 1 - ) - @property def major(self) -> int: return self._major From b5134fd4903b91250bb6db16a8d52ff0b2469686 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Aug 2024 16:13:25 +0100 Subject: [PATCH 0922/1722] fix build --- base/base/cgroupsv2.cpp | 6 +++++- base/base/cgroupsv2.h | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index b4ca8271d64..e0e37c8729b 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -51,8 +51,9 @@ fs::path cgroupV2PathOfProcess() #endif } -std::optional getCgroupsV2PathContainingFile(std::string_view file_name) +std::optional getCgroupsV2PathContainingFile([[maybe_unused]] std::string_view file_name) { +#if defined(OS_LINUX) if (!cgroupsV2Enabled()) return {}; @@ -70,4 +71,7 @@ std::optional getCgroupsV2PathContainingFile(std::string_view file_ current_cgroup = current_cgroup.parent_path(); } return {}; +#else + return {}; +#endif } diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h index 925a399471e..a6276474254 100644 --- a/base/base/cgroupsv2.h +++ b/base/base/cgroupsv2.h @@ -19,4 +19,4 @@ std::filesystem::path cgroupV2PathOfProcess(); /// Returns the most nested cgroup dir containing the specified file. /// If cgroups v2 is not enabled - returns an empty optional. -std::optional getCgroupsV2PathContainingFile(std::string_view file_name); +std::optional getCgroupsV2PathContainingFile([[maybe_unused]] std::string_view file_name); From 835fc9ca7601df8295997aaf22e8f1e5792abc90 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 22:37:46 +0000 Subject: [PATCH 0923/1722] Add more tests, fix conflicts --- .../Serializations/SerializationObject.cpp | 7 ++++++ ...n_merges_new_type_in_shared_data.reference | 4 +++ ...24_json_merges_new_type_in_shared_data.sql | 12 +++++++++ ...n_merges_new_type_in_shared_data.reference | 22 ++++++++++++++++ ...ed_json_merges_new_type_in_shared_data.sql | 25 +++++++++++++++++++ 5 files changed, 70 insertions(+) create mode 100644 tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.reference create mode 100644 tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.sql create mode 100644 tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.reference create mode 100644 tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.sql diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index a1580ffedd4..2dd25e540cc 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -411,6 +411,12 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( const auto & shared_data = column_object.getSharedDataPtr(); auto * object_state = checkAndGetState(state); + if (column_object.getMaxDynamicPaths() != object_state->max_dynamic_paths) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of max_dynamic_paths parameter of Object. Expected: {}, Got: {}", object_state->max_dynamic_paths, column_object.getMaxDynamicPaths()); + + if (column_object.getDynamicPaths().size() != object_state->sorted_dynamic_paths.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch of number of dynamic paths in Object. Expected: {}, Got: {}", object_state->sorted_dynamic_paths.size(), column_object.getDynamicPaths().size()); + settings.path.push_back(Substream::ObjectData); for (const auto & path : sorted_typed_paths) @@ -532,6 +538,7 @@ void SerializationObject::deserializeBinaryBulkWithMultipleStreams( /// If it's a new object column, set dynamic paths and statistics. if (column_object.empty()) { + column_object.setMaxDynamicPaths(structure_state->max_dynamic_paths); column_object.setDynamicPaths(structure_state->sorted_dynamic_paths); column_object.setStatistics(structure_state->statistics); } diff --git a/tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.reference b/tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.reference new file mode 100644 index 00000000000..9d58b3a35db --- /dev/null +++ b/tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.reference @@ -0,0 +1,4 @@ +Array(Nullable(Int64)) true +Int64 false +Array(Nullable(Int64)) false +Int64 false diff --git a/tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.sql b/tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.sql new file mode 100644 index 00000000000..c96d67c0d47 --- /dev/null +++ b/tests/queries/0_stateless/03224_json_merges_new_type_in_shared_data.sql @@ -0,0 +1,12 @@ +set allow_experimental_json_type = 1; + +drop table if exists test; +create table test (json JSON(max_dynamic_paths=1)) engine=MergeTree order by tuple() settings min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1; +insert into test select '{"b" : 42}' from numbers(5); +insert into test select '{"a" : 42, "b" : [1, 2, 3]}' from numbers(5); +optimize table test final; +select distinct dynamicType(json.b) as type, isDynamicElementInSharedData(json.b) from test order by type; +insert into test select '{"b" : 42}' from numbers(5); +optimize table test final; +select distinct dynamicType(json.b) as type, isDynamicElementInSharedData(json.b) from test order by type; +drop table test; diff --git a/tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.reference b/tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.reference new file mode 100644 index 00000000000..b45d9bb97da --- /dev/null +++ b/tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.reference @@ -0,0 +1,22 @@ +Array(JSON(max_dynamic_types=16, max_dynamic_paths=2)) true +Int64 false +Array(JSON(max_dynamic_types=16, max_dynamic_paths=2)) false +Int64 false +['c'] +['d'] +Array(JSON(max_dynamic_types=16, max_dynamic_paths=2)) false +Int64 false +['c'] +['d'] +Int64 true +None false +Int64 true +None false +Array(JSON(max_dynamic_types=16, max_dynamic_paths=2)) false +Int64 false +['c'] +['d'] +Int64 false +None false +Int64 false +None false diff --git a/tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.sql b/tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.sql new file mode 100644 index 00000000000..b22b8b4fb75 --- /dev/null +++ b/tests/queries/0_stateless/03224_nested_json_merges_new_type_in_shared_data.sql @@ -0,0 +1,25 @@ +set allow_experimental_json_type = 1; + +drop table if exists test; +create table test (json JSON(max_dynamic_paths=8)) engine=MergeTree order by tuple() settings min_rows_for_wide_part = 1, min_bytes_for_wide_part = 1; +insert into test select materialize('{"a" : 42}')::JSON(max_dynamic_paths=8) from numbers(5); +insert into test select materialize('{"a1" : 42, "a2" : 42, "a3" : 42, "a4" : 42, "a5" : 42, "a6" : 42, "a7" : 42, "a8" : 42, "a" : [{"c" : 42}]}')::JSON(max_dynamic_paths=8) from numbers(5); +optimize table test final; +select distinct dynamicType(json.a) as type, isDynamicElementInSharedData(json.a) from test order by type; +insert into test select materialize('{"a1" : 42, "a2" : 42, "a3" : 42, "a4" : 42, "a5" : 42, "a6" : 42, "a7" : 42, "a8" : 42, "a" : [{"d" : 42}]}')::JSON(max_dynamic_paths=8) from numbers(5); +optimize table test final; +select distinct dynamicType(json.a) as type, isDynamicElementInSharedData(json.a) from test order by type; +select distinct JSONSharedDataPaths(arrayJoin(json.a[])) as path from test order by path; +insert into test select materialize('{"a" : 42}')::JSON(max_dynamic_paths=8) from numbers(5); +optimize table test final; +select distinct dynamicType(json.a) as type, isDynamicElementInSharedData(json.a) from test order by type; +select distinct JSONDynamicPaths(arrayJoin(json.a[])) as path from test order by path; +select distinct dynamicType(arrayJoin(json.a[].c)) as type, isDynamicElementInSharedData(arrayJoin(json.a[].c)) from test order by type; +select distinct dynamicType(arrayJoin(json.a[].d)) as type, isDynamicElementInSharedData(arrayJoin(json.a[].d)) from test order by type; +insert into test select materialize('{"a" : 42}')::JSON(max_dynamic_paths=8) from numbers(5); +optimize table test final; +select distinct dynamicType(json.a) as type, isDynamicElementInSharedData(json.a) from test order by type; +select distinct JSONDynamicPaths(arrayJoin(json.a[])) as path from test order by path; +select distinct dynamicType(arrayJoin(json.a[].c)) as type, isDynamicElementInSharedData(arrayJoin(json.a[].c)) from test order by type; +select distinct dynamicType(arrayJoin(json.a[].d)) as type, isDynamicElementInSharedData(arrayJoin(json.a[].d)) from test order by type; +drop table test; From 217963757e9a57fd3cadb4de68f6c2914ba67a91 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 23:21:24 +0000 Subject: [PATCH 0924/1722] Fix --- src/Columns/ColumnDynamic.cpp | 26 +++++++++++++------------- src/Columns/ColumnDynamic.h | 2 +- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 212e1adabca..1f37add9d2d 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -16,7 +16,6 @@ #include #include #include -#include namespace DB { @@ -56,6 +55,7 @@ ColumnDynamic::ColumnDynamic(size_t max_dynamic_types_) : max_dynamic_types(max_ ColumnDynamic::ColumnDynamic( MutableColumnPtr variant_column_, const DataTypePtr & variant_type_, size_t max_dynamic_types_, size_t global_max_dynamic_types_, const StatisticsPtr & statistics_) : variant_column(std::move(variant_column_)) + , variant_column_ptr(assert_cast(variant_column.get())) , max_dynamic_types(max_dynamic_types_) , global_max_dynamic_types(global_max_dynamic_types_) , statistics(statistics_) @@ -320,7 +320,7 @@ void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) } auto & variant_col = getVariantColumn(); - const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); + const auto & src_variant_col = dynamic_src.getVariantColumn(); auto src_global_discr = src_variant_col.globalDiscriminatorAt(n); auto src_offset = src_variant_col.offsetAt(n); @@ -346,7 +346,7 @@ void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_column_ptr->insertFrom(*dynamic_src.variant_column, n, *global_discriminators_mapping); + variant_col.insertFrom(*dynamic_src.variant_column, n, *global_discriminators_mapping); return; } @@ -364,7 +364,7 @@ void ColumnDynamic::doInsertFrom(const IColumn & src_, size_t n) if (addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator[dynamic_src.variant_info.variant_names[src_global_discr]]; - variant_column_ptr->insertIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); + variant_col.insertIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(n)); return; } @@ -388,16 +388,15 @@ void ColumnDynamic::doInsertRangeFrom(const IColumn & src_, size_t start, size_t "[start({}) + length({}) > src.size()({})]", start, length, src_.size()); const auto & dynamic_src = assert_cast(src_); + auto & variant_col = getVariantColumn(); /// Check if we have the same variants in both columns. if (variant_info.variant_names == dynamic_src.variant_info.variant_names) { - variant_column_ptr->insertRangeFrom(*dynamic_src.variant_column, start, length); + variant_col.insertRangeFrom(*dynamic_src.variant_column, start, length); return; } - auto & variant_col = getVariantColumn(); - /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { @@ -604,15 +603,15 @@ void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size #endif { const auto & dynamic_src = assert_cast(src_); + auto & variant_col = getVariantColumn(); /// Check if we have the same variants in both columns. if (variant_info.variant_names == dynamic_src.variant_info.variant_names) { - variant_column_ptr->insertManyFrom(*dynamic_src.variant_column, position, length); + variant_col.insertManyFrom(*dynamic_src.variant_column, position, length); return; } - auto & variant_col = getVariantColumn(); const auto & src_variant_col = assert_cast(*dynamic_src.variant_column); auto src_global_discr = src_variant_col.globalDiscriminatorAt(position); auto src_offset = src_variant_col.offsetAt(position); @@ -647,7 +646,7 @@ void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size /// If variants are different, we need to extend our variant with new variants. if (auto * global_discriminators_mapping = combineVariants(dynamic_src.variant_info)) { - variant_column_ptr->insertManyFrom(*dynamic_src.variant_column, position, length, *global_discriminators_mapping); + variant_col.insertManyFrom(*dynamic_src.variant_column, position, length, *global_discriminators_mapping); return; } @@ -663,7 +662,7 @@ void ColumnDynamic::doInsertManyFrom(const IColumn & src_, size_t position, size if (addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator[dynamic_src.variant_info.variant_names[src_global_discr]]; - variant_column_ptr->insertManyIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position), length); + variant_col.insertManyIntoVariantFrom(discr, src_variant_col.getVariantByGlobalDiscriminator(src_global_discr), src_variant_col.offsetAt(position), length); return; } @@ -810,7 +809,8 @@ const char * ColumnDynamic::skipSerializedInArena(const char * pos) const void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const { - auto discr = variant_column_ptr->globalDiscriminatorAt(n); + const auto & variant_col = getVariantColumn(); + auto discr = variant_col.globalDiscriminatorAt(n); if (discr == ColumnVariant::NULL_DISCRIMINATOR) { hash.update(discr); @@ -818,7 +818,7 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const } hash.update(variant_info.variant_names[discr]); - variant_column_ptr->getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_column_ptr->offsetAt(n), hash); + variant_col.getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_col.offsetAt(n), hash); } #if !defined(DEBUG_OR_SANITIZER_BUILD) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index eb454881d68..2ae862de3af 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -106,7 +106,7 @@ public: return create(variant_column_->assumeMutable(), variant_type, max_dynamic_types_, global_max_dynamic_types_, statistics_); } - static MutablePtr create(size_t max_dynamic_types_ = ColumnVariant::MAX_NESTED_COLUMNS) + static MutablePtr create(size_t max_dynamic_types_ = MAX_DYNAMIC_TYPES_LIMIT) { return Base::create(max_dynamic_types_); } From 46f41d171863ba968dacf6cf224ae5da751b87c1 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Aug 2024 23:51:49 +0000 Subject: [PATCH 0925/1722] Fix --- src/Parsers/ParserDataType.cpp | 2 +- .../01548_create_table_compound_column_format.reference | 4 ++-- .../0_stateless/01548_create_table_compound_column_format.sh | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 6e6e8758c51..d86b659df90 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -273,7 +273,7 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ObjectArgumentParser parser; parser.parse(pos, arg, expected); } - else if (boost::to_lower_copy(type_name) == "nested") + else if (type_name == "Nested") { ParserNameTypePair name_and_type_parser; name_and_type_parser.parse(pos, arg, expected); diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.reference b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference index c6c4dcdfa4a..21e31e8f034 100644 --- a/tests/queries/0_stateless/01548_create_table_compound_column_format.reference +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference @@ -1,12 +1,12 @@ CREATE TABLE test ( `a` Int64, - `b` NESTED(a Int64) + `b` Nested(a Int64) ) ENGINE = TinyLog CREATE TABLE test ( `a` Int64, - `b` TUPLE(a Int64) + `b` Tuple(a Int64) ) ENGINE = TinyLog diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.sh b/tests/queries/0_stateless/01548_create_table_compound_column_format.sh index 99e3aed2825..9065af17dc1 100755 --- a/tests/queries/0_stateless/01548_create_table_compound_column_format.sh +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.sh @@ -4,6 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo "CREATE TABLE test(a Int64, b NESTED(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT +echo "CREATE TABLE test(a Int64, b Nested(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT -echo "CREATE TABLE test(a Int64, b TUPLE(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT \ No newline at end of file +echo "CREATE TABLE test(a Int64, b Tuple(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT \ No newline at end of file From f6f79e188d6c5a16bb327f9e62dce506ea8e8a19 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 00:23:18 +0000 Subject: [PATCH 0926/1722] Apply libunwind changes needed for musl --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index a89d904befe..601db0b0e03 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 +Subproject commit 601db0b0e03018c01710470a37703b618f9cf08b From 0dc7cd7eb40bd3fb80e2c05871ce193a720b296c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 01:12:11 +0000 Subject: [PATCH 0927/1722] Update musl to have unwind info --- contrib/llvm-project-cmake/CMakeLists.txt | 6 ++++++ contrib/sysroot | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index 76e620314a2..f5dce1c4178 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -140,6 +140,12 @@ if (CMAKE_CROSSCOMPILING) message (STATUS "CROSS COMPILING SET LLVM HOST TRIPLE ${LLVM_HOST_TRIPLE}") endif() +# llvm-project/llvm/cmake/config-ix.cmake does a weird thing: it defines _LARGEFILE64_SOURCE, +# then checks if lseek64() function exists, then undefines _LARGEFILE64_SOURCE. +# Then the actual code that uses this function *doesn't* define _LARGEFILE64_SOURCE, so lseek64() +# may not exist and compilation fails. This happens with musl. +add_compile_definitions("_LARGEFILE64_SOURCE") + add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}") set_directory_properties (PROPERTIES diff --git a/contrib/sysroot b/contrib/sysroot index cc385041b22..866364fea62 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit cc385041b226d1fc28ead14dbab5d40a5f821dd8 +Subproject commit 866364fea629aa3e519ec967836561a6b3b21885 From e302e2ef323045bfa7158c0890594ec6ee75c8f1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 01:15:16 +0000 Subject: [PATCH 0928/1722] Conflicts --- src/Interpreters/InterpreterSystemQuery.cpp | 3 ++- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- src/Storages/MaterializedView/RefreshTask.h | 3 ++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index aa69854fee8..21c8b44b374 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -667,7 +667,8 @@ BlockIO InterpreterSystemQuery::execute() task->run(); break; case Type::WAIT_VIEW: - getRefreshTask()->wait(); + for (const auto & task : getRefreshTasks()) + task->wait(); break; case Type::CANCEL_VIEW: for (const auto & task : getRefreshTasks()) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 0b51ae85116..ed5a6652288 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -37,7 +37,7 @@ RefreshTask::RefreshTask( refresh_settings.applyChanges(strategy.settings->changes); } -RefreshTaskHolder RefreshTask::create( +OwnedRefreshTask RefreshTask::create( StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index a17932d34e8..ad9d949e18e 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -26,7 +26,8 @@ public: RefreshTask(StorageMaterializedView * view_, const ASTRefreshStrategy & strategy); /// The only proper way to construct task - static RefreshTaskHolder create( + static OwnedRefreshTask create( + StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); From 0abca8b7ddbafa37da5b1196b21fb816999fd334 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 14 Aug 2024 09:57:59 +0800 Subject: [PATCH 0929/1722] fix doc --- .../sql-reference/functions/string-replace-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 1caa6215b6b..d086c9ee64b 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -263,8 +263,8 @@ overlay(s, replace, position[, length]) - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int.md). -- `length`: Optional. An integer type [Int](../data-types/int.md). +- `position`: An integer type [Int](../data-types/int-uint.md). +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** @@ -300,8 +300,8 @@ overlayUTF8(s, replace, position[, length]) - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int.md). -- `length`: Optional. An integer type [Int](../data-types/int.md). +- `position`: An integer type [Int](../data-types/int-uint.md). +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** From 6bed26a52764558ba2b52752bdfd126fd943c616 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 14 Aug 2024 02:19:34 +0000 Subject: [PATCH 0930/1722] tidy --- src/Storages/MergeTree/MergeTask.cpp | 171 ++++++++++++++------------- src/Storages/MergeTree/MergeTask.h | 3 +- 2 files changed, 90 insertions(+), 84 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3b117168e33..56e17ac1884 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -145,7 +145,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu } } - for (const auto & projection : global_ctx->projections_to_rebuild) + for (const auto * projection : global_ctx->projections_to_rebuild) { Names projection_columns_vec = projection->getRequiredColumns(); std::copy(projection_columns_vec.cbegin(), projection_columns_vec.cend(), @@ -496,17 +496,76 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() } +void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const +{ + const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; + /// Under throw mode, we still choose to drop projections due to backward compatibility since some + /// users might have projections before this change. + if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary + && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) + return; + + /// These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. + const bool merge_may_reduce_rows = + global_ctx->cleanup || + global_ctx->deduplicate || + ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || + ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || + ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; + + const auto & projections = global_ctx->metadata_snapshot->getProjections(); + + for (const auto & projection : projections) + { + if (merge_may_reduce_rows) + { + global_ctx->projections_to_rebuild.push_back(&projection); + continue; + } + + MergeTreeData::DataPartsVector projection_parts; + for (const auto & part : global_ctx->future_part->parts) + { + auto it = part->getProjectionParts().find(projection.name); + if (it != part->getProjectionParts().end()) + projection_parts.push_back(it->second); + } + if (projection_parts.size() == global_ctx->future_part->parts.size()) + { + global_ctx->projections_to_merge.push_back(&projection); + global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); + } + else if (projection_parts.empty()) + { + LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); + } + else + { + LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); + global_ctx->projections_to_rebuild.push_back(&projection); + } + } + + const auto & settings = global_ctx->context->getSettingsRef(); + + for (const auto * projection : global_ctx->projections_to_rebuild) + ctx->projection_squashes.emplace_back(projection->sample_block.cloneEmpty(), + settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); +} + + void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Block & block) const { for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) { const auto & projection = *global_ctx->projections_to_rebuild[i]; Block block_to_squash = projection.calculate(block, global_ctx->context); - ctx->projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - auto squashed_chunk = Squashing::squash(ctx->projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); + auto & projection_squash_plan = ctx->projection_squashes[i]; + projection_squash_plan.setHeader(block_to_squash.cloneEmpty()); + Chunk squashed_chunk = Squashing::squash(projection_squash_plan.add({block_to_squash.getColumns(), block_to_squash.rows()})); if (squashed_chunk) { - auto result = ctx->projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); tmp_part.finalize(); @@ -517,6 +576,30 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo } +void MergeTask::ExecuteAndFinalizeHorizontalPart::finalizeProjections() const +{ + for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) + { + const auto & projection = *global_ctx->projections_to_rebuild[i]; + auto & projection_squash_plan = ctx->projection_squashes[i]; + auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + if (squashed_chunk) + { + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( + *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); + temp_part.finalize(); + temp_part.part->getDataPartStorage().commitTransaction(); + ctx->projection_parts[projection.name].emplace_back(std::move(temp_part.part)); + } + } + + ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); + if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) + constructTaskForProjectionPartsMerge(); +} + + void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPartsMerge() const { auto && [name, parts] = *ctx->projection_parts_iterator; @@ -591,27 +674,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() return true; } - /// finalize projections - // calculateProjections(global_ctx->merging_executor->getHeader().cloneEmpty()); - for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) - { - const auto & projection = *global_ctx->projections_to_rebuild[i]; - auto & projection_squash_plan = ctx->projection_squashes[i]; - auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); - if (squashed_chunk) - { - auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *global_ctx->data, ctx->log, result, projection, global_ctx->new_data_part.get(), ++ctx->projection_block_num); - temp_part.finalize(); - temp_part.part->getDataPartStorage().commitTransaction(); - ctx->projection_parts[projection.name].emplace_back(std::move(temp_part.part)); - } - } - - ctx->projection_parts_iterator = std::make_move_iterator(ctx->projection_parts.begin()); - if (ctx->projection_parts_iterator != std::make_move_iterator(ctx->projection_parts.end())) - constructTaskForProjectionPartsMerge(); + finalizeProjections(); global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -1285,64 +1348,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->merging_executor = std::make_unique(global_ctx->merged_pipeline); } -void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRebuild() const -{ - const auto mode = global_ctx->data->getSettings()->deduplicate_merge_projection_mode; - /// Under throw mode, we still choose to drop projections due to backward compatibility since some - /// users might have projections before this change. - if (global_ctx->data->merging_params.mode != MergeTreeData::MergingParams::Ordinary - && (mode == DeduplicateMergeProjectionMode::THROW || mode == DeduplicateMergeProjectionMode::DROP)) - return; - - // These merging modes may or may not reduce number of rows. It's not known until the horizontal stage is finished. - const bool merge_may_reduce_rows = - global_ctx->cleanup || - global_ctx->deduplicate || - ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || - ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || - ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; - - const auto & projections = global_ctx->metadata_snapshot->getProjections(); - - for (const auto & projection : projections) - { - if (merge_may_reduce_rows) - { - global_ctx->projections_to_rebuild.push_back(&projection); - continue; - } - - MergeTreeData::DataPartsVector projection_parts; - for (const auto & part : global_ctx->future_part->parts) - { - auto it = part->getProjectionParts().find(projection.name); - if (it != part->getProjectionParts().end()) - projection_parts.push_back(it->second); - } - if (projection_parts.size() == global_ctx->future_part->parts.size()) - { - global_ctx->projections_to_merge.push_back(&projection); - global_ctx->projections_to_merge_parts[projection.name].assign(projection_parts.begin(), projection_parts.end()); - } - else if (projection_parts.empty()) - { - LOG_DEBUG(ctx->log, "Projection {} will not be merged or rebuilt because all parts don't have it", projection.name); - } - else - { - LOG_DEBUG(ctx->log, "Projection {} will be rebuilt because some parts don't have it", projection.name); - global_ctx->projections_to_rebuild.push_back(&projection); - } - } - - const auto & settings = global_ctx->context->getSettingsRef(); - - for (const auto * projection : global_ctx->projections_to_rebuild) - { - ctx->projection_squashes.emplace_back(projection->sample_block.cloneEmpty(), settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); - } -} - MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm() const { diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index f60a32252b2..d9cc7c1dbad 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -227,7 +227,7 @@ private: std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; - // For projections to rebuild + /// For projections to rebuild using ProjectionNameToItsBlocks = std::map; ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; @@ -275,6 +275,7 @@ private: void prepareProjectionsToMergeAndRebuild() const; void calculateProjections(const Block & block) const; + void finalizeProjections() const; void constructTaskForProjectionPartsMerge() const; bool executeMergeProjections(); From da1c98a771c5b6cd74b3e1ba00fd4e01574489e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 14 Aug 2024 04:54:33 +0200 Subject: [PATCH 0931/1722] Update the limits --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 874095e39dc..2e6e7bbebe5 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -391,8 +391,8 @@ done # wait for minio to flush its batch if it has any sleep 1 clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE" -clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" -clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" +clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 --max_result_bytes 0 --max_result_rows 0 --max_rows_to_read 0 --max_bytes_to_read 0 -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow" +clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 --max_result_bytes 0 --max_result_rows 0 --max_rows_to_read 0 --max_bytes_to_read 0 -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow" # Stop server so we can safely read data with clickhouse-local. # Why do we read data with clickhouse-local? From 0716b460db35524f7cfa82501b5d1d2812904688 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 03:00:51 +0000 Subject: [PATCH 0932/1722] Fix duplicate symbol errors --- base/harmful/harmful.c | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 54b552a84ea..19bb962999f 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -66,13 +66,11 @@ TRAP(gethostbyname) TRAP(gethostbyname2) TRAP(gethostent) TRAP(getlogin) -TRAP(getmntent) TRAP(getnetbyaddr) TRAP(getnetbyname) TRAP(getnetent) TRAP(getnetgrent) TRAP(getnetgrent_r) -TRAP(getopt) TRAP(getopt_long) TRAP(getopt_long_only) TRAP(getpass) @@ -133,7 +131,6 @@ TRAP(nrand48) TRAP(__ppc_get_timebase_freq) TRAP(ptsname) TRAP(putchar_unlocked) -TRAP(putenv) TRAP(pututline) TRAP(pututxline) TRAP(putwchar_unlocked) @@ -148,7 +145,6 @@ TRAP(sethostent) TRAP(sethostid) TRAP(setkey) //TRAP(setlocale) // Used by replxx at startup -TRAP(setlogmask) TRAP(setnetent) TRAP(setnetgrent) TRAP(setprotoent) @@ -203,7 +199,6 @@ TRAP(lgammal) TRAP(nftw) TRAP(nl_langinfo) TRAP(putc_unlocked) -TRAP(rand) /** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern * implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams * are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external @@ -288,4 +283,14 @@ TRAP(tss_get) TRAP(tss_set) TRAP(tss_delete) +#ifndef USE_MUSL +/// These produce duplicate symbol errors when statically linking with musl. +/// Maybe we can remove them from the musl fork. +TRAP(getopt) +TRAP(putenv) +TRAP(setlogmask) +TRAP(rand) +TRAP(getmntent) +#endif + #endif From c869b0651932bc61d5040395a3a2d0689485a5a1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 03:13:57 +0000 Subject: [PATCH 0933/1722] Remove getpwuid() calls in Poco::PathImpl --- base/poco/Foundation/src/Path_UNIX.cpp | 18 +++++------------- contrib/sysroot | 2 +- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/base/poco/Foundation/src/Path_UNIX.cpp b/base/poco/Foundation/src/Path_UNIX.cpp index 957a62db180..fb2ed71622f 100644 --- a/base/poco/Foundation/src/Path_UNIX.cpp +++ b/base/poco/Foundation/src/Path_UNIX.cpp @@ -48,25 +48,17 @@ std::string PathImpl::currentImpl() std::string PathImpl::homeImpl() { std::string path; -#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE) size_t buf_size = 1024; // Same as glibc use for getpwuid std::vector buf(buf_size); struct passwd res; struct passwd* pwd = nullptr; getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd); -#else - struct passwd* pwd = getpwuid(getuid()); -#endif if (pwd) path = pwd->pw_dir; else { -#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE) getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd); -#else - pwd = getpwuid(geteuid()); -#endif if (pwd) path = pwd->pw_dir; else @@ -82,7 +74,7 @@ std::string PathImpl::configHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Preferences/"); #else @@ -97,7 +89,7 @@ std::string PathImpl::dataHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Application Support/"); #else @@ -112,7 +104,7 @@ std::string PathImpl::cacheHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Caches/"); #else @@ -127,7 +119,7 @@ std::string PathImpl::tempHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Caches/"); #else @@ -159,7 +151,7 @@ std::string PathImpl::tempImpl() std::string PathImpl::configImpl() { std::string path; - + #if POCO_OS == POCO_OS_MAC_OS_X path = "/Library/Preferences/"; #else diff --git a/contrib/sysroot b/contrib/sysroot index 866364fea62..c2d74e21ba1 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 866364fea629aa3e519ec967836561a6b3b21885 +Subproject commit c2d74e21ba1b8a27966e344693e176f927e4eb50 From f740cf4eaa71621fb518c6d5668e8356f452a979 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Aug 2024 09:54:03 +0200 Subject: [PATCH 0934/1722] Fix data race on SampleKey --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a6ef0063069..6efd3a5c97f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -369,7 +369,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( /// If sample and final are used together no need to calculate sampling expression twice. /// The first time it was calculated for final, because sample key is a part of the PK. /// So, assume that we already have calculated column. - ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); + ASTPtr sampling_key_ast; if (final) { @@ -377,6 +377,12 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( /// We do spoil available_real_columns here, but it is not used later. available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type)); } + else + { + sampling_key_ast = metadata_snapshot->getSamplingKeyAST()->clone(); + } + + chassert(sampling_key_ast != nullptr); if (has_lower_limit) { From 6170a8663fc85b95cda4a7617975b06cc6c007f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Aug 2024 12:31:13 +0000 Subject: [PATCH 0935/1722] Bump usearch to 2.13.2 --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- .../mergetree-family/annindexes.md | 2 + .../MergeTreeIndexVectorSimilarity.cpp | 121 +++++++++++------- .../MergeTreeIndexVectorSimilarity.h | 26 ++-- ...r_search_index_creation_negative.reference | 2 - ..._vector_search_index_creation_negative.sql | 6 - 7 files changed, 90 insertions(+), 71 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index 18d17686124..91a76d1ac51 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit 18d17686124ddebd9fe55eee56b2e0273a613d4b +Subproject commit 91a76d1ac519b3b9dc8957734a3dabd985f00c26 diff --git a/contrib/usearch b/contrib/usearch index e6c81f78c64..e21a5778a0d 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit e6c81f78c64c0d8119f854691a06e60660638a25 +Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4 diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index e73d6f07a32..097b0f5850a 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -59,6 +59,8 @@ Parameters: - `ef_construction`: (optional, default: 128) - `ef_search`: (optional, default: 64) +Value 0 for parameters `m`, `ef_construction`, and `ef_search` refers to the default value. + Example: ```sql diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 346f69140bb..fbbc66bd8db 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -2,9 +2,6 @@ #if USE_USEARCH -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wpass-failed" - #include #include #include @@ -46,15 +43,15 @@ namespace { /// The only indexing method currently supported by USearch -std::set methods = {"hnsw"}; +const std::set methods = {"hnsw"}; /// Maps from user-facing name to internal name -std::unordered_map distanceFunctionToMetricKind = { +const std::unordered_map distanceFunctionToMetricKind = { {"L2Distance", unum::usearch::metric_kind_t::l2sq_k}, {"cosineDistance", unum::usearch::metric_kind_t::cos_k}}; /// Maps from user-facing name to internal name -std::unordered_map quantizationToScalarKind = { +const std::unordered_map quantizationToScalarKind = { {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; @@ -96,13 +93,18 @@ USearchIndexWithSerialization::USearchIndexWithSerialization( unum::usearch::scalar_kind_t scalar_kind, UsearchHnswParams usearch_hnsw_params) { - unum::usearch::metric_punned_t metric(dimensions, metric_kind, scalar_kind); + USearchIndex::metric_t metric(dimensions, metric_kind, scalar_kind); unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search); config.enable_key_lookups = false; /// we don't do row-to-vector lookups - USearchIndex usearch_index = USearchIndex::make(metric, config); - swap(usearch_index); + if (auto error = config.validate(); error) /// already called in vectorSimilarityIndexValidator, call again because usearch may change the config in-place + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release())); + + if (auto result = USearchIndex::make(metric, config); !result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not create vector similarity index. Error: {}", String(result.error.release())); + else + swap(result.index); } void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const @@ -113,9 +115,8 @@ void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const return true; }; - auto result = Base::save_to_stream(callback); - if (result.error) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index, error: " + String(result.error.release())); + if (auto result = Base::save_to_stream(callback); !result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index. Error: {}", String(result.error.release())); } void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) @@ -126,26 +127,43 @@ void USearchIndexWithSerialization::deserialize(ReadBuffer & istr) return true; }; - auto result = Base::load_from_stream(callback); - if (result.error) + if (auto result = Base::load_from_stream(callback); !result) /// See the comment in MergeTreeIndexGranuleVectorSimilarity::deserializeBinary why we throw here - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index, error: " + String(result.error.release()) + " Please drop the index and create it again."); + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index. Please drop the index and create it again. Error: {}", String(result.error.release())); + + if (!try_reserve(limits())) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); } USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStatistics() const { + USearchIndex::stats_t global_stats = Base::stats(); + Statistics statistics = { .max_level = max_level(), .connectivity = connectivity(), - .size = size(), /// number of vectors - .capacity = capacity(), /// number of vectors reserved - .memory_usage = memory_usage(), /// in bytes, the value is not exact + .size = size(), + .capacity = capacity(), + .memory_usage = memory_usage(), .bytes_per_vector = bytes_per_vector(), .scalar_words = scalar_words(), - .statistics = stats()}; + .nodes = global_stats.nodes, + .edges = global_stats.edges, + .max_edges = global_stats.max_edges, + .level_stats = {}}; + + for (size_t i = 0; i < statistics.max_level; ++i) + statistics.level_stats.push_back(Base::stats(i)); + return statistics; } +String USearchIndexWithSerialization::Statistics::toString() const +{ + return fmt::format("max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}, bytes_per_vector = {}, scalar_words = {}, nodes = {}, edges = {}, max_edges = {}", + max_level, connectivity, size, capacity, ReadableSize(memory_usage), bytes_per_vector, scalar_words, nodes, edges, max_edges); + +} MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity( const String & index_name_, const Block & index_sample_block_, @@ -186,8 +204,7 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) index->serialize(ostr); auto statistics = index->getStatistics(); - LOG_TRACE(logger, "Wrote vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", - statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage)); + LOG_TRACE(logger, "Wrote vector similarity index: {}", statistics.toString()); } void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) @@ -209,8 +226,7 @@ void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, index->deserialize(istr); auto statistics = index->getStatistics(); - LOG_TRACE(logger, "Loaded vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}", - statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage)); + LOG_TRACE(logger, "Loaded vector similarity index: {}", statistics.toString()); } MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilarity( @@ -290,19 +306,24 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ if (!index) index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); + /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp + if (index->size() + num_rows > std::numeric_limits::max()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index in column {} would exceed 4 billion entries", index_column_name); + /// Reserving space is mandatory - if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) + if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); for (size_t row = 0; row < num_rows; ++row) { - auto rc = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); - if (!rc) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release())); - - ProfileEvents::increment(ProfileEvents::USearchAddCount); - ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members); - ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, rc.computed_distances); + if (auto result = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release())); + else + { + ProfileEvents::increment(ProfileEvents::USearchAddCount); + ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members); + ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances); + } } } else @@ -356,17 +377,16 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); - auto result = index->search(reference_vector.data(), limit); - if (result.error) - throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release())); + auto search_result = index->search(reference_vector.data(), limit); + if (!search_result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index. Error: {}", String(search_result.error.release())); ProfileEvents::increment(ProfileEvents::USearchSearchCount); - ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members); - ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, result.computed_distances); + ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, search_result.visited_members); + ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, search_result.computed_distances); - std::vector neighbors(result.size()); /// indexes of dots which were closest to the reference vector - std::vector distances(result.size()); - result.dump_to(neighbors.data(), distances.data()); + std::vector neighbors(search_result.size()); /// indexes of vectors which were closest to the reference vector + search_result.dump_to(neighbors.data()); std::vector granules; granules.reserve(neighbors.size()); @@ -414,14 +434,13 @@ MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition( MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { - const bool has_six_args = (index.arguments.size() == 6); - + /// Default parameters: unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet()); - - /// use defaults for the other parameters unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k; UsearchHnswParams usearch_hnsw_params; + /// Optional parameters: + const bool has_six_args = (index.arguments.size() == 6); if (has_six_args) { scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet()); @@ -466,12 +485,16 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta { if (!quantizationToScalarKind.contains(index.arguments[2].safeGet())) throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind)); - if (index.arguments[3].safeGet() < 2) - throw Exception(ErrorCodes::INCORRECT_DATA, "Fourth argument (M) of vector similarity index must be > 1"); - if (index.arguments[4].safeGet() < 1) - throw Exception(ErrorCodes::INCORRECT_DATA, "Fifth argument (ef_construction) of vector similarity index must be > 0"); - if (index.arguments[5].safeGet() < 1) - throw Exception(ErrorCodes::INCORRECT_DATA, "Sixth argument (ef_search) of vector similarity index must be > 0"); + + /// Call Usearche's own parameter validation method for HNSW-specific parameters + UInt64 m = index.arguments[3].safeGet(); + UInt64 ef_construction = index.arguments[4].safeGet(); + UInt64 ef_search = index.arguments[5].safeGet(); + + unum::usearch::index_dense_config_t config(m, ef_construction, ef_search); + + if (auto error = config.validate(); error) + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release())); } /// Check that the index is created on a single column diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h index f7098c1626c..c4c03254d2d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h @@ -4,12 +4,9 @@ #if USE_USEARCH -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wpass-failed" -# include -# include -# include -#pragma clang diagnostic pop +#include +#include +#include namespace DB { @@ -21,7 +18,7 @@ struct UsearchHnswParams size_t ef_search = unum::usearch::default_expansion_search(); }; -using USearchIndex = unum::usearch::index_dense_gt; +using USearchIndex = unum::usearch::index_dense_t; class USearchIndexWithSerialization : public USearchIndex { @@ -41,13 +38,18 @@ public: { size_t max_level; size_t connectivity; - size_t size; - size_t capacity; - size_t memory_usage; - /// advanced stats: + size_t size; /// number of indexed vectors + size_t capacity; /// reserved number of indexed vectors + size_t memory_usage; /// byte size (not exact) size_t bytes_per_vector; size_t scalar_words; - Base::stats_t statistics; + size_t nodes; + size_t edges; + size_t max_edges; + + std::vector level_stats; /// for debugging, excluded from getStatistics() + + String toString() const; }; Statistics getStatistics() const; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference index b6d034208d0..f18daa6e02e 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference @@ -3,8 +3,6 @@ Two or six index arguments 2nd argument (distance function) must be String and L2Distance or cosineDistance 3nd argument (quantization), if given, must be String and f32, f16, ... 4nd argument (M), if given, must be UInt64 and > 1 -5nd argument (ef_construction), if given, must be UInt64 and > 0 -6nd argument (ef_search), if given, must be UInt64 and > 0 Must be created on single column Must be created on Array(Float32) columns Rejects INSERTs of Arrays with different sizes diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index 7c2ddfe81fc..de9d37e1000 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -27,12 +27,6 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar SELECT '4nd argument (M), if given, must be UInt64 and > 1'; CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -SELECT '5nd argument (ef_construction), if given, must be UInt64 and > 0'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 'invalid', 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 0, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -SELECT '6nd argument (ef_search), if given, must be UInt64 and > 0'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 'invalid')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 0)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } SELECT 'Must be created on single column'; CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } From ca5fd57db7d7718a275b31200447a986019e3349 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 14 Aug 2024 09:35:54 +0000 Subject: [PATCH 0936/1722] Fix builds --- src/Common/StringHashForHeterogeneousLookup.h | 5 +++++ .../Serializations/tests/gtest_object_serialization.cpp | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/StringHashForHeterogeneousLookup.h b/src/Common/StringHashForHeterogeneousLookup.h index 0983fd460d6..56d8ccf0009 100644 --- a/src/Common/StringHashForHeterogeneousLookup.h +++ b/src/Common/StringHashForHeterogeneousLookup.h @@ -20,6 +20,11 @@ struct StringHashForHeterogeneousLookup { return hash_type()(str); } + + auto operator()(const char * data) const + { + return hash_type()(data); + } }; } diff --git a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp index 8a0c712fa86..f104b75af9b 100644 --- a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp +++ b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp @@ -17,7 +17,7 @@ TEST(ObjectSerialization, FieldBinarySerialization) ReadBufferFromString istr(ostr.str()); Field object2; serialization->deserializeBinary(object2, istr, FormatSettings()); - ASSERT_EQ(object1, object2.get()); + ASSERT_EQ(object1, object2.safeGet()); } From 8dec996686449e5a541157fdfd5ffb65b2208998 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Aug 2024 06:35:44 +0000 Subject: [PATCH 0937/1722] Fix non-deterministic result order in test_storage_mysql.test_mysql_distributed --- tests/integration/test_storage_mysql/test.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 5948954ff5f..c724c5bb498 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -445,7 +445,7 @@ def test_mysql_distributed(started_cluster): query = "SELECT * FROM (" for i in range(3): query += "SELECT name FROM test_replicas UNION DISTINCT " - query += "SELECT name FROM test_replicas)" + query += "SELECT name FROM test_replicas) ORDER BY name" result = node2.query(query) assert result == "host2\nhost3\nhost4\n" @@ -827,6 +827,9 @@ def test_settings(started_cluster): f"with settings: connect_timeout={connect_timeout}, read_write_timeout={rw_timeout}" ) + node1.query("DROP DATABASE IF EXISTS m") + node1.query("DROP DATABASE IF EXISTS mm") + rw_timeout = 40123001 connect_timeout = 40123002 node1.query( @@ -855,6 +858,9 @@ def test_settings(started_cluster): f"with settings: connect_timeout={connect_timeout}, read_write_timeout={rw_timeout}" ) + node1.query("DROP DATABASE m") + node1.query("DROP DATABASE mm") + drop_mysql_table(conn, table_name) conn.close() @@ -930,6 +936,9 @@ def test_joins(started_cluster): conn.commit() + node1.query("DROP TABLE IF EXISTS test_joins_table_users") + node1.query("DROP TABLE IF EXISTS test_joins_table_tickets") + node1.query( """ CREATE TABLE test_joins_table_users @@ -964,6 +973,9 @@ def test_joins(started_cluster): """ ) == "281607\tFeedback\t2024-06-25 12:09:41\tuser@example.com\n" + node1.query("DROP TABLE test_joins_table_users") + node1.query("DROP TABLE test_joins_table_tickets") + if __name__ == "__main__": with contextmanager(started_cluster)() as cluster: From 9754620c077b85f5b2cb837416a73f3c831842e2 Mon Sep 17 00:00:00 2001 From: sunny <923976094@qq.com> Date: Wed, 14 Aug 2024 17:51:45 +0800 Subject: [PATCH 0938/1722] Update 02352_lightweight_delete_in_partition.sql update test From 9fe31773bdeeeada849a60822a7409ee1aa8782f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 14 Aug 2024 09:52:12 +0000 Subject: [PATCH 0939/1722] Fix part name in 00961_check_table --- tests/queries/0_stateless/00961_check_table.reference | 2 +- tests/queries/0_stateless/00961_check_table.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00961_check_table.reference b/tests/queries/0_stateless/00961_check_table.reference index a0a054898b9..686285bb6aa 100644 --- a/tests/queries/0_stateless/00961_check_table.reference +++ b/tests/queries/0_stateless/00961_check_table.reference @@ -14,4 +14,4 @@ ======== 201902_4_5_1 1 ======== -201801_1_1_0 1 +201801_1_1_2 1 diff --git a/tests/queries/0_stateless/00961_check_table.sql b/tests/queries/0_stateless/00961_check_table.sql index a6abe8103d5..fc3c5435670 100644 --- a/tests/queries/0_stateless/00961_check_table.sql +++ b/tests/queries/0_stateless/00961_check_table.sql @@ -39,6 +39,6 @@ CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; SELECT '========'; -CHECK TABLE mt_table PART '201801_1_1_0'; +CHECK TABLE mt_table PART '201801_1_1_2'; DROP TABLE IF EXISTS mt_table; From 8d7319ccab75d8f5a401bdd63697f7bd6508d27f Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 14 Aug 2024 10:18:00 +0000 Subject: [PATCH 0940/1722] fix wrong format of SYSTEM SYNC REPLICA query Signed-off-by: Duc Canh Le --- src/Parsers/ASTSystemQuery.cpp | 45 ++++++++++--------- ...03205_system_sync_replica_format.reference | 1 + .../03205_system_sync_replica_format.sql | 1 + 3 files changed, 25 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/03205_system_sync_replica_format.reference create mode 100644 tests/queries/0_stateless/03205_system_sync_replica_format.sql diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a730ea0ba3d..7780544d5c2 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -198,6 +198,29 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s print_database_table(); } + if (sync_replica_mode != SyncReplicaMode::DEFAULT) + { + settings.ostr << ' '; + print_keyword(magic_enum::enum_name(sync_replica_mode)); + + // If the mode is LIGHTWEIGHT and specific source replicas are specified + if (sync_replica_mode == SyncReplicaMode::LIGHTWEIGHT && !src_replicas.empty()) + { + settings.ostr << ' '; + print_keyword("FROM"); + settings.ostr << ' '; + + bool first = true; + for (const auto & src : src_replicas) + { + if (!first) + settings.ostr << ", "; + first = false; + settings.ostr << quoteString(src); + } + } + } + if (query_settings) { settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SETTINGS " << (settings.hilite ? hilite_none : ""); @@ -233,28 +256,6 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s print_identifier(disk); } - if (sync_replica_mode != SyncReplicaMode::DEFAULT) - { - settings.ostr << ' '; - print_keyword(magic_enum::enum_name(sync_replica_mode)); - - // If the mode is LIGHTWEIGHT and specific source replicas are specified - if (sync_replica_mode == SyncReplicaMode::LIGHTWEIGHT && !src_replicas.empty()) - { - settings.ostr << ' '; - print_keyword("FROM"); - settings.ostr << ' '; - - bool first = true; - for (const auto & src : src_replicas) - { - if (!first) - settings.ostr << ", "; - first = false; - settings.ostr << quoteString(src); - } - } - } break; } case Type::SYNC_DATABASE_REPLICA: diff --git a/tests/queries/0_stateless/03205_system_sync_replica_format.reference b/tests/queries/0_stateless/03205_system_sync_replica_format.reference new file mode 100644 index 00000000000..aad51dd90b0 --- /dev/null +++ b/tests/queries/0_stateless/03205_system_sync_replica_format.reference @@ -0,0 +1 @@ +SYSTEM SYNC REPLICA db.`table` LIGHTWEIGHT diff --git a/tests/queries/0_stateless/03205_system_sync_replica_format.sql b/tests/queries/0_stateless/03205_system_sync_replica_format.sql new file mode 100644 index 00000000000..329bce80afc --- /dev/null +++ b/tests/queries/0_stateless/03205_system_sync_replica_format.sql @@ -0,0 +1 @@ +SELECT formatQuery('SYSTEM SYNC REPLICA db.table LIGHTWEIGHT'); From 0e0272b2ffdf4286c5ef9766c90b88e096469e92 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 14 Aug 2024 10:21:23 +0000 Subject: [PATCH 0941/1722] Better check for overflow + limit min_marks_for_concurrent_read --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 18 ++++++++++++++++-- .../MergeTree/MergeTreeIndexGranularity.cpp | 15 +++++++++------ ...plicas_min_marks_to_read_overflow.reference | 11 +++++++++++ ...lel_replicas_min_marks_to_read_overflow.sql | 10 +++++++++- 4 files changed, 45 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 901d7c61167..b5b46ef9f41 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -350,7 +350,14 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( /// We have a special logic for local replica. It has to read less data, because in some cases it should /// merge states of aggregate functions or do some other important stuff other than reading from Disk. - const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + if (pool_settings.min_marks_for_concurrent_read > std::numeric_limits::max()) + { + /// limit min marks to read in case it's big, happened in test since due to settings randomzation + pool_settings.min_marks_for_concurrent_read = std::numeric_limits::max(); + multiplier = 1.0f; + } + if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) pool_settings.min_marks_for_concurrent_read = static_cast(result); else @@ -519,7 +526,14 @@ Pipe ReadFromMergeTree::readInOrder( .number_of_current_replica = client_info.number_of_current_replica, }; - const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + if (pool_settings.min_marks_for_concurrent_read > std::numeric_limits::max()) + { + /// limit min marks to read in case it's big, happened in test since due to settings randomzation + pool_settings.min_marks_for_concurrent_read = std::numeric_limits::max(); + multiplier = 1.0f; + } + if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) pool_settings.min_marks_for_concurrent_read = static_cast(result); else diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 2f9a4a47b11..2b924284857 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -105,13 +105,16 @@ size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t num /// See comment in IMergeTreeSelectAlgorithm. if (min_marks_to_read) { - // check that ... - bool overflow = ((1ULL << 63) & min_marks_to_read); // further multiplication by 2 will not overflow - if (!overflow) - overflow = (std::numeric_limits::max() - from_mark) < 2 * min_marks_to_read; // further addition will not overflow + // check overflow + size_t min_marks_to_read_2 = 0; + bool overflow = common::mulOverflow(min_marks_to_read, 2, min_marks_to_read_2); - if (!overflow && from_mark + 2 * min_marks_to_read <= to_mark) - to_mark = from_mark + min_marks_to_read; + size_t to_mark_overwrite = 0; + if (!overflow) + overflow = common::addOverflow(from_mark, min_marks_to_read_2, to_mark_overwrite); + + if (!overflow && to_mark_overwrite < to_mark) + to_mark = to_mark_overwrite; } return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows; diff --git a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference index 7fafd4d13ea..b6c452ba328 100644 --- a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference +++ b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference @@ -1,3 +1,14 @@ +1006 +1007 +1008 +1009 +101 +1010 +1011 +1012 +1013 +1014 +--- 100 100 101 101 102 102 diff --git a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql index 112373e5db2..6f486f8f0fe 100644 --- a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql +++ b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql @@ -6,10 +6,18 @@ INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(10_000); SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SELECT v +FROM test__fuzz_22 +ORDER BY v +LIMIT 10, 10 +SETTINGS merge_tree_min_rows_for_concurrent_read = 9223372036854775806; + +SELECT '---'; + SELECT k, v FROM test__fuzz_22 ORDER BY k LIMIT 100, 10 -SETTINGS merge_tree_min_rows_for_concurrent_read = 9223372036854775806; +SETTINGS optimize_read_in_order=1, merge_tree_min_rows_for_concurrent_read = 9223372036854775806; DROP TABLE test__fuzz_22 SYNC; From 4f77893c79d3c2c8b0304837bd45033301e6d49a Mon Sep 17 00:00:00 2001 From: sunny <923976094@qq.com> Date: Wed, 14 Aug 2024 18:24:03 +0800 Subject: [PATCH 0942/1722] Update ParserDeleteQuery.cpp fix: delete space --- src/Parsers/ParserDeleteQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserDeleteQuery.cpp b/src/Parsers/ParserDeleteQuery.cpp index 3503fbe9ad2..d7cdcd920ad 100644 --- a/src/Parsers/ParserDeleteQuery.cpp +++ b/src/Parsers/ParserDeleteQuery.cpp @@ -62,7 +62,7 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else return false; - + if (query->partition) query->children.push_back(query->partition); From 56d6ef5c4a015f5851923f2c420538456564e790 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 14 Aug 2024 10:53:07 +0000 Subject: [PATCH 0943/1722] Fix 02995_index_10 timeout --- tests/queries/0_stateless/02995_index_10.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02995_index_10.sh b/tests/queries/0_stateless/02995_index_10.sh index 813cc49cbd8..e7e7d3c3b42 100755 --- a/tests/queries/0_stateless/02995_index_10.sh +++ b/tests/queries/0_stateless/02995_index_10.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS test; CREATE TABLE test (a String, b String, c String) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 11; @@ -37,8 +37,9 @@ WHERE a >= (round(pow(sipHash64(1, try), 1 / (3 + sipHash64(2, try) % 8))) AS a1 AND b <= (b1 + round(pow(sipHash64(7, try), 1 / (3 + sipHash64(8, try) % 8))))::String AND c >= (round(pow(sipHash64(9, try), 1 / (3 + sipHash64(10, try) % 8))) AS c1)::String AND c <= (c1 + round(pow(sipHash64(11, try), 1 / (3 + sipHash64(12, try) % 8))))::String -HAVING count() > 0; -" +HAVING count() > 0 +SETTINGS trace_profile_events=0 -- test is too slow with profiling +;" done | ${CLICKHOUSE_CLIENT} -${CLICKHOUSE_CLIENT} "DROP TABLE test" +${CLICKHOUSE_CLIENT} -q "DROP TABLE test" From 476819a56fd71e445da5aa91f34e10abb8ded327 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 14 Aug 2024 11:35:37 +0000 Subject: [PATCH 0944/1722] Changed test to work in parallel --- .../integration/test_storage_iceberg/test.py | 130 +++++++++++++----- 1 file changed, 97 insertions(+), 33 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b66133f5562..176c7e209bd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -165,9 +165,8 @@ def generate_data(spark, start, end): return df -def create_iceberg_table( +def get_creation_expression( storage_type, - node, table_name, cluster, format="Parquet", @@ -182,38 +181,52 @@ def create_iceberg_table( print(bucket) if table_function: return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" elif storage_type == "azure": if table_function: return f""" icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) """ - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" elif storage_type == "local": if table_function: return f""" icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) """ - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" else: raise Exception(f"Unknown iceberg storage type: {storage_type}") +def get_uuid_str(): + return str(uuid.uuid4()).replace("-", "_") + + +def create_iceberg_table( + storage_type, + node, + table_name, + cluster, + format="Parquet", + **kwargs, +): + node.query( + get_creation_expression(storage_type, table_name, cluster, format, **kwargs) + ) + + def create_initial_data_file( cluster, node, query, table_name, compression_method="none" ): @@ -258,7 +271,14 @@ def default_upload_directory( def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_single_iceberg_file_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_single_iceberg_file_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) @@ -281,7 +301,14 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_partition_by_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_partition_by_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -311,7 +338,14 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_iceberg_files_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_multiple_iceberg_files_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -364,7 +398,9 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_types_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_types_" + format_version + "_" + storage_type + "_" + get_uuid_str() + ) data = [ ( @@ -404,8 +440,8 @@ def test_types(started_cluster, format_version, storage_type): == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - table_function_expr = create_iceberg_table( - storage_type, instance, TABLE_NAME, started_cluster, table_function=True + table_function_expr = get_creation_expression( + storage_type, TABLE_NAME, started_cluster, table_function=True ) assert ( instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() @@ -430,7 +466,14 @@ def test_delete_files(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_delete_files_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_delete_files_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -496,7 +539,14 @@ def test_evolved_schema(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_evolved_schema_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_evolved_schema_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -542,7 +592,7 @@ def test_row_based_deletes(started_cluster, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_row_based_deletes_" + storage_type + TABLE_NAME = "test_row_based_deletes_" + storage_type + "_" + get_uuid_str() spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -587,6 +637,8 @@ def test_schema_inference(started_cluster, format_version, storage_type): + format_version + "_" + storage_type + + "_" + + get_uuid_str() ) # Types time, timestamptz, fixed are not supported in Spark. @@ -645,7 +697,14 @@ def test_schema_inference(started_cluster, format_version, storage_type): def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_metadata_selection_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_metadata_selection_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -674,7 +733,12 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage instance = started_cluster.instances["node1"] spark = started_cluster.spark_session TABLE_NAME = ( - "test_metadata_selection_with_uuid_" + format_version + "_" + storage_type + "test_metadata_selection_with_uuid_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() ) spark.sql( @@ -689,7 +753,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage for i in range(50): os.rename( f"/iceberg_data/default/{TABLE_NAME}/metadata/v{i + 1}.metadata.json", - f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", + f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{get_uuid_str()}.metadata.json", ) files = default_upload_directory( @@ -707,7 +771,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage def test_restart_broken_s3(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_restart_broken_table_function_s3" + TABLE_NAME = "test_restart_broken_table_function_s3" + "_" + get_uuid_str() minio_client = started_cluster.minio_client bucket = "broken2" From 962bf1d821a498aaeb6f16e5d4205272cfd00001 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 14 Aug 2024 13:37:14 +0200 Subject: [PATCH 0945/1722] CI: Fix for critical bug fix regex --- tests/ci/changelog.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 39e426945d3..b7f73f22016 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -115,7 +115,6 @@ def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]: # pylint: enable=protected-access if repo_name not in repos: repos[repo_name] = pr.base.repo - in_changelog = False merge_commit = pr.merge_commit_sha if merge_commit is None: logging.warning("PR %s does not have merge-commit, skipping", pr.number) @@ -291,7 +290,7 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri # Normalize bug fixes if ( re.match( - r"(?i)bug\Wfix", + r".*(?i)bug\Wfix", category, ) # Map "Critical Bug Fix" to "Bug fix" category for changelog From fc23f1c1ff5e114298f6b169fbeef80082f776d6 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 14 Aug 2024 12:20:33 +0000 Subject: [PATCH 0946/1722] Fix tests --- src/DataTypes/DataTypeObject.cpp | 9 ++++----- src/Formats/JSONExtractTree.cpp | 4 ++-- src/Functions/JSONPaths.cpp | 8 ++++++++ src/IO/parseDateTimeBestEffort.cpp | 4 ++-- .../0_stateless/01825_new_type_json_10.sql | 1 + .../01825_new_type_json_12.reference | 2 +- .../0_stateless/01825_new_type_json_18.sql | 2 ++ .../0_stateless/01825_new_type_json_9.sql | 1 + .../01825_new_type_json_in_array.sql | 2 ++ .../01825_new_type_json_insert_select.sql | 8 +++++++- .../03205_json_cast_from_string.reference | 4 ++-- .../03205_json_cast_from_string.sql | 6 +++--- ...3206_json_parsing_and_formatting.reference | 20 +++++++++---------- .../03206_json_parsing_and_formatting.sh | 4 ++-- ...9_json_type_horizontal_merges.reference.j2 | 4 ++-- ...209_json_type_vertical_merges.reference.j2 | 4 ++-- 16 files changed, 51 insertions(+), 32 deletions(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index b5d1f429001..d6395155397 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -224,7 +224,7 @@ void replaceJSONTypeNameIfNeeded(String & type_name, size_t max_dynamic_paths, s fmt::format( "JSON(max_dynamic_paths={}, max_dynamic_types={})", max_dynamic_paths / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, - std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))); + max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR)); pos = type_name.find("JSON", pos + 4); } } @@ -458,10 +458,9 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: auto * literal = function->arguments->children[1]->as(); /// Is 1000000 a good maximum for max paths? - size_t min_value = identifier_name == "max_dynamic_types" ? 1 : 0; - size_t max_value = identifier_name == "max_dynamic_types" ? 255 : 1000000; - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.safeGet() < min_value || literal->value.safeGet() > max_value) - throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'{}' parameter for {} type should be a positive integer between {} and {}. Got {}", identifier_name, magic_enum::enum_name(schema_format), min_value, max_value, function->arguments->children[1]->formatForErrorMessage()); + size_t max_value = identifier_name == "max_dynamic_types" ? ColumnDynamic::MAX_DYNAMIC_TYPES_LIMIT : 1000000; + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.safeGet() > max_value) + throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'{}' parameter for {} type should be a positive integer between 0 and {}. Got {}", identifier_name, magic_enum::enum_name(schema_format), max_value, function->arguments->children[1]->formatForErrorMessage()); if (identifier_name == "max_dynamic_types") max_dynamic_types = literal->value.safeGet(); diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 6a568d945fb..cac4dfbb077 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1582,12 +1582,12 @@ public: const std::unordered_set & paths_to_skip_, const std::vector & path_regexps_to_skip_, size_t max_dynamic_paths_, - size_t max_dynamic_types) + size_t max_dynamic_types_) : typed_path_nodes(std::move(typed_path_nodes_)) , paths_to_skip(paths_to_skip_) , dynamic_node(std::make_unique>( max_dynamic_paths_ / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_PATHS_REDUCE_FACTOR, - std::max(max_dynamic_types / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR, 1lu))) + max_dynamic_types_ / DataTypeObject::NESTED_OBJECT_MAX_DYNAMIC_TYPES_REDUCE_FACTOR)) , dynamic_serialization(std::make_shared()) { sorted_paths_to_skip.assign(paths_to_skip.begin(), paths_to_skip.end()); diff --git a/src/Functions/JSONPaths.cpp b/src/Functions/JSONPaths.cpp index 35613e40aac..dfb0386e370 100644 --- a/src/Functions/JSONPaths.cpp +++ b/src/Functions/JSONPaths.cpp @@ -351,6 +351,14 @@ private: auto global_discr = variant_column.globalDiscriminatorAt(i); /// We don't output path with NULL values. It should be checked before calling getDynamicValueType. chassert(global_discr != ColumnVariant::NULL_DISCRIMINATOR); + if (global_discr == dynamic_column->getSharedVariantDiscriminator()) + { + auto value = dynamic_column->getSharedVariant().getDataAt(variant_column.offsetAt(i)); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + return type->getName(); + } + return variant_info.variant_names[global_discr]; } diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index e046e837689..a2afa4ce654 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -560,8 +560,8 @@ ReturnType parseDateTimeBestEffortImpl( else return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected word"); - while (!in.eof() && isAlphaASCII(*in.position())) - ++in.position(); +// while (!in.eof() && isAlphaASCII(*in.position())) +// ++in.position(); /// For RFC 2822 if (has_day_of_week) diff --git a/tests/queries/0_stateless/01825_new_type_json_10.sql b/tests/queries/0_stateless/01825_new_type_json_10.sql index a313adb4757..f586cc4477b 100644 --- a/tests/queries/0_stateless/01825_new_type_json_10.sql +++ b/tests/queries/0_stateless/01825_new_type_json_10.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS t_json_10; CREATE TABLE t_json_10 (o JSON) ENGINE = Memory; INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 1, "c": [{"d": 10, "e": [31]}, {"d": 20, "e": [63, 127]}]}} {"a": {"b": 2, "c": []}} + INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 3, "c": [{"f": 20, "e": [32]}, {"f": 30, "e": [64, 128]}]}} {"a": {"b": 4, "c": []}} SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(o)) as path FROM t_json_10 order by path; diff --git a/tests/queries/0_stateless/01825_new_type_json_12.reference b/tests/queries/0_stateless/01825_new_type_json_12.reference index 36f014f0178..10770498c4a 100644 --- a/tests/queries/0_stateless/01825_new_type_json_12.reference +++ b/tests/queries/0_stateless/01825_new_type_json_12.reference @@ -9,5 +9,5 @@ ('key_6','String') ('key_7','Float64') ('key_7','Int64') -{"obj":{"id":"1","key_0":[{"key_1":[{"key_3":[{"key_4":"1048576","key_5":0.0001048576,"key_6":25.5,"key_7":"1025"},{"key_6":"","key_7":"2"}]}]},{},{"key_1":[{"key_3":[{"key_5":-1,"key_6":"aqbjfiruu","key_7":-922337203685477600},{"key_4":"","key_6":"","key_7":"65537"}]},{"key_3":[{"key_4":"ghdqyeiom","key_5":1048575,"key_7":21474836.48}]}]}]}} +{"obj":{"id":"1","key_0":[{"key_1":[{"key_3":[{"key_4":"1048576","key_5":0.0001048576,"key_6":25.5,"key_7":"1025"},{"key_6":"","key_7":"2"}]}]},{},{"key_1":[{"key_3":[{"key_5":-1,"key_6":"aqbjfiruu","key_7":-922337203685477600},{"key_4":"","key_6":"","key_7":65537}]},{"key_3":[{"key_4":"ghdqyeiom","key_5":1048575,"key_7":21474836.48}]}]}]}} [[[1048576,NULL]],[],[[NULL,''],['ghdqyeiom']]] [[[0.0001048576,NULL]],[],[[-1,NULL],[1048575]]] [[[25.5,'']],[],[['aqbjfiruu',''],[NULL]]] [[[1025,2]],[],[[-922337203685477600,65537],[21474836.48]]] diff --git a/tests/queries/0_stateless/01825_new_type_json_18.sql b/tests/queries/0_stateless/01825_new_type_json_18.sql index 6066ff66fd7..469d37776ea 100644 --- a/tests/queries/0_stateless/01825_new_type_json_18.sql +++ b/tests/queries/0_stateless/01825_new_type_json_18.sql @@ -8,9 +8,11 @@ CREATE TABLE t_json_2(id UInt64, data JSON) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t_json_2 FORMAT JSONEachRow {"id": 1, "data" : {"k1": 1}}; + SELECT id, data, JSONAllPathsWithTypes(data) FROM t_json_2 ORDER BY id; TRUNCATE TABLE t_json_2; INSERT INTO t_json_2 FORMAT JSONEachRow {"id": 1, "data" : {"k1": [1, 2]}}; + SELECT id, data, JSONAllPathsWithTypes(data) FROM t_json_2 ORDER BY id; diff --git a/tests/queries/0_stateless/01825_new_type_json_9.sql b/tests/queries/0_stateless/01825_new_type_json_9.sql index 626016a0a81..161a1d37f2e 100644 --- a/tests/queries/0_stateless/01825_new_type_json_9.sql +++ b/tests/queries/0_stateless/01825_new_type_json_9.sql @@ -7,6 +7,7 @@ SET allow_experimental_json_type = 1; CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id; INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}}; + INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}}; OPTIMIZE TABLE t_json FINAL; diff --git a/tests/queries/0_stateless/01825_new_type_json_in_array.sql b/tests/queries/0_stateless/01825_new_type_json_in_array.sql index fbf47e8607d..42ab1f64681 100644 --- a/tests/queries/0_stateless/01825_new_type_json_in_array.sql +++ b/tests/queries/0_stateless/01825_new_type_json_in_array.sql @@ -7,6 +7,7 @@ DROP TABLE IF EXISTS t_json_array; CREATE TABLE t_json_array (id UInt32, arr Array(JSON)) ENGINE = MergeTree ORDER BY id; INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": 1, "k2": {"k3": 2, "k4": 3}}, {"k1": 2, "k2": {"k5": "foo"}}]} + INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": 3, "k2": {"k3": 4, "k4": 5}}]} @@ -18,6 +19,7 @@ SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(arr))) as path FROM t_ TRUNCATE TABLE t_json_array; INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} + INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow; diff --git a/tests/queries/0_stateless/01825_new_type_json_insert_select.sql b/tests/queries/0_stateless/01825_new_type_json_insert_select.sql index 6219cd2ede2..aff920c06ee 100644 --- a/tests/queries/0_stateless/01825_new_type_json_insert_select.sql +++ b/tests/queries/0_stateless/01825_new_type_json_insert_select.sql @@ -9,12 +9,14 @@ CREATE TABLE type_json_src (id UInt32, data JSON) ENGINE = MergeTree ORDER BY id CREATE TABLE type_json_dst AS type_json_src; INSERT INTO type_json_src VALUES (1, '{"k1": 1, "k2": "foo"}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src; SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; SELECT id, data FROM type_json_dst ORDER BY id; INSERT INTO type_json_src VALUES (2, '{"k1": 2, "k2": "bar"}') (3, '{"k1": 3, "k3": "aaa"}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id > 1; SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; @@ -23,10 +25,12 @@ SELECT id, data FROM type_json_dst ORDER BY id; INSERT INTO type_json_dst VALUES (4, '{"arr": [{"k11": 5, "k22": 6}, {"k11": 7, "k33": 8}]}'); INSERT INTO type_json_src VALUES (5, '{"arr": "not array"}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; TRUNCATE TABLE type_json_src; -INSERT INTO type_json_src VALUES (6, '{"arr": [{"k22": "str1"}]}') +INSERT INTO type_json_src VALUES (6, '{"arr": [{"k22": "str1"}]}'); + INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) AS path FROM type_json_dst ORDER BY path; @@ -45,6 +49,7 @@ SET max_insert_threads = 1; SET output_format_json_named_tuples_as_objects = 1; INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]}; + INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]}; INSERT INTO type_json_dst SELECT data FROM type_json_src; @@ -56,6 +61,7 @@ TRUNCATE TABLE type_json_src; TRUNCATE TABLE type_json_dst; INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]}; + INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]}; INSERT INTO type_json_dst SELECT data FROM type_json_src; diff --git a/tests/queries/0_stateless/03205_json_cast_from_string.reference b/tests/queries/0_stateless/03205_json_cast_from_string.reference index 1c4d820b3b4..b9ac477eef4 100644 --- a/tests/queries/0_stateless/03205_json_cast_from_string.reference +++ b/tests/queries/0_stateless/03205_json_cast_from_string.reference @@ -13,6 +13,6 @@ {"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64','a.b.e':'Int64'} {'a.f':'Int64','g':'Int64'} {"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64'} {'a.b.e':'Int64','a.f':'Int64','g':'Int64'} {"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} -{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'String','a.b.e':'String','a.f':'Int64','g':'Int64'} {'a.b.c.d':'String','a.b.e':'String'} {'a.f':'Int64','g':'Int64'} -{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'String','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'String'} {'a.b.e':'Int64','a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64','a.b.e':'Int64'} {'a.f':'Int64','g':'Int64'} +{"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {'a.b.c.d':'Int64'} {'a.b.e':'Int64','a.f':'Int64','g':'Int64'} {"a":{"b":{"c":{"d":"42"},"e":"43"},"f":"44"},"g":"44"} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} {} {'a.b.c.d':'Int64','a.b.e':'Int64','a.f':'Int64','g':'Int64'} diff --git a/tests/queries/0_stateless/03205_json_cast_from_string.sql b/tests/queries/0_stateless/03205_json_cast_from_string.sql index 6ab88826c86..5ceee134c51 100644 --- a/tests/queries/0_stateless/03205_json_cast_from_string.sql +++ b/tests/queries/0_stateless/03205_json_cast_from_string.sql @@ -17,6 +17,6 @@ select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 2) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 0) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); -select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 2, max_dynamic_types=1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); -select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 1, max_dynamic_types=1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); -select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 0, max_dynamic_types=1) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 2, max_dynamic_types=0) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 1, max_dynamic_types=0) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); +select materialize('{"a" : {"b" : {"c" : {"d" : 42}, "e" : 43}, "f" : 44}, "g" : 44}')::JSON(max_dynamic_paths = 0, max_dynamic_types=0) as json, JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json); diff --git a/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference b/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference index fa6fed341ba..94777ffab1f 100644 --- a/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference +++ b/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference @@ -48,16 +48,16 @@ JSON(a.b.c UInt32, max_dynamic_paths=0) {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} {'a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} JSON(a.b.c UInt32, max_dynamic_types=1) -{"a":{"b":{"c":1,"d":"[0,1]"}},"b":"2020-01-01","c":"42","d":{"e":{"f":"[\\"s1\\",\\"s2\\"]"}}} -{"a":{"b":{"c":2,"d":"[2,3]"}},"b":"[1,2,3]","d":{"e":{"g":"43"}}} -{"a":{"b":{"c":3,"d":"[4,5]"}},"e":"Hello, World!"} -{"a":{"b":{"c":4,"d":"[6,7]"}},"c":"43"} -{"a":{"b":{"c":5,"d":"[8,9]"}},"b":"[\\"b1\\",\\"b2\\"]","d":{"e":{"f":"[\\"s3\\",\\"s4\\"]","g":"44","h":"2020-02-02 10:00:00"}}} -{'a.b.c':'UInt32','a.b.d':'String','b':'String','c':'String','d.e.f':'String'} {'a.b.d':'String','b':'String','c':'String','d.e.f':'String'} {} -{'a.b.c':'UInt32','a.b.d':'String','b':'String','d.e.g':'String'} {'a.b.d':'String','b':'String','d.e.g':'String'} {} -{'a.b.c':'UInt32','a.b.d':'String','e':'String'} {'a.b.d':'String','e':'String'} {} -{'a.b.c':'UInt32','a.b.d':'String','c':'String'} {'a.b.d':'String','c':'String'} {} -{'a.b.c':'UInt32','a.b.d':'String','b':'String','d.e.f':'String','d.e.g':'String','d.e.h':'String'} {'a.b.d':'String','b':'String','d.e.f':'String','d.e.g':'String','d.e.h':'String'} {} +{"a":{"b":{"c":1,"d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} +{"a":{"b":{"c":2,"d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} +{"a":{"b":{"c":3,"d":["4","5"]}},"e":"Hello, World!"} +{"a":{"b":{"c":4,"d":["6","7"]}},"c":"43"} +{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.d':'Array(Nullable(Int64))','e':'String'} {} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} Test small max_read_buffer_size {"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} diff --git a/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh b/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh index 44bef4f0e95..7e53e4388ec 100755 --- a/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh +++ b/tests/queries/0_stateless/03206_json_parsing_and_formatting.sh @@ -31,8 +31,8 @@ $CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DA $CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_paths=0)')" echo "JSON(a.b.c UInt32, max_dynamic_types=1)" -$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_types=1)')" -$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_types=1)')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select json from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_types=0)')" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 -q "select JSONAllPathsWithTypes(json), JSONDynamicPathsWithTypes(json), JSONSharedDataPathsWithTypes(json) from file($DATA_FILE, JSONAsObject, 'json JSON(a.b.c UInt32, max_dynamic_types=0)')" echo "Test small max_read_buffer_size" $CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --max_read_buffer_size=1 -q "select json from file($DATA_FILE, JSONAsObject)" diff --git a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 index 42f24e7f26f..ea4e1da7181 100644 --- a/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 +++ b/tests/queries/0_stateless/03209_json_type_horizontal_merges.reference.j2 @@ -1,4 +1,4 @@ -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=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10; Dynamic paths 100000 a 90000 b @@ -66,7 +66,7 @@ Shared data paths 70000 d 60000 e 10000 g -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 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; Dynamic paths 100000 a 90000 b diff --git a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 index 99257f1d6ce..d292b1454c6 100644 --- a/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 +++ b/tests/queries/0_stateless/03209_json_type_vertical_merges.reference.j2 @@ -1,4 +1,4 @@ -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=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=1; Dynamic paths 100000 a 90000 b @@ -66,7 +66,7 @@ Shared data paths 70000 d 60000 e 10000 g -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 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; Dynamic paths 100000 a 90000 b From e2feaefcaf0e88f86f303c068edcbdacaeb67252 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 15:13:49 +0200 Subject: [PATCH 0947/1722] Update src/Core/Settings.h Co-authored-by: Alexey Katsman --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bc9c6daab1b..2417ddd39e8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1116,7 +1116,7 @@ class IColumn; M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \ M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ - M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHosue will use information about nullability from the data..", 0) \ + M(UInt64Auto, schema_inference_make_columns_nullable, 1, "If set to true, all inferred types will be Nullable in schema inference. When set to false, no columns will be converted to Nullable. When set to 'auto', ClickHouse will use information about nullability from the data.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_read_bools_as_strings, true, "Allow to parse bools as strings in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ From ccb7ecb9a22ddeabe93a5b907e3ad688b04966b4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 15:13:57 +0200 Subject: [PATCH 0948/1722] Update src/Formats/FormatSettings.h Co-authored-by: Alexey Katsman --- src/Formats/FormatSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 479b1a89adf..81b34ff0c55 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -75,7 +75,7 @@ struct FormatSettings Raw }; - UInt64 schema_inference_make_columns_nullable = true; + UInt64 schema_inference_make_columns_nullable = 1; DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; From 28b0aad3f9e54beed27ee384ab81312233abaa84 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 15:16:34 +0200 Subject: [PATCH 0949/1722] Fix python style --- tests/clickhouse-test | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 515b519af3e..8c2da7334d4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1975,9 +1975,6 @@ class TestSuite: ) return random_settings_limits - def is_shebang(line: str) -> bool: - return line.startswith("#!") - def find_tag_line(lines, comment_sign): for line in lines: if line.startswith(comment_sign) and line[ @@ -2077,9 +2074,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( + all_tags_and_random_settings_limits[1] + ) self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From fc9929dc3d87a6f8065d4f46fe5002fab8d5537e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 14 Aug 2024 14:15:05 +0000 Subject: [PATCH 0950/1722] Make tests with azurite repeatable --- tests/integration/helpers/cluster.py | 16 ++++++++++++++ .../integration/test_storage_s3_queue/test.py | 21 ++++++++++++------- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 215718463e8..a97d0f9c340 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -570,6 +570,8 @@ class ClickHouseCluster: self.spark_session = None self.with_azurite = False + self.azurite_container = "cont" + self.blob_service_client = None self._azurite_port = 0 # available when with_hdfs == True @@ -2692,6 +2694,20 @@ class ClickHouseCluster: connection_string ) logging.debug(blob_service_client.get_account_information()) + containers = [c for c in blob_service_client.list_containers(name_starts_with=self.azurite_container) if c.name == self.azurite_container] + if len(containers) > 0: + for c in containers: + blob_service_client.delete_container(c) + + container_client = blob_service_client.get_container_client(self.azurite_container) + if container_client.exists(): + logging.debug(f"azurite container '{self.azurite_container}' exist, deleting all blobs") + for b in container_client.list_blobs(): + container_client.delete_blob(b.name) + else: + logging.debug(f"azurite container '{self.azurite_container}' doesn't exist, creating it") + container_client.create_container() + self.blob_service_client = blob_service_client return except Exception as ex: diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 00ef8499594..ff723d0792a 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -13,7 +13,6 @@ from uuid import uuid4 AVAILABLE_MODES = ["unordered", "ordered"] DEFAULT_AUTH = ["'minio'", "'minio123'"] NO_AUTH = ["NOSIGN"] -AZURE_CONTAINER_NAME = "cont" def prepare_public_s3_bucket(started_cluster): @@ -75,6 +74,17 @@ def s3_queue_setup_teardown(started_cluster): objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) for obj in objects: minio.remove_object(started_cluster.minio_bucket, obj.object_name) + + container_client = started_cluster.blob_service_client.get_container_client( + started_cluster.azurite_container + ) + + if container_client.exists(): + blob_names = [b.name for b in container_client.list_blobs()] + logging.debug(f"Deleting blobs: {blob_names}") + for b in blob_names: + container_client.delete_blob(b) + yield # run test @@ -129,11 +139,6 @@ def started_cluster(): cluster.start() logging.info("Cluster started") - container_client = cluster.blob_service_client.get_container_client( - AZURE_CONTAINER_NAME - ) - container_client.create_container() - yield cluster finally: cluster.shutdown() @@ -190,7 +195,7 @@ def put_s3_file_content(started_cluster, filename, data, bucket=None): def put_azure_file_content(started_cluster, filename, data, bucket=None): client = started_cluster.blob_service_client.get_blob_client( - AZURE_CONTAINER_NAME, filename + started_cluster.azurite_container, filename ) buf = io.BytesIO(data) client.upload_blob(buf, "BlockBlob", len(data)) @@ -313,7 +318,7 @@ def test_delete_after_processing(started_cluster, mode, engine_name): assert len(objects) == 0 else: client = started_cluster.blob_service_client.get_container_client( - AZURE_CONTAINER_NAME + started_cluster.azurite_container ) objects_iterator = client.list_blobs(files_path) for objects in objects_iterator: From 920b88846b585a335b71c6f36a208f8cce8a5a74 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 14 Aug 2024 13:14:01 +0000 Subject: [PATCH 0951/1722] Optionally re-enable compilation with -O0 --- CMakeLists.txt | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 884d5be42de..6fa91fa002f 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -322,17 +322,21 @@ if (DISABLE_OMIT_FRAME_POINTER) set (CMAKE_ASM_FLAGS_ADD "${CMAKE_ASM_FLAGS_ADD} -fno-omit-frame-pointer -mno-omit-leaf-frame-pointer") endif() +# Before you start hating your debugger because it refuses to show variables (''), try building with -DDEBUG_O_LEVEL="0" +# https://stackoverflow.com/questions/63386189/whats-the-difference-between-a-compilers-o0-option-and-og-option/63386263#63386263 +set(DEBUG_O_LEVEL "g" CACHE STRING "The -Ox level used for debug builds") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}") -set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -Og ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}") +set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O${DEBUG_O_LEVEL} ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}") -set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -Og ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}") +set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O${DEBUG_O_LEVEL} ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} ${COMPILER_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") set (CMAKE_ASM_FLAGS_RELWITHDEBINFO "${CMAKE_ASM_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") -set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -Og ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") +set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -O${DEBUG_O_LEVEL} ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") if (OS_DARWIN) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") From 07c4a072fe14567ebe9809d19f869fc7a948f73e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Aug 2024 16:45:02 +0200 Subject: [PATCH 0952/1722] Add debug logging --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 9 ++++++--- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 3 ++- src/Interpreters/Cache/FileCache.cpp | 11 ++++++++++- src/Interpreters/Cache/FileCache.h | 3 ++- src/Interpreters/Cache/FileSegment.cpp | 8 ++++++-- src/Interpreters/Cache/FileSegment.h | 6 +++++- src/Interpreters/Cache/WriteBufferToFileSegment.cpp | 6 ++++-- 7 files changed, 35 insertions(+), 11 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index b471f3fc58f..286d06bc424 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -645,8 +645,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size); + std::string failure_reason; bool continue_predownload = file_segment.reserve( - current_predownload_size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds); + current_predownload_size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds, failure_reason); if (continue_predownload) { LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); @@ -1002,7 +1003,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); - bool success = file_segment.reserve(size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds); + std::string failure_reason; + bool success = file_segment.reserve(size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds, failure_reason); if (success) { chassert(file_segment.getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); @@ -1028,7 +1030,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TRACE(log, "Bypassing cache because writeCache method failed"); } else - LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size); + LOG_TRACE(log, "No space left in cache to reserve {} bytes, reason: {}, " + "will continue without cache download", failure_reason, size); if (!success) { diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 382c4a80cc4..103ae0e1832 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -91,7 +91,8 @@ bool FileSegmentRangeWriter::write(char * data, size_t size, size_t offset, File size_t size_to_write = std::min(available_size, size); - bool reserved = file_segment->reserve(size_to_write, reserve_space_lock_wait_timeout_milliseconds); + std::string failure_reason; + bool reserved = file_segment->reserve(size_to_write, reserve_space_lock_wait_timeout_milliseconds, failure_reason); if (!reserved) { appendFilesystemCacheLog(*file_segment); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 13c70b38543..b1a1f629b00 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -804,7 +804,8 @@ bool FileCache::tryReserve( const size_t size, FileCacheReserveStat & reserve_stat, const UserInfo & user, - size_t lock_wait_timeout_milliseconds) + size_t lock_wait_timeout_milliseconds, + std::string & failure_reason) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheReserveMicroseconds); @@ -817,6 +818,7 @@ bool FileCache::tryReserve( if (cache_is_being_resized.load(std::memory_order_relaxed)) { ProfileEvents::increment(ProfileEvents::FilesystemCacheFailToReserveSpaceBecauseOfCacheResize); + failure_reason = "cache is being resized"; return false; } @@ -824,6 +826,7 @@ bool FileCache::tryReserve( if (!cache_lock) { ProfileEvents::increment(ProfileEvents::FilesystemCacheFailToReserveSpaceBecauseOfLockContention); + failure_reason = "cache contention"; return false; } @@ -847,6 +850,7 @@ bool FileCache::tryReserve( LOG_TEST(log, "Query limit exceeded, space reservation failed, " "recache_on_query_limit_exceeded is disabled (while reserving for {}:{})", file_segment.key(), file_segment.offset()); + failure_reason = "query limit exceeded"; return false; } @@ -877,6 +881,7 @@ bool FileCache::tryReserve( if (!query_priority->collectCandidatesForEviction( size, required_elements_num, reserve_stat, eviction_candidates, {}, user.user_id, cache_lock)) { + failure_reason = "cannot evict enough space for query limit"; return false; } @@ -891,11 +896,15 @@ bool FileCache::tryReserve( if (!main_priority->collectCandidatesForEviction( size, required_elements_num, reserve_stat, eviction_candidates, queue_iterator, user.user_id, cache_lock)) { + failure_reason = "cannot evict enough space"; return false; } if (!file_segment.getKeyMetadata()->createBaseDirectory()) + { + failure_reason = "not enough space on device"; return false; + } if (eviction_candidates.size() > 0) { diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 07be802a940..efa504689eb 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -165,7 +165,8 @@ public: size_t size, FileCacheReserveStat & stat, const UserInfo & user, - size_t lock_wait_timeout_milliseconds); + size_t lock_wait_timeout_milliseconds, + std::string & failure_reason); std::vector getFileSegmentInfos(const UserID & user_id); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index c46fb978ae4..cfbdfbaa257 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -502,7 +502,11 @@ LockedKeyPtr FileSegment::lockKeyMetadata(bool assert_exists) const return metadata->tryLock(); } -bool FileSegment::reserve(size_t size_to_reserve, size_t lock_wait_timeout_milliseconds, FileCacheReserveStat * reserve_stat) +bool FileSegment::reserve( + size_t size_to_reserve, + size_t lock_wait_timeout_milliseconds, + std::string & failure_reason, + FileCacheReserveStat * reserve_stat) { if (!size_to_reserve) throw Exception(ErrorCodes::LOGICAL_ERROR, "Zero space reservation is not allowed"); @@ -554,7 +558,7 @@ bool FileSegment::reserve(size_t size_to_reserve, size_t lock_wait_timeout_milli if (!reserve_stat) reserve_stat = &dummy_stat; - bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user, lock_wait_timeout_milliseconds); + bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user, lock_wait_timeout_milliseconds, failure_reason); if (!reserved) setDownloadFailedUnlocked(lock()); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 25ffb880b45..e90ebdbf8fe 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -201,7 +201,11 @@ public: /// Try to reserve exactly `size` bytes (in addition to the getDownloadedSize() bytes already downloaded). /// Returns true if reservation was successful, false otherwise. - bool reserve(size_t size_to_reserve, size_t lock_wait_timeout_milliseconds, FileCacheReserveStat * reserve_stat = nullptr); + bool reserve( + size_t size_to_reserve, + size_t lock_wait_timeout_milliseconds, + std::string & failure_reason, + FileCacheReserveStat * reserve_stat = nullptr); /// Write data into reserved space. void write(char * from, size_t size, size_t offset_in_file); diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index e6ebf6ad50c..e43bbacdbc5 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -75,7 +75,8 @@ void WriteBufferToFileSegment::nextImpl() FileCacheReserveStat reserve_stat; /// In case of an error, we don't need to finalize the file segment /// because it will be deleted soon and completed in the holder's destructor. - bool ok = file_segment->reserve(bytes_to_write, reserve_space_lock_wait_timeout_milliseconds, &reserve_stat); + std::string failure_reason; + bool ok = file_segment->reserve(bytes_to_write, reserve_space_lock_wait_timeout_milliseconds, failure_reason, &reserve_stat); if (!ok) { @@ -84,9 +85,10 @@ void WriteBufferToFileSegment::nextImpl() reserve_stat_msg += fmt::format("{} hold {}, can release {}; ", toString(kind), ReadableSize(stat.non_releasable_size), ReadableSize(stat.releasable_size)); - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Failed to reserve {} bytes for {}: {}(segment info: {})", + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Failed to reserve {} bytes for {}: reason {}, {}(segment info: {})", bytes_to_write, file_segment->getKind() == FileSegmentKind::Temporary ? "temporary file" : "the file in cache", + failure_reason, reserve_stat_msg, file_segment->getInfoForLog() ); From 4827b8bb1c7a77e50912ab40d5c009c43d20f6ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 14 Aug 2024 14:56:02 +0000 Subject: [PATCH 0953/1722] Make S3Queue tests repeatable --- .../integration/test_storage_s3_queue/test.py | 97 ++++++++++++++----- 1 file changed, 72 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index ff723d0792a..08a8a7cac81 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1,6 +1,7 @@ import io import logging import random +import string import time import pytest @@ -267,6 +268,10 @@ def create_mv( ) +def generate_random_string(length=6): + return "".join(random.choice(string.ascii_lowercase) for i in range(length)) + + @pytest.mark.parametrize("mode", ["unordered", "ordered"]) @pytest.mark.parametrize("engine_name", ["S3Queue", "AzureQueue"]) def test_delete_after_processing(started_cluster, mode, engine_name): @@ -276,6 +281,8 @@ def test_delete_after_processing(started_cluster, mode, engine_name): files_path = f"{table_name}_data" files_num = 5 row_num = 10 + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" if engine_name == "S3Queue": storage = "s3" else: @@ -290,7 +297,7 @@ def test_delete_after_processing(started_cluster, mode, engine_name): table_name, mode, files_path, - additional_settings={"after_processing": "delete"}, + additional_settings={"after_processing": "delete", "keeper_path": keeper_path}, engine_name=engine_name, ) create_mv(node, table_name, dst_table_name) @@ -333,7 +340,8 @@ def test_failed_retry(started_cluster, mode, engine_name): dst_table_name = f"{table_name}_dst" files_path = f"{table_name}_data" file_path = f"{files_path}/trash_test.csv" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" retries_num = 3 values = [ @@ -391,7 +399,8 @@ def test_failed_retry(started_cluster, mode, engine_name): def test_direct_select_file(started_cluster, mode): node = started_cluster.instances["instance"] table_name = f"test.direct_select_file_{mode}" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" file_path = f"{files_path}/test.csv" @@ -496,8 +505,17 @@ def test_direct_select_multiple_files(started_cluster, mode): node = started_cluster.instances["instance"] table_name = f"direct_select_multiple_files_{mode}" files_path = f"{table_name}_data" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" - create_table(started_cluster, node, table_name, mode, files_path) + create_table( + started_cluster, + node, + table_name, + mode, + files_path, + additional_settings={"keeper_path": keeper_path}, + ) for i in range(5): rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] values_csv = ( @@ -520,14 +538,23 @@ def test_direct_select_multiple_files(started_cluster, mode): @pytest.mark.parametrize("mode", AVAILABLE_MODES) -def test_streaming_to_view_(started_cluster, mode): +def test_streaming_to_view(started_cluster, mode): node = started_cluster.instances["instance"] table_name = f"streaming_to_view_{mode}" dst_table_name = f"{table_name}_dst" files_path = f"{table_name}_data" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" total_values = generate_random_files(started_cluster, files_path, 10) - create_table(started_cluster, node, table_name, mode, files_path) + create_table( + started_cluster, + node, + table_name, + mode, + files_path, + additional_settings={"keeper_path": keeper_path}, + ) create_mv(node, table_name, dst_table_name) expected_values = set([tuple(i) for i in total_values]) @@ -549,7 +576,8 @@ def test_streaming_to_many_views(started_cluster, mode): node = started_cluster.instances["instance"] table_name = f"streaming_to_many_views_{mode}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" for i in range(3): @@ -587,7 +615,8 @@ def test_streaming_to_many_views(started_cluster, mode): def test_multiple_tables_meta_mismatch(started_cluster): node = started_cluster.instances["instance"] table_name = f"multiple_tables_meta_mismatch" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" create_table( @@ -680,7 +709,8 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): node = started_cluster.instances["instance"] table_name = f"multiple_tables_streaming_sync_{mode}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 300 @@ -763,7 +793,8 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): node_2 = started_cluster.instances["instance2"] table_name = f"multiple_tables_streaming_sync_distributed_{mode}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 300 row_num = 50 @@ -838,7 +869,8 @@ def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] table_name = "max_set_age" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" max_age = 20 files_to_generate = 10 @@ -949,10 +981,9 @@ def test_max_set_age(started_cluster): def test_max_set_size(started_cluster): node = started_cluster.instances["instance"] table_name = f"max_set_size" - dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" - max_age = 10 files_to_generate = 10 create_table( @@ -996,7 +1027,8 @@ def test_drop_table(started_cluster): node = started_cluster.instances["instance"] table_name = f"test_drop" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 300 @@ -1029,6 +1061,8 @@ def test_s3_client_reused(started_cluster): table_name = f"test.test_s3_client_reused" dst_table_name = f"{table_name}_dst" files_path = f"{table_name}_data" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" row_num = 10 def get_created_s3_clients_count(): @@ -1062,6 +1096,7 @@ def test_s3_client_reused(started_cluster): additional_settings={ "after_processing": "delete", "s3queue_processing_threads_num": 1, + "keeper_path": keeper_path, }, auth=NO_AUTH, bucket=started_cluster.minio_public_bucket, @@ -1119,7 +1154,8 @@ def test_processing_threads(started_cluster, mode): node = started_cluster.instances["instance"] table_name = f"processing_threads_{mode}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 300 processing_threads = 32 @@ -1186,7 +1222,8 @@ def test_shards(started_cluster, mode, processing_threads): node = started_cluster.instances["instance"] table_name = f"test_shards_{mode}_{processing_threads}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 300 shards_num = 3 @@ -1313,7 +1350,8 @@ def test_shards_distributed(started_cluster, mode, processing_threads): node_2 = started_cluster.instances["instance2"] table_name = f"test_shards_distributed_{mode}_{processing_threads}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 300 row_num = 300 @@ -1466,8 +1504,8 @@ def test_settings_check(started_cluster): node = started_cluster.instances["instance"] node_2 = started_cluster.instances["instance2"] table_name = f"test_settings_check" - dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" mode = "ordered" @@ -1509,7 +1547,10 @@ def test_processed_file_setting(started_cluster, processing_threads): node = started_cluster.instances["instance"] table_name = f"test_processed_file_setting_{processing_threads}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}_{processing_threads}" + # A unique path is necessary for repeatable tests + keeper_path = ( + f"/clickhouse/test_{table_name}_{processing_threads}_{generate_random_string()}" + ) files_path = f"{table_name}_data" files_to_generate = 10 @@ -1560,7 +1601,10 @@ def test_processed_file_setting_distributed(started_cluster, processing_threads) node_2 = started_cluster.instances["instance2"] table_name = f"test_processed_file_setting_distributed_{processing_threads}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = ( + f"/clickhouse/test_{table_name}_{processing_threads}_{generate_random_string()}" + ) files_path = f"{table_name}_data" files_to_generate = 10 @@ -1614,7 +1658,8 @@ def test_upgrade(started_cluster): table_name = f"test_upgrade" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 10 @@ -1655,7 +1700,8 @@ def test_exception_during_insert(started_cluster): table_name = f"test_exception_during_insert" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 10 @@ -1708,7 +1754,8 @@ def test_commit_on_limit(started_cluster): table_name = f"test_commit_on_limit" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}" + # A unique path is necessary for repeatable tests + keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" files_path = f"{table_name}_data" files_to_generate = 10 From c2f6751dafe3ecd0841f85799c0b15ae91e7cef9 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 14 Aug 2024 17:16:16 +0200 Subject: [PATCH 0954/1722] fix bug in key condition --- src/Storages/MergeTree/KeyCondition.cpp | 29 ++----------------- .../03221_key_condition_bug.reference | 1 + .../0_stateless/03221_key_condition_bug.sql | 11 +++++++ 3 files changed, 14 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/03221_key_condition_bug.reference create mode 100644 tests/queries/0_stateless/03221_key_condition_bug.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dfb43c4e75d..d0391dd9695 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -888,31 +888,6 @@ static Field applyFunctionForField( return (*col)[0]; } -static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) -{ - /// Fallback for fields without block reference. - if (field.isExplicit()) - return applyFunctionForField(func, current_type, field); - - String result_name = "_" + func->getName() + "_" + toString(field.column_idx); - const auto & columns = field.columns; - size_t result_idx = columns->size(); - - for (size_t i = 0; i < result_idx; ++i) - { - if ((*columns)[i].name == result_name) - result_idx = i; - } - - if (result_idx == columns->size()) - { - ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; - field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); - (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); - } - - return {field.columns, field.row_idx, result_idx}; -} /// Sequentially applies functions to the column, returns `true` /// if all function arguments are compatible with functions @@ -2563,13 +2538,13 @@ std::optional KeyCondition::applyMonotonicFunctionsChainToRange( /// Thus we can safely use isNull() as an -Inf/+Inf indicator here. if (!key_range.left.isNull()) { - key_range.left = applyFunction(func, current_type, key_range.left); + key_range.left = applyFunctionForField(func, current_type, key_range.left); key_range.left_included = true; } if (!key_range.right.isNull()) { - key_range.right = applyFunction(func, current_type, key_range.right); + key_range.right = applyFunctionForField(func, current_type, key_range.right); key_range.right_included = true; } diff --git a/tests/queries/0_stateless/03221_key_condition_bug.reference b/tests/queries/0_stateless/03221_key_condition_bug.reference new file mode 100644 index 00000000000..e373ee695f6 --- /dev/null +++ b/tests/queries/0_stateless/03221_key_condition_bug.reference @@ -0,0 +1 @@ +50 diff --git a/tests/queries/0_stateless/03221_key_condition_bug.sql b/tests/queries/0_stateless/03221_key_condition_bug.sql new file mode 100644 index 00000000000..bac3e631a81 --- /dev/null +++ b/tests/queries/0_stateless/03221_key_condition_bug.sql @@ -0,0 +1,11 @@ +CREATE TABLE IF NOT EXISTS report_metrics_v2 +( + `a` UInt64 +) Engine = MergeTree() +ORDER BY a; + +insert into report_metrics_v2 SELECT * FROM system.numbers LIMIT 50000; + +SELECT count(*) from report_metrics_v2 WHERE (intDiv(a, 50) = 200) AND (intDiv(a, 50000) = 0); + +DROP TABLE report_metrics_v2; From 2aef696856cbded795aedc23b7b8963799b8ebe6 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 14 Aug 2024 15:27:37 +0000 Subject: [PATCH 0955/1722] make test runnable multiple times --- .../test_delayed_replica_failover/test.py | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index 1116d225b8c..ed63a47e030 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -20,21 +20,29 @@ node_1_2 = cluster.add_instance("node_1_2", with_zookeeper=True) node_2_1 = cluster.add_instance("node_2_1", with_zookeeper=True) node_2_2 = cluster.add_instance("node_2_2", with_zookeeper=True) +# For test to be runnable multiple times +seqno = 0 @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() + yield cluster + finally: + cluster.shutdown() + +@pytest.fixture(scope="function", autouse=True) +def create_tables(): + global seqno + try: + seqno += 1 for shard in (1, 2): for replica in (1, 2): node = cluster.instances["node_{}_{}".format(shard, replica)] node.query( - """ -CREATE TABLE replicated (d Date, x UInt32) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}') PARTITION BY toYYYYMM(d) ORDER BY d""".format( - shard=shard, instance=node.name - ) + f"CREATE TABLE replicated (d Date, x UInt32) ENGINE = " + f"ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated_{seqno}', '{node.name}') PARTITION BY toYYYYMM(d) ORDER BY d" ) node_1_1.query( @@ -42,10 +50,15 @@ CREATE TABLE replicated (d Date, x UInt32) ENGINE = "Distributed('test_cluster', 'default', 'replicated')" ) - yield cluster + yield finally: - cluster.shutdown() + node_1_1.query("DROP TABLE distributed") + + node_1_1.query("DROP TABLE replicated") + node_1_2.query("DROP TABLE replicated") + node_2_1.query("DROP TABLE replicated") + node_2_2.query("DROP TABLE replicated") def test(started_cluster): From 23bb47293d3de355710cb93f6bd7dcd7b130ea79 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 14 Aug 2024 18:13:00 +0200 Subject: [PATCH 0956/1722] cache result column with ptr hex string --- src/Storages/MergeTree/KeyCondition.cpp | 30 +++++++++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d0391dd9695..bad9c2b422c 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -888,6 +888,32 @@ static Field applyFunctionForField( return (*col)[0]; } +static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) +{ + /// Fallback for fields without block reference. + if (field.isExplicit()) + return applyFunctionForField(func, current_type, field); + + std::stringstream buf; + buf << "_" << func.get() << "_" << toString(field.column_idx); + String result_name = buf.str(); + const auto & columns = field.columns; + size_t result_idx = columns->size(); + + for (size_t i = 0; i < result_idx; ++i) + { + if ((*columns)[i].name == result_name) + result_idx = i; + } + + if (result_idx == columns->size()) + { + ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; + field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); + (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); + } + return {field.columns, field.row_idx, result_idx}; +} /// Sequentially applies functions to the column, returns `true` /// if all function arguments are compatible with functions @@ -2538,13 +2564,13 @@ std::optional KeyCondition::applyMonotonicFunctionsChainToRange( /// Thus we can safely use isNull() as an -Inf/+Inf indicator here. if (!key_range.left.isNull()) { - key_range.left = applyFunctionForField(func, current_type, key_range.left); + key_range.left = applyFunction(func, current_type, key_range.left); key_range.left_included = true; } if (!key_range.right.isNull()) { - key_range.right = applyFunctionForField(func, current_type, key_range.right); + key_range.right = applyFunction(func, current_type, key_range.right); key_range.right_included = true; } From 537f3bcd76fbbbfb1ee4b7b718b0630fad95509c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 14 Aug 2024 16:13:17 +0000 Subject: [PATCH 0957/1722] Automatic style fix --- tests/integration/helpers/cluster.py | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a97d0f9c340..0b6cf03d467 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2694,18 +2694,30 @@ class ClickHouseCluster: connection_string ) logging.debug(blob_service_client.get_account_information()) - containers = [c for c in blob_service_client.list_containers(name_starts_with=self.azurite_container) if c.name == self.azurite_container] + containers = [ + c + for c in blob_service_client.list_containers( + name_starts_with=self.azurite_container + ) + if c.name == self.azurite_container + ] if len(containers) > 0: for c in containers: blob_service_client.delete_container(c) - container_client = blob_service_client.get_container_client(self.azurite_container) + container_client = blob_service_client.get_container_client( + self.azurite_container + ) if container_client.exists(): - logging.debug(f"azurite container '{self.azurite_container}' exist, deleting all blobs") + logging.debug( + f"azurite container '{self.azurite_container}' exist, deleting all blobs" + ) for b in container_client.list_blobs(): container_client.delete_blob(b.name) else: - logging.debug(f"azurite container '{self.azurite_container}' doesn't exist, creating it") + logging.debug( + f"azurite container '{self.azurite_container}' doesn't exist, creating it" + ) container_client.create_container() self.blob_service_client = blob_service_client From 8e6096dee72acc5ee75eb05ddfb9384767f52648 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 14 Aug 2024 16:13:22 +0000 Subject: [PATCH 0958/1722] Automatic style fix --- tests/integration/test_delayed_replica_failover/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index ed63a47e030..f1034e26b25 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -23,6 +23,7 @@ node_2_2 = cluster.add_instance("node_2_2", with_zookeeper=True) # For test to be runnable multiple times seqno = 0 + @pytest.fixture(scope="module") def started_cluster(): try: From acc88af2cbe5f993cf244cf44a0c448c2e76bf37 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 14 Aug 2024 18:27:20 +0200 Subject: [PATCH 0959/1722] fix style --- src/Storages/MergeTree/KeyCondition.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index bad9c2b422c..a297440e8f8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -894,8 +894,10 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & if (field.isExplicit()) return applyFunctionForField(func, current_type, field); - std::stringstream buf; - buf << "_" << func.get() << "_" << toString(field.column_idx); + WriteBufferFromOwnString buf; + writeText("_", buf); + writePointerHex(func.get(), buf); + writeText("_" + toString(field.column_idx), buf); String result_name = buf.str(); const auto & columns = field.columns; size_t result_idx = columns->size(); @@ -912,6 +914,7 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); } + return {field.columns, field.row_idx, result_idx}; } From cf58e8c1e37dda01b73b0cbd2553e3b460aa28ad Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 14 Aug 2024 16:35:45 +0000 Subject: [PATCH 0960/1722] fix data race in `DynamicResourceManager::updateConfiguration` --- .../Nodes/DynamicResourceManager.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index 01aa7df48d3..6b9f6318903 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -184,14 +184,20 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi // Resource update leads to loss of runtime data of nodes and may lead to temporary violation of constraints (e.g. limits) // Try to minimise this by reusing "equal" resources (initialized with the same configuration). + std::vector resources_to_attach; for (auto & [name, new_resource] : new_state->resources) { if (auto iter = state->resources.find(name); iter != state->resources.end()) // Resource update { State::ResourcePtr old_resource = iter->second; if (old_resource->equals(*new_resource)) + { new_resource = old_resource; // Rewrite with older version to avoid loss of runtime data + continue; + } } + // It is new or updated resource + resources_to_attach.emplace_back(new_resource); } // Commit new state @@ -199,17 +205,14 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi state = new_state; // Attach new and updated resources to the scheduler - for (auto & [name, resource] : new_state->resources) + for (auto & resource : resources_to_attach) { const SchedulerNodePtr & root = resource->nodes.find("/")->second.ptr; - if (root->parent == nullptr) + resource->attached_to = &scheduler; + scheduler.event_queue->enqueue([this, root] { - resource->attached_to = &scheduler; - scheduler.event_queue->enqueue([this, root] - { - scheduler.attachChild(root); - }); - } + scheduler.attachChild(root); + }); } // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable From 61b96ed7498a19c315d03d6d23330c06837dc990 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 14 Aug 2024 16:43:12 +0000 Subject: [PATCH 0961/1722] Make rest of the tests repeatable --- .../integration/test_storage_s3_queue/test.py | 77 ++++++++++++------- 1 file changed, 48 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 08a8a7cac81..664d537a8d1 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -68,8 +68,8 @@ def s3_queue_setup_teardown(started_cluster): instance = started_cluster.instances["instance"] instance_2 = started_cluster.instances["instance2"] - instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") - instance_2.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + instance.query("DROP DATABASE IF EXISTS default; CREATE DATABASE default;") + instance_2.query("DROP DATABASE IF EXISTS default; CREATE DATABASE default;") minio = started_cluster.minio_client objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) @@ -276,7 +276,7 @@ def generate_random_string(length=6): @pytest.mark.parametrize("engine_name", ["S3Queue", "AzureQueue"]) def test_delete_after_processing(started_cluster, mode, engine_name): node = started_cluster.instances["instance"] - table_name = f"test.delete_after_processing_{mode}_{engine_name}" + table_name = f"delete_after_processing_{mode}_{engine_name}" dst_table_name = f"{table_name}_dst" files_path = f"{table_name}_data" files_num = 5 @@ -336,7 +336,7 @@ def test_delete_after_processing(started_cluster, mode, engine_name): @pytest.mark.parametrize("engine_name", ["S3Queue", "AzureQueue"]) def test_failed_retry(started_cluster, mode, engine_name): node = started_cluster.instances["instance"] - table_name = f"test.failed_retry_{mode}_{engine_name}" + table_name = f"failed_retry_{mode}_{engine_name}" dst_table_name = f"{table_name}_dst" files_path = f"{table_name}_data" file_path = f"{files_path}/trash_test.csv" @@ -398,9 +398,9 @@ def test_failed_retry(started_cluster, mode, engine_name): @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_file(started_cluster, mode): node = started_cluster.instances["instance"] - table_name = f"test.direct_select_file_{mode}" + table_name = f"direct_select_file_{mode}" # A unique path is necessary for repeatable tests - keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" + keeper_path = f"/clickhouse/test_{table_name}_{mode}_{generate_random_string()}" files_path = f"{table_name}_data" file_path = f"{files_path}/test.csv" @@ -461,7 +461,7 @@ def test_direct_select_file(started_cluster, mode): ] == [] # New table with different zookeeper path - keeper_path = f"/clickhouse/test_{table_name}_{mode}_2" + keeper_path = f"{keeper_path}_2" create_table( started_cluster, node, @@ -791,10 +791,12 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): node = started_cluster.instances["instance"] node_2 = started_cluster.instances["instance2"] - table_name = f"multiple_tables_streaming_sync_distributed_{mode}" + # A unique table name is necessary for repeatable tests + table_name = ( + f"multiple_tables_streaming_sync_distributed_{mode}_{generate_random_string()}" + ) dst_table_name = f"{table_name}_dst" - # A unique path is necessary for repeatable tests - keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" + keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 row_num = 50 @@ -1058,7 +1060,7 @@ def test_drop_table(started_cluster): def test_s3_client_reused(started_cluster): node = started_cluster.instances["instance"] - table_name = f"test.test_s3_client_reused" + table_name = f"test_s3_client_reused" dst_table_name = f"{table_name}_dst" files_path = f"{table_name}_data" # A unique path is necessary for repeatable tests @@ -1698,10 +1700,10 @@ def test_upgrade(started_cluster): def test_exception_during_insert(started_cluster): node = started_cluster.instances["instance_too_many_parts"] - table_name = f"test_exception_during_insert" + # A unique table name is necessary for repeatable tests + table_name = f"test_exception_during_insert_{generate_random_string()}" dst_table_name = f"{table_name}_dst" - # A unique path is necessary for repeatable tests - keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" + keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 10 @@ -1715,6 +1717,7 @@ def test_exception_during_insert(started_cluster): "keeper_path": keeper_path, }, ) + node.rotate_logs() total_values = generate_random_files( started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 ) @@ -1731,34 +1734,49 @@ def test_exception_during_insert(started_cluster): ) assert "Too many parts" in exception + original_parts_to_throw_insert = 0 + modified_parts_to_throw_insert = 10 node.replace_in_config( "/etc/clickhouse-server/config.d/merge_tree.xml", - "parts_to_throw_insert>0", - "parts_to_throw_insert>10", + f"parts_to_throw_insert>{original_parts_to_throw_insert}", + f"parts_to_throw_insert>{modified_parts_to_throw_insert}", ) - node.restart_clickhouse() + try: + node.restart_clickhouse() - def get_count(): - return int(node.query(f"SELECT count() FROM {dst_table_name}")) + def get_count(): + return int(node.query(f"SELECT count() FROM {dst_table_name}")) - expected_rows = 10 - for _ in range(20): - if expected_rows == get_count(): - break - time.sleep(1) - assert expected_rows == get_count() + expected_rows = 10 + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + assert expected_rows == get_count() + finally: + node.replace_in_config( + "/etc/clickhouse-server/config.d/merge_tree.xml", + f"parts_to_throw_insert>{modified_parts_to_throw_insert}", + f"parts_to_throw_insert>{original_parts_to_throw_insert}", + ) + node.restart_clickhouse() def test_commit_on_limit(started_cluster): node = started_cluster.instances["instance"] - table_name = f"test_commit_on_limit" + # A unique table name is necessary for repeatable tests + table_name = f"test_commit_on_limit_{generate_random_string()}" dst_table_name = f"{table_name}_dst" - # A unique path is necessary for repeatable tests - keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" + keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 10 + failed_files_event_before = int( + node.query( + "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" + ) + ) create_table( started_cluster, node, @@ -1833,7 +1851,8 @@ def test_commit_on_limit(started_cluster): assert "test_999999.csv" in get_processed_files() - assert 1 == int( + node.count_in_log(f"Setting file {files_path}/test_9999.csv as failed") + assert failed_files_event_before + 1 == int( node.query( "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" ) From f11478398ec563218644eb3d8c16ae6f223c1a13 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 14 Aug 2024 16:43:26 +0000 Subject: [PATCH 0962/1722] Update version_date.tsv and changelogs after v24.3.7.30-lts --- docs/changelogs/v24.3.7.30-lts.md | 29 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 30 insertions(+) create mode 100644 docs/changelogs/v24.3.7.30-lts.md diff --git a/docs/changelogs/v24.3.7.30-lts.md b/docs/changelogs/v24.3.7.30-lts.md new file mode 100644 index 00000000000..f945a54840f --- /dev/null +++ b/docs/changelogs/v24.3.7.30-lts.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.7.30-lts (c8a28cf4331) FIXME as compared to v24.3.6.48-lts (b2d33c3c45d) + +#### Improvement +* Backported in [#68103](https://github.com/ClickHouse/ClickHouse/issues/68103): Distinguish booleans and integers while parsing values for custom settings: ``` SET custom_a = true; SET custom_b = 1; ```. [#62206](https://github.com/ClickHouse/ClickHouse/pull/62206) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67931](https://github.com/ClickHouse/ClickHouse/issues/67931): Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68062](https://github.com/ClickHouse/ClickHouse/issues/68062): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#67812](https://github.com/ClickHouse/ClickHouse/issues/67812): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67848](https://github.com/ClickHouse/ClickHouse/issues/67848): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68271](https://github.com/ClickHouse/ClickHouse/issues/68271): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67806](https://github.com/ClickHouse/ClickHouse/issues/67806): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67834](https://github.com/ClickHouse/ClickHouse/issues/67834): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#68206](https://github.com/ClickHouse/ClickHouse/issues/68206): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68089](https://github.com/ClickHouse/ClickHouse/issues/68089): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68120](https://github.com/ClickHouse/ClickHouse/issues/68120): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67676](https://github.com/ClickHouse/ClickHouse/pull/67676) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68074](https://github.com/ClickHouse/ClickHouse/issues/68074): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f46353277e2..71a4a722a36 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -13,6 +13,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 v24.3.5.46-lts 2024-07-03 v24.3.4.147-lts 2024-06-13 From 7b1bca2b488685e3953c9b2950d788d565bff73d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 14 Aug 2024 16:55:59 +0000 Subject: [PATCH 0963/1722] Add missing assertion --- tests/integration/test_storage_s3_queue/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 664d537a8d1..34fb1eaf1fe 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1851,7 +1851,9 @@ def test_commit_on_limit(started_cluster): assert "test_999999.csv" in get_processed_files() - node.count_in_log(f"Setting file {files_path}/test_9999.csv as failed") + assert 1 == int( + node.count_in_log(f"Setting file {files_path}/test_9999.csv as failed") + ) assert failed_files_event_before + 1 == int( node.query( "SELECT value FROM system.events WHERE name = 'ObjectStorageQueueFailedFiles' SETTINGS system_events_show_zero_values=1" From 209d4eb016a58acb27826ce96e61db884490b66f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Aug 2024 19:00:17 +0200 Subject: [PATCH 0964/1722] Fix build --- src/Interpreters/Cache/Metadata.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 7e4b76d3cc6..6399691bcf6 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -705,7 +705,8 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optionalavailable(); - if (!file_segment.reserve(size, reserve_space_lock_wait_timeout_milliseconds)) + std::string failure_reason; + if (!file_segment.reserve(size, reserve_space_lock_wait_timeout_milliseconds, failure_reason)) { LOG_TEST( log, "Failed to reserve space during background download " From 207d160dd2fff9cb0cb4ec95970cd79c4e13d86b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 14 Aug 2024 19:01:54 +0200 Subject: [PATCH 0965/1722] address comments --- src/Storages/MergeTree/KeyCondition.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index a297440e8f8..98c02b3e894 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -888,12 +888,17 @@ static Field applyFunctionForField( return (*col)[0]; } +/// applyFunction will execute the function with one `field` or the column which `field` refers to. static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) { + chassert(func != nullptr); /// Fallback for fields without block reference. if (field.isExplicit()) return applyFunctionForField(func, current_type, field); + /// We will cache the function result inside `field.columns`, because this function will call many times + /// from many fields from same column. When the column is huge, for example there are thousands of marks, we need a cache. + /// The cache key is like `_[function_pointer]_[param_column_id]` to identify a unique pair. WriteBufferFromOwnString buf; writeText("_", buf); writePointerHex(func.get(), buf); @@ -910,6 +915,7 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & if (result_idx == columns->size()) { + /// When cache is missed, we calculate the whole column where the field comes from. This will avoid repeated calculation. ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); From 7a4bd49c4285ef3489d90aa163fb6b1858a43562 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 14 Aug 2024 17:16:36 +0000 Subject: [PATCH 0966/1722] More improvements in integration test --- .../test_alter_settings_on_cluster/test.py | 5 +++ .../test_always_fetch_merged/test.py | 3 ++ .../test.py | 6 +-- .../test_async_load_databases/test.py | 14 +++++-- .../test.py | 42 +++++++++---------- 5 files changed, 42 insertions(+), 28 deletions(-) diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py index 32f7f2efa30..7a552c383c9 100644 --- a/tests/integration/test_alter_settings_on_cluster/test.py +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -73,3 +73,8 @@ def test_default_database_on_cluster(started_cluster): database="test_default_database", sql="SHOW CREATE test_local_table FORMAT TSV", ).endswith("old_parts_lifetime = 100\n") + + ch1.query( + database="test_default_database", + sql="DROP TABLE test_local_table ON CLUSTER 'cluster' SYNC", + ) diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index ca8e775fb97..3bbfc8867f8 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -80,3 +80,6 @@ def test_replica_always_download(started_cluster): assert int(node1_parts) < 10 assert int(node2_parts) < 10 + + node1.query_with_retry("DROP TABLE test_table SYNC") + node2.query_with_retry("DROP TABLE test_table SYNC") diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 93319a56d0f..0ea076b1468 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -157,7 +157,7 @@ def test_with_replicated_merge_tree(): array_size_range=[10, 50], ) - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE {} SYNC".format(table_name)) def test_with_replicated_merge_tree_multithread(): @@ -185,7 +185,7 @@ def test_with_replicated_merge_tree_multithread(): array_size_range=[10, 15], ) - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE {} SYNC".format(table_name)) # Ensure that the combined duration of inserts with adaptive timeouts is less than @@ -369,4 +369,4 @@ def test_change_queries_frequency(): for line in res.splitlines(): assert int(line) == min_ms - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index d06897b1045..f36cff76ea2 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -28,9 +28,6 @@ def started_cluster(): """ CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; CREATE DATABASE IF NOT EXISTS test; - DROP TABLE IF EXISTS test.elements; - CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log; - INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7); """ ) @@ -49,6 +46,13 @@ def get_status(dictionary_name): def test_dict_get_data(started_cluster): query = instance.query + query( + "CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log;" + ) + query( + "INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7);" + ) + # dictionaries_lazy_load == false, so these dictionary are not loaded. assert get_status("dep_x") == "NOT_LOADED" assert get_status("dep_y") == "NOT_LOADED" @@ -97,6 +101,8 @@ def test_dict_get_data(started_cluster): assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "XX\n" assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ZZ\n" + query("DROP TABLE IF EXISTS test.elements;") + instance.restart_clickhouse() def dependent_tables_assert(): @@ -175,3 +181,5 @@ def test_multiple_tables(started_cluster): random.shuffle(order) for i in order: assert query(f"select count() from test.table_{i}") == "100\n" + for i in range(tables_count): + query(f"drop table test.table_{i}") diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 622620e232a..efdd96004d2 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -26,26 +26,24 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - try: - cluster.start() - node1.query("SET log_queries = 1;") - node1.query("CREATE DATABASE replica;") - query_create = """CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=MergeTree() - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;""" - time.sleep(2) - node1.query(query_create) - node1.query("""INSERT INTO replica.test VALUES (1, now())""") - node1.query("SYSTEM FLUSH LOGS;") + node1.query("SET log_queries = 1;") + node1.query("CREATE DATABASE replica;") + query_create = """CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;""" + time.sleep(2) + node1.query(query_create) + node1.query("""INSERT INTO replica.test VALUES (1, now())""") + node1.query("SYSTEM FLUSH LOGS;") - test_query = ( - "SELECT count() > 0 ? 'ok' : 'fail' FROM system.asynchronous_metric_log" - ) - assert "ok\n" in node1.query(test_query) - finally: - cluster.shutdown() + test_query = ( + "SELECT count() > 0 ? 'ok' : 'fail' FROM system.asynchronous_metric_log" + ) + assert "ok\n" in node1.query(test_query) + node1.query("DROP TABLE replica.test") + node1.query("DROP DATABASE replica") From 2e5f45a7ad4924affb1ff8b0e5a40b59b6549621 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Wed, 14 Aug 2024 17:53:33 +0000 Subject: [PATCH 0967/1722] rename: S3DiskNoKeyErrors -> DiskS3NoSuchKeyErrors --- src/Common/CurrentMetrics.cpp | 2 +- src/IO/S3/Client.cpp | 4 ++-- tests/integration/test_checking_s3_blobs_paranoid/test.py | 2 +- tests/integration/test_storage_delta/test.py | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index b6dd14d292c..67890568941 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -307,7 +307,7 @@ M(FilteringMarksWithPrimaryKey, "Number of threads currently doing filtering of mark ranges by the primary key") \ M(FilteringMarksWithSecondaryKeys, "Number of threads currently doing filtering of mark ranges by secondary keys") \ \ - M(S3DiskNoKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \ + M(DiskS3NoSuchKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index a966e370ca1..8338a235387 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -46,7 +46,7 @@ namespace ProfileEvents namespace CurrentMetrics { - extern const Metric S3DiskNoKeyErrors; + extern const Metric DiskS3NoSuchKeyErrors; } namespace DB @@ -701,7 +701,7 @@ RequestResult Client::processRequestResult(RequestResult && outcome) const return std::forward(outcome); if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) - CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors); + CurrentMetrics::add(CurrentMetrics::DiskS3NoSuchKeyErrors); String enriched_message = fmt::format( "{} {}", diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 73f2888ce00..76a0f30f82e 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -708,7 +708,7 @@ def test_no_key_found_disk(cluster, broken_s3): """ SELECT value FROM system.metrics - WHERE metric = 'S3DiskNoKeyErrors' + WHERE metric = 'DiskS3NoSuchKeyErrors' """ ).strip() ) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 054b79ff6fe..a595d01e6b3 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -464,7 +464,7 @@ def test_restart_broken(started_cluster): """ SELECT value FROM system.metrics - WHERE metric = 'S3DiskNoKeyErrors' + WHERE metric = 'DiskS3NoSuchKeyErrors' """ ).strip() ) From 844cdd8937cce17060ea8b54fdfc2428d3015f44 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 14 Aug 2024 20:38:09 +0200 Subject: [PATCH 0968/1722] update toInterval functions --- .../functions/type-conversion-functions.md | 400 +++++++++++++++++- 1 file changed, 380 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 1e618b8cdab..cd6fd9ab839 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -4866,30 +4866,23 @@ Result: └───────┴───────────────┴──────┴──────────────┴──────────────┴──────────────────────┘ ``` -## toInterval(Year\|Quarter\|Month\|Week\|Day\|Hour\|Minute\|Second) +## toIntervalYear -Converts a Number type argument to an [Interval](../data-types/special-data-types/interval.md) data type. +Returns an interval of `n` years of data type [IntervalYear](../data-types/special-data-types/interval.md). **Syntax** ``` sql -toIntervalSecond(number) -toIntervalMinute(number) -toIntervalHour(number) -toIntervalDay(number) -toIntervalWeek(number) -toIntervalMonth(number) -toIntervalQuarter(number) -toIntervalYear(number) +toIntervalYear(n) ``` **Arguments** -- `number` — Duration of interval. Positive integer number. +- `n` — Number of years. Positive integer number. [Int*](../data-types/int-uint.md). **Returned values** -- The value in `Interval` data type. +- Interval of `n` years. [IntervalYear](../data-types/special-data-types/interval.md). **Example** @@ -4898,19 +4891,386 @@ Query: ``` sql WITH toDate('2019-01-01') AS date, - INTERVAL 1 WEEK AS interval_week, - toIntervalWeek(1) AS interval_to_week -SELECT - date + interval_week, - date + interval_to_week; + toIntervalYear(1) AS interval_to_year +SELECT date + interval_to_year ``` Result: ```response -┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┠-│ 2019-01-08 │ 2019-01-08 │ -└───────────────────────────┴──────────────────────────────┘ +┌─plus(date, interval_to_year)─┠+│ 2020-01-01 │ +└──────────────────────────────┘ +``` + +## toIntervalQuarter + +Returns an interval of `n` quarters of data type [IntervalQuarter](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalQuarter(n) +``` + +**Arguments** + +- `n` — Number of quarters. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` quarters. [IntervalQuarter](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalQuarter(1) AS interval_to_quarter +SELECT date + interval_to_quarter +``` + +Result: + +```response +┌─plus(date, interval_to_quarter)─┠+│ 2019-04-01 │ +└─────────────────────────────────┘ +``` + +## toIntervalMonth + +Returns an interval of `n` months of data type [IntervalMonth](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMonth(n) +``` + +**Arguments** + +- `n` — Number of m. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` months. [IntervalMonth](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalMonth(1) AS interval_to_month +SELECT date + interval_to_month +``` + +Result: + +```response +┌─plus(date, interval_to_month)─┠+│ 2019-02-01 │ +└───────────────────────────────┘ +``` + +## toIntervalWeek + +Returns an interval of `n` weeks of data type [IntervalWeek](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalWeek(n) +``` + +**Arguments** + +- `n` — Number of weeks. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` weeks. [IntervalWeek](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalWeek(1) AS interval_to_week +SELECT date + interval_to_week +``` + +Result: + +```response +┌─plus(date, interval_to_week)─┠+│ 2019-01-08 │ +└──────────────────────────────┘ +``` + +## toIntervalDay + +Returns an interval of `n` days of data type [IntervalDay](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalDay(n) +``` + +**Arguments** + +- `n` — Number of days. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` days. [IntervalDay](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalDay(5) AS interval_to_days +SELECT date + interval_to_days +``` + +Result: + +```response +┌─plus(date, interval_to_days)─┠+│ 2019-01-06 │ +└──────────────────────────────┘ +``` + +## toIntervalHour + +Returns an interval of `n` hours of data type [IntervalHour](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalHour(n) +``` + +**Arguments** + +- `n` — Number of hours. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` hours. [IntervalHour](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalHour(12) AS interval_to_hours +SELECT date + interval_to_hours +``` + +Result: + +```response +┌─plus(date, interval_to_hours)─┠+│ 2019-01-01 12:00:00 │ +└───────────────────────────────┘ +``` + +## toIntervalMinute + +Returns an interval of `n` minutes of data type [IntervalMinute](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMinute(n) +``` + +**Arguments** + +- `n` — Number of minutes. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` minutes. [IntervalMinute](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalMinute(12) AS interval_to_minutes +SELECT date + interval_to_minutes +``` + +Result: + +```response +┌─plus(date, interval_to_minutes)─┠+│ 2019-01-01 00:12:00 │ +└─────────────────────────────────┘ +``` + +## toIntervalSecond + +Returns an interval of `n` seconds of data type [IntervalSecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalSecond(n) +``` + +**Arguments** + +- `n` — Number of seconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` seconds. [IntervalSecond](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDate('2019-01-01') AS date, + toIntervalSecond(30) AS interval_to_seconds +SELECT date + interval_to_seconds +``` + +Result: + +```response +┌─plus(date, interval_to_seconds)─┠+│ 2019-01-01 00:00:30 │ +└─────────────────────────────────┘ +``` + +## toIntervalMillisecond + +Returns an interval of `n` milliseconds of data type [IntervalMillisecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMillisecond(n) +``` + +**Arguments** + +- `n` — Number of milliseconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` milliseconds. [IntervalMilliseconds](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDateTime('2019-01-01') AS date, + toIntervalMillisecond(30) AS interval_to_milliseconds +SELECT date + interval_to_milliseconds +``` + +Result: + +```response +┌─plus(date, interval_to_milliseconds)─┠+│ 2019-01-01 00:00:00.030 │ +└──────────────────────────────────────┘ +``` + +## toIntervalMicrosecond + +Returns an interval of `n` microseconds of data type [IntervalMicrosecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalMicrosecond(n) +``` + +**Arguments** + +- `n` — Number of microseconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` microseconds. [IntervalMicrosecond](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDateTime('2019-01-01') AS date, + toIntervalMicrosecond(30) AS interval_to_microseconds +SELECT date + interval_to_microseconds +``` + +Result: + +```response +┌─plus(date, interval_to_microseconds)─┠+│ 2019-01-01 00:00:00.000030 │ +└──────────────────────────────────────┘ +``` + +## toIntervalNanosecond + +Returns an interval of `n` nanoseconds of data type [IntervalNanosecond](../data-types/special-data-types/interval.md). + +**Syntax** + +``` sql +toIntervalNanosecond(n) +``` + +**Arguments** + +- `n` — Number of nanoseconds. Positive integer number. [Int*](../data-types/int-uint.md). + +**Returned values** + +- Interval of `n` nanoseconds. [IntervalNanosecond](../data-types/special-data-types/interval.md). + +**Example** + +Query: + +``` sql +WITH + toDateTime('2019-01-01') AS date, + toIntervalNanosecond(30) AS interval_to_nanoseconds +SELECT date + interval_to_nanoseconds +``` + +Result: + +```response +┌─plus(date, interval_to_nanoseconds)─┠+│ 2019-01-01 00:00:00.000000030 │ +└─────────────────────────────────────┘ ``` ## parseDateTime From 5e037e5ba852ebd1984d957f21a4925fea8de2ff Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 14 Aug 2024 20:45:50 +0200 Subject: [PATCH 0969/1722] CI: Minor fixes for changelog and release exceptions --- .github/workflows/create_release.yml | 1 + tests/ci/changelog.py | 2 -- tests/ci/create_release.py | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 73613c65266..eb16c25f604 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -101,6 +101,7 @@ jobs: --volume=".:/wd" --workdir="/wd" \ clickhouse/style-test \ ./tests/ci/changelog.py -v --debug-helpers \ + --gh-user-or-token ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} \ --jobs=5 \ --output="./docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index b7f73f22016..554ba339892 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -19,7 +19,6 @@ from env_helper import TEMP_PATH from git_helper import git_runner, is_shallow from github_helper import GitHub, PullRequest, PullRequests, Repository from s3_helper import S3Helper -from get_robot_token import get_best_robot_token from ci_utils import Shell from version_helper import ( FILE_WITH_VERSION_PATH, @@ -172,7 +171,6 @@ def parse_args() -> argparse.Namespace: parser.add_argument( "--gh-user-or-token", help="user name or GH token to authenticate", - default=get_best_robot_token(), ) parser.add_argument( "--gh-password", diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index b5ea61e1952..68268b033fe 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -484,7 +484,7 @@ class ReleaseInfo: ) else: if not dry_run: - assert not self.changelog_pr + assert not self.version_bump_pr self.prs_merged = res From 7251bc59f88293e789de7bfcd33f43511a9b04f5 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 14 Aug 2024 20:22:19 +0000 Subject: [PATCH 0970/1722] Fix tidy build --- src/Columns/ColumnObject.cpp | 6 +++--- src/Columns/ColumnObject.h | 6 +++--- src/DataTypes/DataTypesBinaryEncoding.cpp | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index d4f752cda95..b7194ef50e7 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -250,7 +250,7 @@ void ColumnObject::insertData(const char *, size_t) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertData is not supported for {}", getName()); } -ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(const std::string_view path) +ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(std::string_view path) { if (dynamic_paths.size() == max_dynamic_paths) return nullptr; @@ -263,7 +263,7 @@ ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(const std::string_view path return it_ptr->second; } -void ColumnObject::addNewDynamicPath(const std::string_view path) +void ColumnObject::addNewDynamicPath(std::string_view path) { if (!tryToAddNewDynamicPath(path)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add new dynamic path as the limit ({}) on dynamic paths is reached", max_dynamic_paths); @@ -605,7 +605,7 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co } } -void ColumnObject::serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const std::string_view path, const IColumn & column, size_t n) +void ColumnObject::serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, std::string_view path, const IColumn & column, size_t n) { /// Don't store Null values in shared data. We consider Null value equivalent to the absence /// of this path in the row because we cannot distinguish these 2 cases for dynamic paths. diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 8412a50281a..db31674daec 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -215,15 +215,15 @@ public: /// Try to add new dynamic path. Returns pointer to the new dynamic /// path column or nullptr if limit on dynamic paths is reached. - ColumnDynamic * tryToAddNewDynamicPath(const std::string_view path); + ColumnDynamic * tryToAddNewDynamicPath( std::string_view path); /// Throws an exception if cannot add. - void addNewDynamicPath(const std::string_view path); + void addNewDynamicPath(std::string_view path); void setDynamicPaths(const std::vector & paths); void setMaxDynamicPaths(size_t max_dynamic_paths_); void setStatistics(const StatisticsPtr & statistics_) { statistics = statistics_; } - void serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, const std::string_view path, const IColumn & column, size_t n); + void serializePathAndValueIntoSharedData(ColumnString * shared_data_paths, ColumnString * shared_data_values, std::string_view path, const IColumn & column, size_t n); void deserializeValueFromSharedData(const ColumnString * shared_data_values, size_t n, IColumn & column) const; /// Paths in shared data are sorted in each row. Use this method to find the lower bound for specific path in the row. diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index 5c32c21bbf3..dc0f2f3f5aa 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -100,7 +100,7 @@ enum class BinaryTypeIndex : uint8_t /// In future we can introduce more arguments in the JSON data type definition. /// To support such changes, use versioning in the serialization of JSON type. -static const UInt8 TYPE_JSON_SERIALIZATION_VERSION = 0; +const UInt8 TYPE_JSON_SERIALIZATION_VERSION = 0; BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) { From da4da136674e5b9e587d535742331d232639e11f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 14 Aug 2024 22:50:14 +0200 Subject: [PATCH 0971/1722] Fix style --- src/Columns/ColumnObject.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index db31674daec..f530ed29ef3 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -215,7 +215,7 @@ public: /// Try to add new dynamic path. Returns pointer to the new dynamic /// path column or nullptr if limit on dynamic paths is reached. - ColumnDynamic * tryToAddNewDynamicPath( std::string_view path); + ColumnDynamic * tryToAddNewDynamicPath(std::string_view path); /// Throws an exception if cannot add. void addNewDynamicPath(std::string_view path); From 3ca00440af8129b64c45cba47f212b8f897b9a54 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 20:51:23 +0000 Subject: [PATCH 0972/1722] Fix off-by-one inline function info in stack traces --- src/Common/StackTrace.cpp | 27 +++++++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 76277cbc993..80329b4aec1 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -248,8 +248,31 @@ void StackTrace::forEachFrame( auto dwarf_it = dwarfs.try_emplace(object->name, object->elf).first; DB::Dwarf::LocationInfo location; - if (dwarf_it->second.findAddress( - uintptr_t(current_frame.physical_addr), location, mode, inline_frames)) + uintptr_t adjusted_addr = uintptr_t(current_frame.physical_addr); + if (i > 0) + { + /// For non-innermost stack frames, the address points to the *next* instruction + /// after the `call` instruction. But we want the line number and inline function + /// information for the `call` instruction. So subtract 1 from the address. + /// Caveats: + /// * The `call` instruction can be longer than 1 byte, so addr-1 is in the middle + /// of the instruction. That's ok for debug info lookup: address ranges in debug + /// info cover the whole instruction. + /// * If the stack trace unwound out of a signal handler, the stack frame just + /// outside the signal didn't do a function call. It was interrupted by signal. + /// There's no `call` instruction, and decrementing the address is incorrect. + /// We may get incorrect line number and inlined functions in this case. + /// Unfortunate. + /// Note that libunwind, when producing this stack trace, knows whether this + /// frame is interrupted by signal or not. We could propagate this information + /// from libunwind to here and avoid subtracting 1 in this case, but currently + /// we don't do this. + /// But we don't do the decrement for findSymbol() below (because `call` is + /// ~never the last instruction of a function), so the function name should be + /// correct for both pre-signal frames and regular frames. + adjusted_addr -= 1; + } + if (dwarf_it->second.findAddress(adjusted_addr, location, mode, inline_frames)) { current_frame.file = location.file.toString(); current_frame.line = location.line; From aa38024b0e0cdc4a839446df2e7de974efc6b7e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 14 Aug 2024 20:59:08 +0000 Subject: [PATCH 0973/1722] Fix UBSan: lower upper bound for min_marks_for_concurrent_read --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 3ece7b1c5c8..734e67bda24 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -353,10 +353,11 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( /// We have a special logic for local replica. It has to read less data, because in some cases it should /// merge states of aggregate functions or do some other important stuff other than reading from Disk. auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; - if (pool_settings.min_marks_for_concurrent_read > std::numeric_limits::max()) + const auto min_marks_for_concurrent_read_limit = std::numeric_limits::max() >> 1; + if (pool_settings.min_marks_for_concurrent_read > min_marks_for_concurrent_read_limit) { /// limit min marks to read in case it's big, happened in test since due to settings randomzation - pool_settings.min_marks_for_concurrent_read = std::numeric_limits::max(); + pool_settings.min_marks_for_concurrent_read = min_marks_for_concurrent_read_limit; multiplier = 1.0f; } @@ -529,10 +530,11 @@ Pipe ReadFromMergeTree::readInOrder( }; auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; - if (pool_settings.min_marks_for_concurrent_read > std::numeric_limits::max()) + const auto min_marks_for_concurrent_read_limit = std::numeric_limits::max() >> 1; + if (pool_settings.min_marks_for_concurrent_read > min_marks_for_concurrent_read_limit) { /// limit min marks to read in case it's big, happened in test since due to settings randomzation - pool_settings.min_marks_for_concurrent_read = std::numeric_limits::max(); + pool_settings.min_marks_for_concurrent_read = min_marks_for_concurrent_read_limit; multiplier = 1.0f; } From 43f4a3665d0c2efa8a4b93dc8b39de0ca81985d1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 21:35:10 +0000 Subject: [PATCH 0974/1722] Check that setProcessListElement() is not called on global context --- src/Interpreters/Context.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4a08fd5fe5b..3cc09370e86 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2957,6 +2957,9 @@ ProgressCallback Context::getProgressCallback() const void Context::setProcessListElement(QueryStatusPtr elem) { + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have process list element"); + /// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed. process_list_elem = elem; has_process_list_elem = elem.get(); From d88299b94dc9563f494d6f1ad9374d1055ad13a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 14 Aug 2024 23:46:40 +0200 Subject: [PATCH 0975/1722] check that merge entries are valid --- src/Storages/MergeTree/MergeList.cpp | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 8fbb163384e..2d48ed4537a 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -10,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, const ContextPtr & context) : table_id{table_id_} , partition_id{future_part->part_info.partition_id} @@ -21,8 +26,15 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta , merge_type{future_part->merge_type} , merge_algorithm{MergeAlgorithm::Undecided} { + auto format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; + if (result_part_name != result_part_info.getPartNameV1()) + format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; + for (const auto & source_part : future_part->parts) { + if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); + source_part_names.emplace_back(source_part->name); source_part_paths.emplace_back(source_part->getDataPartStorage().getFullPath()); @@ -42,6 +54,9 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta part->partition.serializeText(part->storage, out, {}); } + if (is_mutation && future_part->parts.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}", future_part->parts.size(), result_part_info.getPartNameV1()); + thread_group = ThreadGroup::createForBackgroundProcess(context); } From 88fa8cb5bb52c7efd2b8e132236ee10ba0dd33d4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 15 Aug 2024 00:06:41 +0200 Subject: [PATCH 0976/1722] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 2d48ed4537a..1eb2e707194 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -32,7 +32,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta for (const auto & source_part : future_part->parts) { - if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) + if (!source_part->getParentPart() && !result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); source_part_names.emplace_back(source_part->name); From 9f44cd85925b3d691afd71a9ecb9ccc9ec3bf57a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 14 Aug 2024 22:19:17 +0000 Subject: [PATCH 0977/1722] fix typo --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6c9a29c5bb9..149f9a3e80b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,7 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, merge_preferred_step_execution_time_ms, 100, "Target time to execetion of one step of merge. Can be exceeded if one step takes longer time", 0) \ + M(Milliseconds, merge_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From b077f2cc9c11b01c443eb1ff976457965f7297ee Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Thu, 15 Aug 2024 02:31:10 +0200 Subject: [PATCH 0978/1722] performance comparison test for output_format_parquet_write_page_index setting added new performance comparison test for output_format_parquet_write_page_index setting --- tests/performance/parquet_read_with_index.xml | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 tests/performance/parquet_read_with_index.xml diff --git a/tests/performance/parquet_read_with_index.xml b/tests/performance/parquet_read_with_index.xml new file mode 100644 index 00000000000..1bb2d8eb4a2 --- /dev/null +++ b/tests/performance/parquet_read_with_index.xml @@ -0,0 +1,30 @@ + + + INSERT INTO FUNCTION file('test_pq_index', Parquet) SELECT * FROM generateRandom('int64_column Nullable(Int64), tuple_column Tuple(a Nullable(String), b Nullable(Float64), c Tuple(i UInt32, j UInt32)),array_tuple_column Array(Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64))), map_tuple_column Map(String, Tuple(a Nullable(String), b Nullable(Float64), c Nullable(Int64)))') limit 1000000 SETTINGS output_format_parquet_use_custom_encoder=false, output_format_parquet_write_page_index=true + + + + SELECT * FROM file('test_pq_index', Parquet, 'tuple_column Tuple(a Nullable(String))') Format Null + + + + SELECT tuple_column.a FROM file('test_pq_index', Parquet) Format Null + + + + SELECT tuple_column.a FROM file('test_pq_index', Parquet, 'tuple_column Tuple(a Nullable(String))') Format Null + + + + SELECT tuple_column.c.i FROM file('test_pq_index', Parquet) Format Null + + + + SELECT * FROM file('test_pq_index', Parquet, 'array_tuple_column Array (Tuple(a Nullable(String)))') Format Null + + + + SELECT * FROM file('test_pq_index', Parquet, 'map_tuple_column Map(String, Tuple(a Nullable(String)))') Format Null + + + From 690c4d0803366a3a6fe1887e5c01b35d80f501f9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 15 Aug 2024 09:04:22 +0800 Subject: [PATCH 0979/1722] update --- src/Processors/Transforms/WindowTransform.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index cae817380e0..bd11aa4cd28 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1157,14 +1157,7 @@ void WindowTransform::appendChunk(Chunk & chunk) // Initialize output columns. for (auto & ws : workspaces) { - if (ws.window_function_impl) - block.casted_columns.push_back(ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices)); - else - { - /// `castColumn` returns nullptr at default, so it's OK to put nullptr as a placeholder here - /// it should not be used in fact. - block.casted_columns.push_back(nullptr); - } + block.casted_columns.push_back(ws.window_function_impl ? ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices) : nullptr); block.output_columns.push_back(ws.aggregate_function->getResultType() ->createColumn()); From e6c03886f06ee19e3d677be073814154c1842ef6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 15 Aug 2024 03:55:12 +0200 Subject: [PATCH 0980/1722] Add a test for #57324 --- .../03223_analyzer_with_cube_fuzz.reference | 0 .../03223_analyzer_with_cube_fuzz.sql | 27 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.reference create mode 100644 tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql diff --git a/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.reference b/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql b/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql new file mode 100644 index 00000000000..c19d11fbe0c --- /dev/null +++ b/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree ORDER BY a; +CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree ORDER BY key; +insert into t1 Select number, number from numbers(100000); +insert into t2 Select number, number from numbers(100000); + + +SELECT + 1 * 1000.0001, + (count(1.) = -2147483647) AND (count(a) = 1.1920928955078125e-7) AND (count(val) = 1048577) AND (sum(val) = ((NULL * 1048576) / -9223372036854775807)) AND (sum(a) = ((9223372036854775806 * 10000000000.) / 1048575)) +FROM +( + SELECT + a, + val + FROM t1 + FULL OUTER JOIN t2 ON (t1.a = t2.key) OR (1 * inf) OR (t1.b = t2.key) +) +GROUP BY '65537' + WITH CUBE +FORMAT Null +SETTINGS max_block_size = 100, join_use_nulls = 1, max_execution_time = 1., max_result_rows = 0, max_result_bytes = 0; -- { serverError TIMEOUT_EXCEEDED } + +DROP TABLE t1; +DROP TABLE t2; From 7bebc448f30a026e41563553788f463243aa18fd Mon Sep 17 00:00:00 2001 From: megao Date: Thu, 15 Aug 2024 10:42:22 +0800 Subject: [PATCH 0981/1722] fix progress value of view_refreshview --- src/Storages/System/StorageSystemViewRefreshes.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 30539ed6b6a..97065f2b2fb 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -86,7 +86,8 @@ void StorageSystemViewRefreshes::fillData( res_columns[i++]->insert(refresh.exception_message); res_columns[i++]->insert(refresh.refresh_count); - res_columns[i++]->insert(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read); +// res_columns[i++]->insert(std::min(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read, static_cast(1))); + res_columns[i++]->insert(std::min(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read, 1.0)); res_columns[i++]->insert(refresh.progress.elapsed_ns / 1e9); res_columns[i++]->insert(refresh.progress.read_rows); res_columns[i++]->insert(refresh.progress.read_bytes); From c9bfff3934ba5257dbfd1d43294b27155c6aec30 Mon Sep 17 00:00:00 2001 From: megao Date: Thu, 15 Aug 2024 10:52:17 +0800 Subject: [PATCH 0982/1722] fix progress value of view_refreshview --- src/Storages/System/StorageSystemViewRefreshes.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 97065f2b2fb..3941c4c39c2 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -86,7 +86,6 @@ void StorageSystemViewRefreshes::fillData( res_columns[i++]->insert(refresh.exception_message); res_columns[i++]->insert(refresh.refresh_count); -// res_columns[i++]->insert(std::min(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read, static_cast(1))); res_columns[i++]->insert(std::min(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read, 1.0)); res_columns[i++]->insert(refresh.progress.elapsed_ns / 1e9); res_columns[i++]->insert(refresh.progress.read_rows); From d31b36ca50d8905b8eaac945c197c425d5ddb208 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 07:16:01 +0000 Subject: [PATCH 0983/1722] Rework integration flaky check as it was timeouting too fast as run as a single group --- tests/ci/integration_tests_runner.py | 98 +++++++++++++++++----------- 1 file changed, 60 insertions(+), 38 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 2b348be8b51..1513a8bee4f 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -784,40 +784,65 @@ class ClickhouseIntegrationTestsRunner: logging.info("Starting check with retries") final_retry = 0 logs = [] - tires_num = 1 if should_fail else FLAKY_TRIES_COUNT - for i in range(tires_num): - final_retry += 1 - logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.try_run_test_group( - repo_path, - "bugfix" if should_fail else "flaky", - tests_to_run, - 1, - 1, - FLAKY_REPEAT_COUNT, - ) - logs += log_paths - if counters["FAILED"]: - logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) - description_prefix = "Failed tests found: " - result_state = "failure" - if not should_fail: + counters = { + "ERROR": [], + "PASSED": [], + "FAILED": [], + "SKIPPED": [], + "BROKEN": [], + "NOT_FAILED": [], + } # type: Dict + tests_times = defaultdict(float) # type: Dict + tests_log_paths = defaultdict(list) + + for test_to_run in tests_to_run: + tries_num = 1 if should_fail else FLAKY_TRIES_COUNT + for i in range(tries_num): + final_retry += 1 + logging.info("Running tests for the %s time", i) + group_counters, group_test_times, log_paths = self.try_run_test_group( + repo_path, + "bugfix" if should_fail else "flaky", + [test_to_run], + 1, + 1, + FLAKY_REPEAT_COUNT, + ) + for counter, value in group_counters.items(): + logging.info( + "Tests from group %s stats, %s count %s", + test_to_run, + counter, + len(value), + ) + counters[counter] += value + + for test_name, test_time in group_test_times.items(): + tests_times[test_name] = test_time + tests_log_paths[test_name] = log_paths + if not should_fail and ( + group_counters["FAILED"] or group_counters["ERROR"] + ): + logging.info( + "Unexpected failure in group %s. Fail fast for current group", + test_to_run, + ) break - if counters["ERROR"]: - description_prefix = "Failed tests found: " - logging.info("Found error tests: %s", " ".join(counters["ERROR"])) - # NOTE "error" result state will restart the whole test task, - # so we use "failure" here - result_state = "failure" - if not should_fail: - break - logging.info("Try is OK, all tests passed, going to clear env") - clear_ip_tables_and_restart_daemons() - logging.info("And going to sleep for some time") - if time.time() - start > MAX_TIME_SECONDS: - logging.info("Timeout reached, going to finish flaky check") - break - time.sleep(5) + + if group_counters["FAILED"]: + logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) + description_prefix = "Failed tests found: " + result_state = "failure" + if group_counters["ERROR"]: + description_prefix = "Failed tests found: " + logging.info("Found error tests: %s", " ".join(counters["ERROR"])) + # NOTE "error" result state will restart the whole test task, + # so we use "failure" here + result_state = "failure" + logging.info("Try is OK, all tests passed, going to clear env") + clear_ip_tables_and_restart_daemons() + logging.info("And going to sleep for some time") + time.sleep(5) test_result = [] for state in ("ERROR", "FAILED", "PASSED", "SKIPPED"): @@ -828,13 +853,10 @@ class ClickhouseIntegrationTestsRunner: else: text_state = state test_result += [ - ( - c + " (✕" + str(final_retry) + ")", - text_state, - f"{tests_times[c]:.2f}", - ) + (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] + status_text = description_prefix + ", ".join( [ str(n).lower().replace("failed", "fail") + ": " + str(len(c)) From 0bb076a4d381fcc4e9827bebedbbe46ded9b9278 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 08:08:00 +0000 Subject: [PATCH 0984/1722] Improve schema inference of date times --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Formats/EscapingRuleUtils.cpp | 5 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Formats/SchemaInferenceUtils.cpp | 119 +++++--- src/IO/ReadHelpers.cpp | 29 +- src/IO/ReadHelpers.h | 77 +++-- src/IO/parseDateTimeBestEffort.cpp | 77 ++++- src/IO/parseDateTimeBestEffort.h | 8 + .../03222_date_time_inference.reference | 253 +++++++++++++++++ .../0_stateless/03222_date_time_inference.sql | 268 ++++++++++++++++++ 12 files changed, 761 insertions(+), 79 deletions(-) create mode 100644 tests/queries/0_stateless/03222_date_time_inference.reference create mode 100644 tests/queries/0_stateless/03222_date_time_inference.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0808e8eb49f..ad6cc89c5cd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1136,6 +1136,7 @@ class IColumn; M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ + M(Bool, input_format_try_infer_datetimes_only_datetime64, false, "When input_format_try_infer_datetimes is enabled, infer only DateTime64 but not DateTime types", 0) \ M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats (except JSON, where exponent numbers are always inferred)", 0) \ M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \ M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 20a8721c10e..b344a141a46 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -88,6 +88,7 @@ static std::initializer_list(have_dates) + static_cast(have_datetimes))); + bool have_datetimes = type_indexes.contains(TypeIndex::DateTime); + bool have_datetimes64 = type_indexes.contains(TypeIndex::DateTime64); + bool all_dates_or_datetimes = (type_indexes.size() == (static_cast(have_dates) + static_cast(have_datetimes) + static_cast(have_datetimes64))); - if (!all_dates_or_datetimes && (have_dates || have_datetimes)) + if (!all_dates_or_datetimes && (have_dates || have_datetimes || have_datetimes64)) { for (auto & type : data_types) { - if (isDate(type) || isDateTime64(type)) + if (isDate(type) || isDateTime(type) || isDateTime64(type)) type = std::make_shared(); } type_indexes.erase(TypeIndex::Date); type_indexes.erase(TypeIndex::DateTime); + type_indexes.erase(TypeIndex::DateTime64); type_indexes.insert(TypeIndex::String); return; } - if (have_dates && have_datetimes) + for (auto & type : data_types) { - for (auto & type : data_types) + if (isDate(type) && (have_datetimes || have_datetimes64)) { - if (isDate(type)) + if (have_datetimes64) type = std::make_shared(9); + else + type = std::make_shared(); + type_indexes.erase(TypeIndex::Date); + } + else if (isDateTime(type) && have_datetimes64) + { + type = std::make_shared(9); + type_indexes.erase(TypeIndex::DateTime); } - - type_indexes.erase(TypeIndex::Date); } } @@ -697,55 +705,87 @@ namespace bool tryInferDate(std::string_view field) { - if (field.empty()) + /// Minimum length of Date text representation is 8 (YYYY-M-D) and maximum is 10 (YYYY-MM-DD) + if (field.size() < 8 || field.size() > 10) return false; - ReadBufferFromString buf(field); - Float64 tmp_float; /// Check if it's just a number, and if so, don't try to infer Date from it, /// because we can interpret this number as a Date (for example 20000101 will be 2000-01-01) /// and it will lead to inferring Date instead of simple Int64/UInt64 in some cases. - if (tryReadFloatText(tmp_float, buf) && buf.eof()) - return false; - - buf.seek(0, SEEK_SET); /// Return position to the beginning - - DayNum tmp; - return tryReadDateText(tmp, buf) && buf.eof(); - } - - bool tryInferDateTime(std::string_view field, const FormatSettings & settings) - { - if (field.empty()) + if (std::all_of(field.begin(), field.end(), isNumericASCII)) return false; ReadBufferFromString buf(field); - Float64 tmp_float; + DayNum tmp; + return tryReadDateText(tmp, buf, DateLUT::instance(), /*allowed_delimiters=*/"-/:") && buf.eof(); + } + + DataTypePtr tryInferDateTimeOrDateTime64(std::string_view field, const FormatSettings & settings) + { + /// Don't try to infer DateTime if string is too long. + /// It's difficult to say what is the real maximum length of + /// DateTime we can parse using BestEffort approach. + /// 50 symbols is more or less valid limit for date times that makes sense. + if (field.empty() || field.size() > 50) + return nullptr; + + /// Check that we have at least one digit, don't infer datetime form strings like "Apr"/"May"/etc. + if (!std::any_of(field.begin(), field.end(), isNumericASCII)) + return nullptr; + /// Check if it's just a number, and if so, don't try to infer DateTime from it, /// because we can interpret this number as a timestamp and it will lead to - /// inferring DateTime instead of simple Int64/Float64 in some cases. + /// inferring DateTime instead of simple Int64 in some cases. + if (std::all_of(field.begin(), field.end(), isNumericASCII)) + return nullptr; + + ReadBufferFromString buf(field); + Float64 tmp_float; + /// Check if it's a float value, and if so, don't try to infer DateTime from it, + /// because it will lead to inferring DateTime instead of simple Float64 in some cases. if (tryReadFloatText(tmp_float, buf) && buf.eof()) - return false; + return nullptr; + + buf.seek(0, SEEK_SET); /// Return position to the beginning + if (!settings.try_infer_datetimes_only_datetime64) + { + time_t tmp; + switch (settings.date_time_input_format) + { + case FormatSettings::DateTimeInputFormat::Basic: + if (tryReadDateTimeText(tmp, buf, DateLUT::instance(), /*allowed_date_delimiters=*/"-/:", /*allowed_time_delimiters=*/":") && buf.eof()) + return std::make_shared(); + break; + case FormatSettings::DateTimeInputFormat::BestEffort: + if (tryParseDateTimeBestEffortStrict(tmp, buf, DateLUT::instance(), DateLUT::instance("UTC"), /*allowed_date_delimiters=*/"-/:") && buf.eof()) + return std::make_shared(); + break; + case FormatSettings::DateTimeInputFormat::BestEffortUS: + if (tryParseDateTimeBestEffortUSStrict(tmp, buf, DateLUT::instance(), DateLUT::instance("UTC"), /*allowed_date_delimiters=*/"-/:") && buf.eof()) + return std::make_shared(); + break; + } + } buf.seek(0, SEEK_SET); /// Return position to the beginning DateTime64 tmp; switch (settings.date_time_input_format) { case FormatSettings::DateTimeInputFormat::Basic: - if (tryReadDateTime64Text(tmp, 9, buf) && buf.eof()) - return true; + if (tryReadDateTime64Text(tmp, 9, buf, DateLUT::instance(), /*allowed_date_delimiters=*/"-/:", /*allowed_time_delimiters=*/":") && buf.eof()) + return std::make_shared(9); break; case FormatSettings::DateTimeInputFormat::BestEffort: - if (tryParseDateTime64BestEffort(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof()) - return true; + if (tryParseDateTime64BestEffortStrict(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC"), /*allowed_date_delimiters=*/"-/:") && buf.eof()) + return std::make_shared(9); break; case FormatSettings::DateTimeInputFormat::BestEffortUS: - if (tryParseDateTime64BestEffortUS(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC")) && buf.eof()) - return true; + if (tryParseDateTime64BestEffortUSStrict(tmp, 9, buf, DateLUT::instance(), DateLUT::instance("UTC"), /*allowed_date_delimiters=*/"-/:") && buf.eof()) + return std::make_shared(9); break; } - return false; + return nullptr; } template @@ -1439,8 +1479,11 @@ DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const Forma if (settings.try_infer_dates && tryInferDate(field)) return std::make_shared(); - if (settings.try_infer_datetimes && tryInferDateTime(field, settings)) - return std::make_shared(9); + if (settings.try_infer_datetimes) + { + if (auto type = tryInferDateTimeOrDateTime64(field, settings)) + return type; + } return nullptr; } diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 9559462e62b..48d788512e4 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1271,7 +1271,7 @@ template void readJSONArrayInto, void>(PaddedPODArray, bool>(PaddedPODArray & s, ReadBuffer & buf); template -ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf) +ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf, const char * allowed_delimiters) { static constexpr bool throw_exception = std::is_same_v; @@ -1318,6 +1318,9 @@ ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf) } else { + if (!isSymbolIn(*buf.position(), allowed_delimiters)) + return error(); + ++buf.position(); if (!append_digit(month)) @@ -1325,7 +1328,11 @@ ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf) append_digit(month); if (!buf.eof() && !isNumericASCII(*buf.position())) + { + if (!isSymbolIn(*buf.position(), allowed_delimiters)) + return error(); ++buf.position(); + } else return error(); @@ -1338,12 +1345,12 @@ ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf) return ReturnType(true); } -template void readDateTextFallback(LocalDate &, ReadBuffer &); -template bool readDateTextFallback(LocalDate &, ReadBuffer &); +template void readDateTextFallback(LocalDate &, ReadBuffer &, const char * allowed_delimiters); +template bool readDateTextFallback(LocalDate &, ReadBuffer &, const char * allowed_delimiters); template -ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut) +ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters, const char * allowed_time_delimiters) { static constexpr bool throw_exception = std::is_same_v; @@ -1413,6 +1420,9 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D if (!isNumericASCII(s[0]) || !isNumericASCII(s[1]) || !isNumericASCII(s[2]) || !isNumericASCII(s[3]) || !isNumericASCII(s[5]) || !isNumericASCII(s[6]) || !isNumericASCII(s[8]) || !isNumericASCII(s[9])) return false; + + if (!isSymbolIn(s[4], allowed_date_delimiters) || !isSymbolIn(s[7], allowed_date_delimiters)) + return false; } UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -1443,6 +1453,9 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D if (!isNumericASCII(s[0]) || !isNumericASCII(s[1]) || !isNumericASCII(s[3]) || !isNumericASCII(s[4]) || !isNumericASCII(s[6]) || !isNumericASCII(s[7])) return false; + + if (!isSymbolIn(s[2], allowed_time_delimiters) || !isSymbolIn(s[5], allowed_time_delimiters)) + return false; } hour = (s[0] - '0') * 10 + (s[1] - '0'); @@ -1488,10 +1501,10 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D return ReturnType(true); } -template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); -template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); -template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); -template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); +template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); +template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); +template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); +template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); template diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ffba4fafb5c..39e1cb12b5c 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -703,13 +703,28 @@ struct NullOutput }; template -ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf); +ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf, const char * allowed_delimiters); + +inline bool isSymbolIn(char symbol, const char * symbols) +{ + if (symbols == nullptr) + return true; + + const char * pos = symbols; + while (*pos) + { + if (*pos == symbol) + return true; + ++pos; + } + return false; +} /// In YYYY-MM-DD format. /// For convenience, Month and Day parts can have single digit instead of two digits. /// Any separators other than '-' are supported. template -inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) +inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf, const char * allowed_delimiters = nullptr) { static constexpr bool throw_exception = std::is_same_v; @@ -753,6 +768,9 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) } else { + if (!isSymbolIn(pos[-1], allowed_delimiters)) + return error(); + if (!isNumericASCII(pos[0])) return error(); @@ -768,6 +786,9 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) if (isNumericASCII(pos[-1]) || !isNumericASCII(pos[0])) return error(); + if (!isSymbolIn(pos[-1], allowed_delimiters)) + return error(); + day = pos[0] - '0'; if (isNumericASCII(pos[1])) { @@ -783,7 +804,7 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) return ReturnType(true); } else - return readDateTextFallback(date, buf); + return readDateTextFallback(date, buf, allowed_delimiters); } inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) @@ -797,15 +818,15 @@ inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) } template -inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut) +inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_delimiters = nullptr) { static constexpr bool throw_exception = std::is_same_v; LocalDate local_date; if constexpr (throw_exception) - readDateTextImpl(local_date, buf); - else if (!readDateTextImpl(local_date, buf)) + readDateTextImpl(local_date, buf, allowed_delimiters); + else if (!readDateTextImpl(local_date, buf, allowed_delimiters)) return false; ExtendedDayNum ret = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day()); @@ -814,15 +835,15 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLU } template -inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut) +inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_delimiters = nullptr) { static constexpr bool throw_exception = std::is_same_v; LocalDate local_date; if constexpr (throw_exception) - readDateTextImpl(local_date, buf); - else if (!readDateTextImpl(local_date, buf)) + readDateTextImpl(local_date, buf, allowed_delimiters); + else if (!readDateTextImpl(local_date, buf, allowed_delimiters)) return false; /// When the parameter is out of rule or out of range, Date32 uses 1925-01-01 as the default value (-DateLUT::instance().getDayNumOffsetEpoch(), -16436) and Date uses 1970-01-01. @@ -846,19 +867,19 @@ inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTI readDateTextImpl(date, buf, date_lut); } -inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf) +inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf, const char * allowed_delimiters = nullptr) { - return readDateTextImpl(date, buf); + return readDateTextImpl(date, buf, allowed_delimiters); } -inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_delimiters = nullptr) { - return readDateTextImpl(date, buf, time_zone); + return readDateTextImpl(date, buf, time_zone, allowed_delimiters); } -inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_delimiters = nullptr) { - return readDateTextImpl(date, buf, time_zone); + return readDateTextImpl(date, buf, time_zone, allowed_delimiters); } UUID parseUUID(std::span src); @@ -975,13 +996,13 @@ inline T parseFromString(std::string_view str) template -ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut); +ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr); /** In YYYY-MM-DD hh:mm:ss or YYYY-MM-DD format, according to specified time zone. * As an exception, also supported parsing of unix timestamp in form of decimal number. */ template -inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut) +inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) { static constexpr bool throw_exception = std::is_same_v; @@ -1014,6 +1035,9 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons if (!isNumericASCII(s[0]) || !isNumericASCII(s[1]) || !isNumericASCII(s[2]) || !isNumericASCII(s[3]) || !isNumericASCII(s[5]) || !isNumericASCII(s[6]) || !isNumericASCII(s[8]) || !isNumericASCII(s[9])) return ReturnType(false); + + if (!isSymbolIn(s[4], allowed_date_delimiters) || !isSymbolIn(s[7], allowed_date_delimiters)) + return ReturnType(false); } UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -1033,6 +1057,9 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons if (!isNumericASCII(s[11]) || !isNumericASCII(s[12]) || !isNumericASCII(s[14]) || !isNumericASCII(s[15]) || !isNumericASCII(s[17]) || !isNumericASCII(s[18])) return ReturnType(false); + + if (!isSymbolIn(s[13], allowed_time_delimiters) || !isSymbolIn(s[16], allowed_time_delimiters)) + return ReturnType(false); } hour = (s[11] - '0') * 10 + (s[12] - '0'); @@ -1057,11 +1084,11 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons return readIntTextImpl(datetime, buf); } else - return readDateTimeTextFallback(datetime, buf, date_lut); + return readDateTimeTextFallback(datetime, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); } template -inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut) +inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) { static constexpr bool throw_exception = std::is_same_v; @@ -1075,7 +1102,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re { try { - readDateTimeTextImpl(whole, buf, date_lut); + readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); } catch (const DB::Exception &) { @@ -1085,7 +1112,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } else { - auto ok = readDateTimeTextImpl(whole, buf, date_lut); + auto ok = readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); if (!ok && (buf.eof() || *buf.position() != '.')) return ReturnType(false); } @@ -1168,14 +1195,14 @@ inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer readDateTimeTextImpl(datetime64, scale, buf, date_lut); } -inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) { - return readDateTimeTextImpl(datetime, buf, time_zone); + return readDateTimeTextImpl(datetime, buf, time_zone, allowed_date_delimiters, allowed_time_delimiters); } -inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) { - return readDateTimeTextImpl(datetime64, scale, buf, date_lut); + return readDateTimeTextImpl(datetime64, scale, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); } inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index e046e837689..68122a37df6 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -82,13 +82,14 @@ struct DateTimeSubsecondPart UInt8 digits; }; -template +template ReturnType parseDateTimeBestEffortImpl( time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, - DateTimeSubsecondPart * fractional) + DateTimeSubsecondPart * fractional, + const char * allowed_date_delimiters = nullptr) { auto on_error = [&](int error_code [[maybe_unused]], FormatStringHelper fmt_string [[maybe_unused]], @@ -170,22 +171,36 @@ ReturnType parseDateTimeBestEffortImpl( fractional->digits = 3; readDecimalNumber<3>(fractional->value, digits + 10); } + else if constexpr (strict) + { + /// Fractional part is not allowed. + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected fractional part"); + } return ReturnType(true); } else if (num_digits == 10 && !year && !has_time) { + if (strict && month) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + /// This is unix timestamp. readDecimalNumber<10>(res, digits); return ReturnType(true); } else if (num_digits == 9 && !year && !has_time) { + if (strict && month) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + /// This is unix timestamp. readDecimalNumber<9>(res, digits); return ReturnType(true); } else if (num_digits == 14 && !year && !has_time) { + if (strict && month) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + /// This is YYYYMMDDhhmmss readDecimalNumber<4>(year, digits); readDecimalNumber<2>(month, digits + 4); @@ -197,6 +212,9 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 8 && !year) { + if (strict && month) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + /// This is YYYYMMDD readDecimalNumber<4>(year, digits); readDecimalNumber<2>(month, digits + 4); @@ -272,6 +290,9 @@ ReturnType parseDateTimeBestEffortImpl( else return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected number of decimal digits after year and month: {}", num_digits); } + + if (!isSymbolIn(delimiter_after_year, allowed_date_delimiters)) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: '{}' delimiter between date parts is not allowed", delimiter_after_year); } } else if (num_digits == 2 || num_digits == 1) @@ -403,9 +424,16 @@ ReturnType parseDateTimeBestEffortImpl( else { if (day_of_month) + { + if (strict && hour) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: hour component is duplicated"); + hour = hour_or_day_of_month_or_month; + } else + { day_of_month = hour_or_day_of_month_or_month; + } } } else if (num_digits != 0) @@ -446,6 +474,11 @@ ReturnType parseDateTimeBestEffortImpl( fractional->digits = num_digits; readDecimalNumber(fractional->value, num_digits, digits); } + else if (strict) + { + /// Fractional part is not allowed. + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected fractional part"); + } } else if (c == '+' || c == '-') { @@ -582,12 +615,24 @@ ReturnType parseDateTimeBestEffortImpl( return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: neither Date nor Time was parsed successfully"); if (!day_of_month) + { + if constexpr (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: day of month is required"); day_of_month = 1; + } + if (!month) + { + if constexpr (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month is required"); month = 1; + } if (!year) { + if constexpr (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: year is required"); + /// If year is not specified, it will be the current year if the date is unknown or not greater than today, /// otherwise it will be the previous year. /// This convoluted logic is needed to parse the syslog format, which looks as follows: "Mar 3 01:33:48". @@ -654,20 +699,20 @@ ReturnType parseDateTimeBestEffortImpl( return ReturnType(true); } -template -ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone) +template +ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters = nullptr) { time_t whole; DateTimeSubsecondPart subsecond = {0, 0}; // needs to be explicitly initialized sine it could be missing from input string if constexpr (std::is_same_v) { - if (!parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond)) + if (!parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond, allowed_date_delimiters)) return false; } else { - parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond); + parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond, allowed_date_delimiters); } @@ -730,4 +775,24 @@ bool tryParseDateTime64BestEffortUS(DateTime64 & res, UInt32 scale, ReadBuffer & return parseDateTime64BestEffortImpl(res, scale, in, local_time_zone, utc_time_zone); } +bool tryParseDateTimeBestEffortStrict(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters) +{ + return parseDateTimeBestEffortImpl(res, in, local_time_zone, utc_time_zone, nullptr, allowed_date_delimiters); +} + +bool tryParseDateTimeBestEffortUSStrict(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters) +{ + return parseDateTimeBestEffortImpl(res, in, local_time_zone, utc_time_zone, nullptr, allowed_date_delimiters); +} + +bool tryParseDateTime64BestEffortStrict(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters) +{ + return parseDateTime64BestEffortImpl(res, scale, in, local_time_zone, utc_time_zone, allowed_date_delimiters); +} + +bool tryParseDateTime64BestEffortUSStrict(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters) +{ + return parseDateTime64BestEffortImpl(res, scale, in, local_time_zone, utc_time_zone, allowed_date_delimiters); +} + } diff --git a/src/IO/parseDateTimeBestEffort.h b/src/IO/parseDateTimeBestEffort.h index 22af44f9e76..6dd052b67a3 100644 --- a/src/IO/parseDateTimeBestEffort.h +++ b/src/IO/parseDateTimeBestEffort.h @@ -63,4 +63,12 @@ void parseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, bool tryParseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone); void parseDateTime64BestEffortUS(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone); bool tryParseDateTime64BestEffortUS(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone); + +/// More strict version of best effort parsing. Requires day, month and year to be present, checks for allowed +/// delimiters between date components, makes additional correctness checks. Used in schema inference if date times. +bool tryParseDateTimeBestEffortStrict(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters); +bool tryParseDateTimeBestEffortUSStrict(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters); +bool tryParseDateTime64BestEffortStrict(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters); +bool tryParseDateTime64BestEffortUSStrict(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, const char * allowed_date_delimiters); + } diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference new file mode 100644 index 00000000000..3288308a1d0 --- /dev/null +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -0,0 +1,253 @@ +Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date +String +2020_01_01 String +2020_1_01 String +2020_01_1 String +2020_1_1 String +2020a01a01 String +2020a1a01 String +2020a01a1 String +2020a1a1 String +20200101 String +DateTime +2020-01-02 18:42:42 DateTime +2020-01-02 18:42:42 DateTime +2020-01-02 18:42:42 DateTime +String +2020_01_01 42:42:42 String +2020a01a01 42:42:42 String +2020-01-01 42.42.42 String +2020-01-01 42 42 42 String +2020-01-01 42a42a42 String +DateTime64 +2020-01-02 18:42:42.424200000 DateTime64(9) +2020-01-02 18:42:42.424200000 DateTime64(9) +2020-01-02 18:42:42.424200000 DateTime64(9) +String +2020_01_01 42:42:42.4242 String +2020a01a01 42:42:42.4242 String +2020-01-01 42.42.42.4242 String +2020-01-01 42 42 42.4242 String +2020-01-01 42a42a42.4242 String +DateTime/DateTime64 best effort +2000-01-01 00:00:00 DateTime +2000-01-01 01:00:00 DateTime +2000-01-01 01:00:00.000000000 DateTime64(9) +2017-01-01 22:02:03 DateTime +2017-01-01 22:02:03.000000000 DateTime64(9) +2017-01-01 21:02:03 DateTime +2017-01-01 21:02:03.000000000 DateTime64(9) +2017-01-01 22:02:03 DateTime +2017-01-01 22:02:03.000000000 DateTime64(9) +2017-01-02 01:02:03 DateTime +2017-01-02 01:02:03.000000000 DateTime64(9) +1970-01-02 01:02:03 DateTime +1970-01-02 01:02:03.000000000 DateTime64(9) +1970-01-02 01:02:03 DateTime +1970-01-02 01:02:03.000000000 DateTime64(9) +2018-02-11 03:40:50 DateTime +2018-02-11 03:40:50.000000000 DateTime64(9) +2000-04-17 01:02:03 DateTime +2000-04-17 01:02:03.000000000 DateTime64(9) +1970-01-02 01:00:00 DateTime +1970-01-02 01:00:00.000000000 DateTime64(9) +1970-01-02 01:02:03 DateTime +1970-01-02 01:02:03.000000000 DateTime64(9) +1970-01-02 01:02:03 DateTime +1970-01-02 01:02:03.000000000 DateTime64(9) +2015-12-31 20:00:00 DateTime +2015-12-31 20:00:00 DateTime +2016-01-01 00:00:00 DateTime +2016-01-01 00:00:00 DateTime +2017-01-01 22:02:03 DateTime +2017-01-01 22:02:03.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-02 04:04:05 DateTime +2017-01-02 04:04:05.000000000 DateTime64(9) +2017-01-02 02:34:05 DateTime +2017-01-02 02:34:05.000000000 DateTime64(9) +2017-01-02 00:04:05 DateTime +2017-01-02 00:04:05.000000000 DateTime64(9) +2017-01-02 02:04:05 DateTime +2017-01-02 02:04:05.000000000 DateTime64(9) +2017-01-02 00:04:05 DateTime +2017-01-02 00:04:05.000000000 DateTime64(9) +2017-01-01 18:04:05 DateTime +2017-01-01 18:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-01 23:04:05 DateTime +2017-01-01 23:04:05.000000000 DateTime64(9) +2017-02-01 23:04:05 DateTime +2017-02-01 23:04:05.000000000 DateTime64(9) +2017-06-01 23:04:05 DateTime +2017-06-01 23:04:05.000000000 DateTime64(9) +2017-01-02 00:04:05 DateTime +2017-01-02 00:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-01-02 04:04:05 DateTime +2017-01-02 04:04:05.000000000 DateTime64(9) +2017-01-02 04:04:05 DateTime +2017-01-02 04:04:05.000000000 DateTime64(9) +2017-01-02 02:04:05 DateTime +2017-01-02 02:04:05.000000000 DateTime64(9) +2017-01-02 03:04:05 DateTime +2017-01-02 03:04:05.000000000 DateTime64(9) +2017-04-01 11:22:33 DateTime +2017-04-01 11:22:33.000000000 DateTime64(9) +2017-04-01 22:02:03 DateTime +2017-04-01 22:02:03.000000000 DateTime64(9) +2017-04-01 22:02:03 DateTime +2017-04-01 22:02:03.000000000 DateTime64(9) +2017-04-02 01:02:03 DateTime +2017-04-02 01:02:03.000000000 DateTime64(9) +2017-04-02 11:22:33 DateTime +2017-04-02 11:22:33.000000000 DateTime64(9) +2017-04-02 01:02:03 DateTime +2017-04-02 01:02:03.000000000 DateTime64(9) +2017-04-02 01:22:33 DateTime +2017-04-02 01:22:33.000000000 DateTime64(9) +2017-04-02 01:02:03 DateTime +2017-04-02 01:02:03.000000000 DateTime64(9) +2017-04-02 01:02:33 DateTime +2017-04-02 01:02:33.000000000 DateTime64(9) +2017-04-01 22:02:03 DateTime +2017-04-01 22:02:03.000000000 DateTime64(9) +2017-04-02 01:02:03 DateTime +2017-04-02 01:02:03.000000000 DateTime64(9) +2017-04-01 22:02:03 DateTime +2017-04-01 22:02:03.000000000 DateTime64(9) +2017-04-01 21:02:03 DateTime +2017-04-01 21:02:03.000000000 DateTime64(9) +2017-04-02 01:02:03 DateTime +2017-04-02 01:02:03.000000000 DateTime64(9) +2017-01-01 22:02:03 DateTime +2017-01-01 22:02:03.000000000 DateTime64(9) +2017-04-25 01:02:03 DateTime +2017-04-25 01:02:03.000000000 DateTime64(9) +2017-04-25 01:02:03 DateTime +2017-04-25 01:02:03.000000000 DateTime64(9) +2017-01-25 01:02:03 DateTime +2017-01-25 01:02:03.000000000 DateTime64(9) +2017-01-24 22:02:03 DateTime +2017-01-24 22:02:03.000000000 DateTime64(9) +2017-01-25 13:02:03 DateTime +2017-01-25 13:02:03.000000000 DateTime64(9) +2017-01-25 01:02:03 DateTime +2017-01-25 01:02:03.000000000 DateTime64(9) +2017-01-25 01:02:03 DateTime +2017-01-25 01:02:03.000000000 DateTime64(9) +2017-01-24 22:02:03 DateTime +2017-01-24 22:02:03.000000000 DateTime64(9) +2017-01-24 22:02:03 DateTime +2017-01-24 22:02:03.000000000 DateTime64(9) +2017-01-25 10:02:03 DateTime +2017-01-25 10:02:03.000000000 DateTime64(9) +2017-01-25 10:02:03 DateTime +2017-01-25 10:02:03.000000000 DateTime64(9) +2017-01-25 10:02:03 DateTime +2017-01-25 10:02:03.000000000 DateTime64(9) +2017-01-25 09:32:03 DateTime +2017-01-25 09:32:03.000000000 DateTime64(9) +2017-01-25 01:02:03 DateTime +2017-01-25 01:02:03.000000000 DateTime64(9) +2017-01-25 13:02:03 DateTime +2017-01-25 13:02:03.000000000 DateTime64(9) +2017-01-25 13:02:03 DateTime +2017-01-25 13:02:03.000000000 DateTime64(9) +2017-01-25 10:02:03 DateTime +2017-01-25 10:02:03.000000000 DateTime64(9) +2018-02-11 03:40:50 DateTime +2018-02-11 03:40:50.000000000 DateTime64(9) +2018-02-11 03:40:50 DateTime +2018-02-11 03:40:50.000000000 DateTime64(9) +String +2 String +20 String +200 String +2000 String +20000 String +200001 String +2000010 String +20000101 String +200001010 String +2000010101 String +20000101010 String +200001010101 String +2000010101010 String +20000101010101 String +2.1 String +20.1 String +200.1 String +2000.1 String +20000.1 String +200001.1 String +2000010.1 String +20000101.1 String +200001010.1 String +2000010101.1 String +20000101010.1 String +200001010101.1 String +2000010101010.1 String +20000101010101.1 String +Mar String +Mar1 String +Mar 1 String +Mar01 String +Mar 01 String +Mar2020 String +Mar 2020 String +Mar012020 String +Mar 012020 String +Mar01012020 String +Mar 01012020 String +Mar0101202001 String +Mar 0101202001 String +Mar010120200101 String +Mar 010120200101 String +Mar01012020010101 String +Mar 01012020010101 String +Mar01012020010101.000 String +Mar 0101202001010101.000 String +2000 01 01 01:00:00 String +2000 01 01 01:00:00.000 String +2000a01a01 01:00:00 String +2000a01a01 01:00:00.000 String +2000-01-01 01 00 00 String +2000-01-01 01 00 00.000 String +2000-01-01 01-00-00 String +2000-01-01 01-00-00.000 String +2000-01-01 01a00a00 String +2000-01-01 01a00a00.000 String +2000-01 01:00:00 String +2000-01 01:00:00.000 String +2000 01 String +2000-01 String +Mar 2000 00:00:00 String +Mar 2000 00:00:00.000 String +2000 00:00:00 String +2000 00:00:00.000 String +Mar 2000-01-01 00:00:00 String +Mar 2000-01-01 00:00:00.000 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql new file mode 100644 index 00000000000..01266a88d55 --- /dev/null +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -0,0 +1,268 @@ +set input_format_try_infer_datetimes = 1; +set input_format_try_infer_dates = 1; +set schema_inference_make_columns_nullable = 0; +set input_format_json_try_infer_numbers_from_strings = 0; + +select 'Date'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:01:01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:1:01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:01:1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:1:1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-1-01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-1-1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020/01/01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020/1/01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020/01/1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020/1/1"}'); + +select 'String'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020_01_01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020_1_01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020_01_1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020_1_1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020a01a01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020a1a01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020a01a1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020a1a1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20200101"}'); + +select 'DateTime'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:01:01 42:42:42"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020/01/01 42:42:42"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42:42:42"}'); + +select 'String'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020_01_01 42:42:42"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020a01a01 42:42:42"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42.42.42"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42 42 42"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42a42a42"}'); + +select 'DateTime64'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:01:01 42:42:42.4242"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020/01/01 42:42:42.4242"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42:42:42.4242"}'); + +select 'String'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020_01_01 42:42:42.4242"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020a01a01 42:42:42.4242"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42.42.42.4242"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42 42 42.4242"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020-01-01 42a42a42.4242"}'); + +set date_time_input_format='best_effort'; +select 'DateTime/DateTime64 best effort'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 00:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203.000 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203 MSK+0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203.000 MSK+0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203.000 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/17 010203.000Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/1970 010203Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/1970 010203.000Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/70 010203Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "02/01/70 010203.000Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "11 Feb 2018 06:40:50 +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "11 Feb 2018 06:40:50.000 +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "17 Apr 2000 2 1:2:3"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "17 Apr 2000 2 1:2:3.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "19700102 01:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "19700102 01:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "19700102010203Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "19700102010203Z.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "1970/01/02 010203Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "1970/01/02 010203.000Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2016-01-01MSD"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2016-01-01 MSD"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2016-01-01UTC"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2016-01-01Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "201701 02 010203 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "201701 02 010203.000 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+0"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+0"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+0000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+0000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05 -0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000 -0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+030"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+030"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05+900"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000+900"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05GMT"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000GMT"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05 MSD"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000 MSD"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05 MSD Feb"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000 MSD Feb"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05 MSD Jun"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000 MSD Jun"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02 03:04:05.000 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05+00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05.000+00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05 -0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05.000 -0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05-0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05.000-0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05+0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05.000+0100"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017-01-02T03:04:05.000Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 01 11:22:33"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 01 11:22:33.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 010203 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 010203.000 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 01:2:3 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 01:2:3.000 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:02:3"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:02:3.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 11:22:33"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 11:22:33.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:03"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:03.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:22:33"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:22:33.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:33"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:33.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3.000 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3 UTC+0000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3.000 UTC+0000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3.000 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3 UTC+0400"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 02 1:2:3.000 UTC+0400"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 2 1:2:3"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Apr 2 1:2:3.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Jan 02 010203 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2017 Jan 02 010203.000 UTC+0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Apr 2017 01:02:03"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Apr 2017 01:02:03.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Apr 2017 1:2:3"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Apr 2017 1:2:3.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 MSK"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z+03:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z+03:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z +03:00 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z +03:00 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z +0300 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z +0300 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z+03:00 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z+03:00 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z +03:30 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z +03:30 PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3Z Mon"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000Z Mon"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3Z PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000Z PM"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3 Z PM +03:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "25 Jan 2017 1:2:3.000 Z PM +03:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 11 Feb 2018 06:40:50 +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 11 Feb 2018 06:40:50.000 +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun, 11 Feb 2018 06:40:50 +0300"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun, 11 Feb 2018 06:40:50.000 +0300"}'); + +select 'String'; +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000010"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200001010"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000101010"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200001010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000010101010"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000101010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200001.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000010.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000101.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200001010.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000010101.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000101010.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "200001010101.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000010101010.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "20000101010101.1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 0101202001010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000a01a01 01:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000a01a01 01:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01 00 00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01 00 00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01-00-00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01-00-00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01a00a00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01-01 01a00a00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01 01:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01 01:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000-01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000 00:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000 00:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00.000"}'); + + From c48b6d25f763d9c38b4e7ed35dec9a98d913e5c7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 08:58:36 +0000 Subject: [PATCH 0985/1722] more tests --- .../test_filesystem_layout/test.py | 4 ++++ .../integration/test_grant_and_revoke/test.py | 2 ++ .../test_parallel_replicas_failover/test.py | 19 ++++++++++--------- .../test.py | 4 ++-- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 4e719aa0fe9..31d6c830a2f 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -79,3 +79,7 @@ def test_file_path_escaping(started_cluster): "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin", ] ) + node.query("DROP TABLE test.`T.a_b,l-e!` SYNC") + node.query("DROP TABLE `test 2`.`T.a_b,l-e!` SYNC") + node.query("DROP DATABASE test") + node.query("DROP DATABASE `test 2`") diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index e533cced1e4..81cba966cae 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -359,6 +359,8 @@ def test_implicit_create_view_grant(): instance.query("GRANT CREATE VIEW ON test.* TO B", user="A") instance.query("CREATE VIEW test.view_2 AS SELECT 1", user="B") assert instance.query("SELECT * FROM test.view_2") == "1\n" + instance.query("DROP USER A") + instance.query("DROP VIEW test.view_2") def test_implicit_create_temporary_table_grant(): diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index bf25136bff7..2da26ee03c9 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -1,5 +1,5 @@ import pytest - +import uuid from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -25,19 +25,15 @@ def start_cluster(): def create_tables(cluster, table_name, skip_last_replica): - node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node2.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node2.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" ) if not skip_last_replica: node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data @@ -67,7 +63,7 @@ def test_skip_replicas_without_table(start_cluster): for i in range(4): expected_result += f"{i}\t1000\n" - log_comment = "5230b069-9574-407d-9b80-891b5a175f41" + log_comment = uuid.uuid4() assert ( node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", @@ -88,6 +84,8 @@ def test_skip_replicas_without_table(start_cluster): ) == "1\t1\n" ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") def test_skip_unresponsive_replicas(start_cluster): @@ -112,3 +110,6 @@ def test_skip_unresponsive_replicas(start_cluster): ) == expected_result ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") + node3.query(f"DROP TABLE {table_name} SYNC") diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index cab3fb46fe9..7093e3b3292 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -35,11 +35,10 @@ def start_cluster(): def _create_tables(table_name, table_size, index_granularity): - nodes[0].query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER {cluster_name}") nodes[0].query( f""" - CREATE TABLE IF NOT EXISTS {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) + CREATE TABLE {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}/', '{{replica}}') ORDER BY (key) SETTINGS index_granularity = {index_granularity}, max_bytes_to_merge_at_max_space_in_pool = 0, max_bytes_to_merge_at_max_space_in_pool = 1 @@ -128,3 +127,4 @@ def test_reading_with_invisible_parts( ) == f"{expected}\n" ) + nodes[0].query(f"DROP TABLE {table_name} ON CLUSTER {cluster_name} SYNC") From 8950491fa5af2f1abacbba86181f16fb512b8004 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 11:18:15 +0200 Subject: [PATCH 0986/1722] Fix unit test build --- src/Interpreters/tests/gtest_filecache.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 36acc319f4e..fd602ab5918 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -246,7 +246,8 @@ void download(FileSegment & file_segment) ASSERT_EQ(file_segment.state(), State::DOWNLOADING); ASSERT_EQ(file_segment.getDownloadedSize(), 0); - ASSERT_TRUE(file_segment.reserve(file_segment.range().size(), 1000)); + std::string failure_reason; + ASSERT_TRUE(file_segment.reserve(file_segment.range().size(), 1000, failure_reason)); download(cache_base_path, file_segment); ASSERT_EQ(file_segment.state(), State::DOWNLOADING); @@ -258,7 +259,8 @@ void assertDownloadFails(FileSegment & file_segment) { ASSERT_EQ(file_segment.getOrSetDownloader(), FileSegment::getCallerId()); ASSERT_EQ(file_segment.getDownloadedSize(), 0); - ASSERT_FALSE(file_segment.reserve(file_segment.range().size(), 1000)); + std::string failure_reason; + ASSERT_FALSE(file_segment.reserve(file_segment.range().size(), 1000, failure_reason)); file_segment.complete(); } @@ -957,10 +959,11 @@ TEST_F(FileCacheTest, temporaryData) { ASSERT_EQ(some_data_holder->size(), 5); + std::string failure_reason; for (auto & segment : *some_data_holder) { ASSERT_TRUE(segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segment->reserve(segment->range().size(), 1000)); + ASSERT_TRUE(segment->reserve(segment->range().size(), 1000, failure_reason)); download(*segment); segment->complete(); } From 3af8ba2deb99a0450c448d12a9c1da6858aec987 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 15 Aug 2024 11:21:20 +0200 Subject: [PATCH 0987/1722] Revert "[RFC] Fix settings/current_database in system.processes for async BACKUP/RESTORE" --- src/Backups/BackupsWorker.cpp | 4 ---- src/Interpreters/ProcessList.h | 3 --- 2 files changed, 7 deletions(-) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 8b45c816817..0b93ae6d547 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -490,8 +490,6 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context /// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously. auto process_list_element = context_in_use->getProcessListElement(); - /// Update context to preserve query information in processlist (settings, current_database) - process_list_element->updateContext(context_in_use); thread_pool.scheduleOrThrowOnError( [this, @@ -855,8 +853,6 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt /// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously. auto process_list_element = context_in_use->getProcessListElement(); - /// Update context to preserve query information in processlist (settings, current_database) - process_list_element->updateContext(context_in_use); thread_pool.scheduleOrThrowOnError( [this, diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 248ba947bc1..accb73e12df 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -244,9 +244,6 @@ public: /// Same as checkTimeLimit but it never throws [[nodiscard]] bool checkTimeLimitSoft(); - /// Use it in case of the query left in background to execute asynchronously - void updateContext(ContextWeakPtr weak_context) { context = std::move(weak_context); } - /// Get the reference for the start of the query. Used to synchronize with other Stopwatches UInt64 getQueryCPUStartTime() { return watch.getStart(); } }; From d18b6c63d408c4a70f3e541ce1956a4229bfe452 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 15 Aug 2024 09:41:03 +0000 Subject: [PATCH 0988/1722] Change name of default azurite container to avoid clashing with azure blob storage tests --- tests/integration/helpers/cluster.py | 2 +- tests/integration/test_storage_s3_queue/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0b6cf03d467..53f4f1e1f26 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -570,7 +570,7 @@ class ClickHouseCluster: self.spark_session = None self.with_azurite = False - self.azurite_container = "cont" + self.azurite_container = "azurite-container" self.blob_service_client = None self._azurite_port = 0 diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 34fb1eaf1fe..9e3ee19179a 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -232,7 +232,7 @@ def create_table( url = f"http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/" engine_def = f"{engine_name}('{url}', {auth_params}, {file_format})" else: - engine_def = f"{engine_name}('{started_cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', '{files_path}/', 'CSV')" + engine_def = f"{engine_name}('{started_cluster.env_variables['AZURITE_CONNECTION_STRING']}', '{started_cluster.azurite_container}', '{files_path}/', 'CSV')" node.query(f"DROP TABLE IF EXISTS {table_name}") create_query = f""" From 9f6e472b0cf03f8018b149ad5f7541b4ddec5735 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 15 Aug 2024 11:47:41 +0200 Subject: [PATCH 0989/1722] process regexp flags correctly --- src/Common/OptimizedRegularExpression.cpp | 40 ++++++++++++++--------- src/Common/tests/gtest_optimize_re.cpp | 2 ++ 2 files changed, 26 insertions(+), 16 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 712cab80aff..04e5f846adf 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -244,33 +244,41 @@ const char * analyzeImpl( is_trivial = false; if (!in_square_braces) { - /// Check for case-insensitive flag. - if (pos + 1 < end && pos[1] == '?') + /// it means flag negation + /// there are various possible flags + /// actually only imsU are supported by re2 + auto is_flag_char = [](char x) { - for (size_t offset = 2; pos + offset < end; ++offset) + return x == '-' || x == 'i' || x == 'm' || x == 's' || x == 'U' || x == 'u'; + }; + /// Check for case-insensitive flag. + if (pos + 2 < end && pos[1] == '?' && is_flag_char(pos[2])) + { + size_t offset = 2; + for (; pos + offset < end; ++offset) { - if (pos[offset] == '-' /// it means flag negation - /// various possible flags, actually only imsU are supported by re2 - || (pos[offset] >= 'a' && pos[offset] <= 'z') - || (pos[offset] >= 'A' && pos[offset] <= 'Z')) + if (pos[offset] == 'i') { - if (pos[offset] == 'i') - { - /// Actually it can be negated case-insensitive flag. But we don't care. - has_case_insensitive_flag = true; - break; - } + /// Actually it can be negated case-insensitive flag. But we don't care. + has_case_insensitive_flag = true; } - else + else if (!is_flag_char(pos[offset])) break; } + pos += offset; + /// if this group only contains flags, we have nothing to do. + if (*pos == ')') + { + ++pos; + break; + } } /// (?:regex) means non-capturing parentheses group - if (pos + 2 < end && pos[1] == '?' && pos[2] == ':') + else if (pos + 2 < end && pos[1] == '?' && pos[2] == ':') { pos += 2; } - if (pos + 3 < end && pos[1] == '?' && (pos[2] == '<' || pos[2] == '\'' || (pos[2] == 'P' && pos[3] == '<'))) + else if (pos + 3 < end && pos[1] == '?' && (pos[2] == '<' || pos[2] == '\'' || (pos[2] == 'P' && pos[3] == '<'))) { pos = skipNameCapturingGroup(pos, pos[2] == 'P' ? 3: 2, end); } diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index a9fcb918b24..0730a13f160 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -19,6 +19,8 @@ TEST(OptimizeRE, analyze) }; test_f("abc", "abc", {}, true, true); test_f("c([^k]*)de", ""); + test_f("(?-s)bob", "bob", {}, false, true); + test_f("(?s)bob", "bob", {}, false, true); test_f("abc(de)fg", "abcdefg", {}, false, true); test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}, false, true); test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}, false, true); From bea8e65f4fcfa99cdae21ff4776d509ba4fcd0d7 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 09:48:28 +0000 Subject: [PATCH 0990/1722] Fix tests --- src/IO/parseDateTimeBestEffort.cpp | 20 ++++++++++++++++--- ...ed_dates_in_csv_schema_inference.reference | 2 +- ...03033_dynamic_text_serialization.reference | 10 +++++----- .../03199_json_extract_dynamic.reference | 2 +- ...ad_for_schema_inference_in_cache.reference | 2 +- .../0_stateless/03222_date_time_inference.sql | 1 + 6 files changed, 26 insertions(+), 11 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 68122a37df6..f220577f2cb 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -82,7 +82,7 @@ struct DateTimeSubsecondPart UInt8 digits; }; -template +template ReturnType parseDateTimeBestEffortImpl( time_t & res, ReadBuffer & in, @@ -686,6 +686,20 @@ ReturnType parseDateTimeBestEffortImpl( } }; + if constexpr (strict) + { + if constexpr (is_64) + { + if (year < 1900) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime64: year {} is less than minimum supported year 1900", year); + } + else + { + if (year < 1970) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: year {} is less than minimum supported year 1970", year); + } + } + if (has_time_zone_offset) { res = utc_time_zone.makeDateTime(year, month, day_of_month, hour, minute, second); @@ -707,12 +721,12 @@ ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuf if constexpr (std::is_same_v) { - if (!parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond, allowed_date_delimiters)) + if (!parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond, allowed_date_delimiters)) return false; } else { - parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond, allowed_date_delimiters); + parseDateTimeBestEffortImpl(whole, in, local_time_zone, utc_time_zone, &subsecond, allowed_date_delimiters); } diff --git a/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference b/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference index be82d744a3b..56293ca0e5d 100644 --- a/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference +++ b/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference @@ -1 +1 @@ -c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime) diff --git a/tests/queries/0_stateless/03033_dynamic_text_serialization.reference b/tests/queries/0_stateless/03033_dynamic_text_serialization.reference index d965245266c..aa7b3fc83a2 100644 --- a/tests/queries/0_stateless/03033_dynamic_text_serialization.reference +++ b/tests/queries/0_stateless/03033_dynamic_text_serialization.reference @@ -4,7 +4,7 @@ JSON {"d":"str","dynamicType(d)":"String"} {"d":["1","2","3"],"dynamicType(d)":"Array(Int64)"} {"d":"2020-01-01","dynamicType(d)":"Date"} -{"d":"2020-01-01 10:00:00.000000000","dynamicType(d)":"DateTime64(9)"} +{"d":"2020-01-01 10:00:00","dynamicType(d)":"DateTime"} {"d":{"a":"42","b":"str"},"dynamicType(d)":"Tuple(a Int64, b String)"} {"d":{"a":"43"},"dynamicType(d)":"Tuple(a Int64)"} {"d":{"a":"44","c":["1","2","3"]},"dynamicType(d)":"Tuple(a Int64, c Array(Int64))"} @@ -22,7 +22,7 @@ CSV "str","String" "[1,2,3]","Array(Int64)" "2020-01-01","Date" -"2020-01-01 10:00:00.000000000","DateTime64(9)" +"2020-01-01 10:00:00","DateTime" "[1, 'str', [1, 2, 3]]","String" \N,"None" true,"Bool" @@ -32,18 +32,18 @@ TSV str String [1,2,3] Array(Int64) 2020-01-01 Date -2020-01-01 10:00:00.000000000 DateTime64(9) +2020-01-01 10:00:00 DateTime [1, \'str\', [1, 2, 3]] String \N None true Bool Values -(42,'Int64'),(42.42,'Float64'),('str','String'),([1,2,3],'Array(Int64)'),('2020-01-01','Date'),('2020-01-01 10:00:00.000000000','DateTime64(9)'),(NULL,'None'),(true,'Bool') +(42,'Int64'),(42.42,'Float64'),('str','String'),([1,2,3],'Array(Int64)'),('2020-01-01','Date'),('2020-01-01 10:00:00','DateTime'),(NULL,'None'),(true,'Bool') Cast using parsing 42 Int64 42.42 Float64 [1,2,3] Array(Int64) 2020-01-01 Date -2020-01-01 10:00:00.000000000 DateTime64(9) +2020-01-01 10:00:00 DateTime \N None true Bool 42 Int64 diff --git a/tests/queries/0_stateless/03199_json_extract_dynamic.reference b/tests/queries/0_stateless/03199_json_extract_dynamic.reference index 759b7763cd1..955106946ea 100644 --- a/tests/queries/0_stateless/03199_json_extract_dynamic.reference +++ b/tests/queries/0_stateless/03199_json_extract_dynamic.reference @@ -12,7 +12,7 @@ Hello String [1,2,3] Array(Nullable(Int64)) ['str1','str2','str3'] Array(Nullable(String)) [[[1],[2,3,4]],[[5,6],[7]]] Array(Array(Array(Nullable(Int64)))) -['2020-01-01 00:00:00.000000000','2020-01-01 00:00:00.000000000'] Array(Nullable(DateTime64(9))) +['2020-01-01 00:00:00','2020-01-01 00:00:00'] Array(Nullable(DateTime)) ['2020-01-01','2020-01-01 date'] Array(Nullable(String)) ['2020-01-01','2020-01-01 00:00:00','str'] Array(Nullable(String)) ['2020-01-01','2020-01-01 00:00:00','42'] Array(Nullable(String)) diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference index cd109daac52..13b1138d1c4 100644 --- a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference @@ -1,2 +1,2 @@ x Nullable(Int64) -schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=true, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false +schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=true, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_datetimes_only_datetime64=false, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index 01266a88d55..ebd472294be 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -2,6 +2,7 @@ set input_format_try_infer_datetimes = 1; set input_format_try_infer_dates = 1; set schema_inference_make_columns_nullable = 0; set input_format_json_try_infer_numbers_from_strings = 0; +set session_timezone = 'UTC'; select 'Date'; select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2020:01:01"}'); From b82c231886f2496c01b288a138663c4d430fc7b2 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Thu, 15 Aug 2024 10:37:13 +0000 Subject: [PATCH 0991/1722] Code review feedback - used removeNullable() --- src/Analyzer/ConstantNode.cpp | 11 +++-------- .../03222_datetime64_small_value_const.sql | 2 +- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 3d0f448da4b..3a99ad08ad8 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -177,15 +177,10 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const * It could also lead to ambiguous parsing because we don't know if the string literal represents a date or a Decimal64 literal. * For this reason, we use a string literal representing a date instead of a Decimal64 literal. */ - if ((WhichDataType(constant_value_type->getTypeId()).isDateTime64()) || - (WhichDataType(constant_value_type->getTypeId()).isNullable() && WhichDataType((typeid_cast(constant_value_type.get()))->getNestedType()->getTypeId()).isDateTime64())) + const auto & constant_value_end_type = removeNullable(constant_value_type); /// if Nullable + if (WhichDataType(constant_value_end_type->getTypeId()).isDateTime64()) { - const DataTypeDateTime64 * date_time_type = nullptr; - if (WhichDataType(constant_value_type->getTypeId()).isNullable()) - date_time_type = typeid_cast((typeid_cast(constant_value_type.get()))->getNestedType().get()); - else - date_time_type = typeid_cast(constant_value_type.get()); - + const auto * date_time_type = typeid_cast(constant_value_end_type.get()); DecimalField decimal_value; if (constant_value_literal.tryGet>(decimal_value)) { diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql index 6999ba9662a..af06a622f8d 100644 --- a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -17,7 +17,7 @@ create table shard_0.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree create table shard_1.dt64_03222(id UInt64, dt DateTime64(3)) engine = MergeTree order by id; create table distr_03222_dt64 (id UInt64, dt DateTime64(3)) engine = Distributed(test_cluster_two_shards_different_databases, '', dt64_03222); -insert into shard_0.dt64_03222 values(1, toDateTime64('1970-01-01 00:00:00.000',3)) +insert into shard_0.dt64_03222 values(1, toDateTime64('1970-01-01 00:00:00.000',3)); insert into shard_0.dt64_03222 values(2, toDateTime64('1970-01-01 00:00:02.456',3)); insert into shard_1.dt64_03222 values(3, toDateTime64('1970-01-01 00:00:04.811',3)); insert into shard_1.dt64_03222 values(4, toDateTime64('1970-01-01 00:10:05',3)); From ebe9f6de99d9473495d760cec6b1c0aab8fd5db9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 14 Aug 2024 15:03:55 +0000 Subject: [PATCH 0992/1722] Enable fp16, don't enable simsimd --- contrib/usearch-cmake/CMakeLists.txt | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 83221e3810f..df131e0c528 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -9,4 +9,14 @@ target_include_directories(_usearch SYSTEM INTERFACE ${SIMSIMD_PROJECT_DIR}/include ${USEARCH_PROJECT_DIR}/include) +target_compile_definitions(_usearch INTERFACE USEARCH_USE_FP16LIB) + +# target_compile_definitions(_usearch INTERFACE USEARCH_USE_SIMSIMD) +# ^^ simsimd is not enabled at the moment. Reasons: +# - Vectorization is important for raw scans but not so much for HNSW. We use usearch only for HNSW. +# - Simsimd does compile-time dispatch (choice of SIMD kernels determined by capabilities of the build machine) or dynamic dispatch (SIMD +# kernels chosen at runtime based on cpuid instruction). Since current builds are limited to SSE 4.2 (x86) and NEON (ARM), the speedup of +# the former would be moderate compared to AVX-512 / SVE. The latter is at the moment too fragile with respect to portability across x86 +# and ARM machines ... certain conbinations of quantizations / distance functions / SIMD instructions are not implemented at the moment. + add_library(ch_contrib::usearch ALIAS _usearch) From 6739320bdc72901f756dbef88e25f19435231531 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 14 Aug 2024 15:47:35 +0000 Subject: [PATCH 0993/1722] Don't call 'validate' ('make' calls it under the hood already) --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index fbbc66bd8db..77259b18bb9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -98,9 +98,6 @@ USearchIndexWithSerialization::USearchIndexWithSerialization( unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search); config.enable_key_lookups = false; /// we don't do row-to-vector lookups - if (auto error = config.validate(); error) /// already called in vectorSimilarityIndexValidator, call again because usearch may change the config in-place - throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release())); - if (auto result = USearchIndex::make(metric, config); !result) throw Exception(ErrorCodes::INCORRECT_DATA, "Could not create vector similarity index. Error: {}", String(result.error.release())); else From d4bbe2e9f086669e9d75d178299eb246353a36b4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 15 Aug 2024 09:32:17 +0000 Subject: [PATCH 0994/1722] Add tests for non-default quantization --- .../02354_vector_search_queries.reference | 55 +++++++++++++++++++ .../02354_vector_search_queries.sql | 51 +++++++++++++++++ 2 files changed, 106 insertions(+) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index 7c8e4c0ca59..5cd5ac006db 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -56,3 +56,58 @@ Expression (Projection) Condition: true Parts: 1/1 Granules: 4/4 +-- Non-default quantization +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_f32) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_f16) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_i8) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index dbf0fca32ab..486910ddbbb 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -78,3 +78,54 @@ LIMIT 3 SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann index DROP TABLE tab; + +SELECT '-- Non-default quantization'; +CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +INSERT INTO tab_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f32 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f32 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f16 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f16 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_i8 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_i8 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +DROP TABLE tab_f32; +DROP TABLE tab_f16; +DROP TABLE tab_i8; From b9548504d9c3c9dc2ce2e1010cb058e73e18b789 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 14 Aug 2024 15:44:22 +0000 Subject: [PATCH 0995/1722] Cosmetics --- .../MergeTreeIndexVectorSimilarity.cpp | 127 +++++++++--------- .../02354_vector_search_bugs.reference | 1 + .../0_stateless/02354_vector_search_bugs.sql | 6 + ...r_search_index_creation_negative.reference | 1 - ..._vector_search_index_creation_negative.sql | 5 - .../02354_vector_search_queries.reference | 8 +- .../02354_vector_search_queries.sql | 9 +- 7 files changed, 73 insertions(+), 84 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 77259b18bb9..b215aeae495 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -55,6 +55,7 @@ const std::unordered_map quantizationToSca {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; +/// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported accross all distance functions. template concept is_set = std::same_as>; @@ -253,8 +254,7 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ throw Exception( ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. Position: {}, Block rows: {}.", - *pos, - block.rows()); + *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); @@ -268,63 +268,64 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); const String & index_column_name = index_sample_block.getByPosition(0).name; - ColumnPtr column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); + const ColumnPtr & index_column = block.getByName(index_column_name).column; + ColumnPtr column_cut = index_column->cut(*pos, rows_read); - if (const auto & column_array = typeid_cast(column_cut.get())) + const auto * column_array = typeid_cast(column_cut.get()); + if (!column_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float32) column"); + + if (column_array->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); + + /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays + /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default + /// values which is also empty. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); + + const size_t rows = column_array->size(); + + const auto & column_array_offsets = column_array->getOffsets(); + const size_t dimensions = column_array_offsets[0]; + + if (!index) + index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); + + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. + if (index->dimensions() != dimensions) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); + + /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp + if (index->size() + rows > std::numeric_limits::max()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); + + const auto & column_array_data = column_array->getData(); + const auto & column_array_data_float = typeid_cast(column_array_data); + const auto & column_array_data_float_data = column_array_data_float.getData(); + + /// Check all sizes are the same + for (size_t row = 0; row < rows - 1; ++row) + if (column_array_offsets[row + 1] - column_array_offsets[row] != dimensions) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); + + /// Reserving space is mandatory + if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + rows))) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); + + for (size_t row = 0; row < rows; ++row) { - const auto & column_array_data = column_array->getData(); - const auto & column_array_data_float = typeid_cast(column_array_data); - const auto & column_array_data_float_data = column_array_data_float.getData(); - - const auto & column_array_offsets = column_array->getOffsets(); - const size_t num_rows = column_array_offsets.size(); - - if (column_array->empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); - - /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays - /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default - /// values which is also empty. - if (column_array->isDefaultAt(0)) - throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); - - /// Check all sizes are the same - const size_t dimensions = column_array_offsets[0]; - for (size_t i = 0; i < num_rows - 1; ++i) - if (column_array_offsets[i + 1] - column_array_offsets[i] != dimensions) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); - - /// Also check that previously inserted blocks have the same size as this block. - /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across - /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. - if (index && index->dimensions() != dimensions) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name); - - if (!index) - index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); - - /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp - if (index->size() + num_rows > std::numeric_limits::max()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index in column {} would exceed 4 billion entries", index_column_name); - - /// Reserving space is mandatory - if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows))) - throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); - - for (size_t row = 0; row < num_rows; ++row) + if (auto result = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release())); + else { - if (auto result = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result) - throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release())); - else - { - ProfileEvents::increment(ProfileEvents::USearchAddCount); - ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members); - ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances); - } + ProfileEvents::increment(ProfileEvents::USearchAddCount); + ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members); + ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances); } } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float32) column"); *pos += rows_read; } @@ -483,7 +484,7 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta if (!quantizationToScalarKind.contains(index.arguments[2].safeGet())) throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind)); - /// Call Usearche's own parameter validation method for HNSW-specific parameters + /// Call Usearch's own parameter validation method for HNSW-specific parameters UInt64 m = index.arguments[3].safeGet(); UInt64 ef_construction = index.arguments[4].safeGet(); UInt64 ef_search = index.arguments[5].safeGet(); @@ -498,18 +499,14 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column"); - /// Check data type of the indexed column: + /// Check that the data type is Array(Float32) DataTypePtr data_type = index.sample_block.getDataTypes()[0]; - if (const auto * data_type_array = typeid_cast(data_type.get())) - { - TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); - if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); - } - else - { + const auto * data_type_array = typeid_cast(data_type.get()); + if (!data_type_array) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); + TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); + if (!WhichDataType(nested_type_index).isFloat32()) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); - } } } diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.reference b/tests/queries/0_stateless/02354_vector_search_bugs.reference index a27b086e118..8da05c8a7c0 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.reference +++ b/tests/queries/0_stateless/02354_vector_search_bugs.reference @@ -1,3 +1,4 @@ +Rejects INSERTs of Arrays with different sizes Issue #52258: Empty Arrays or Arrays with default values are rejected It is possible to create parts with different Array vector sizes but there will be an error at query time Correctness of index with > 1 mark diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index 7c66b4b8e45..51e2e6ce2b7 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -7,6 +7,12 @@ SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make DROP TABLE IF EXISTS tab; +SELECT 'Rejects INSERTs of Arrays with different sizes'; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; +INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } +DROP TABLE tab; + SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected'; CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference index f18daa6e02e..5963f4b5834 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.reference @@ -5,4 +5,3 @@ Two or six index arguments 4nd argument (M), if given, must be UInt64 and > 1 Must be created on single column Must be created on Array(Float32) columns -Rejects INSERTs of Arrays with different sizes diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index de9d37e1000..b39b8d3e754 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -38,8 +38,3 @@ CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity('hn CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } - -SELECT 'Rejects INSERTs of Arrays with different sizes'; -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index 5cd5ac006db..374ed1f8abd 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -1,9 +1,7 @@ 10 rows, index_granularity = 8192, GRANULARITY = 1 million --> 1 granule, 1 indexed block -- ORDER-BY-type 5 [0,2] 0 6 [0,2.1] 0.09999990463256836 7 [0,2.2] 0.20000004768371582 -- ORDER-BY-type, EXPLAIN Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -20,11 +18,9 @@ Expression (Projection) Parts: 1/1 Granules: 1/1 12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexed block -- ORDER-BY-type 6 [0,2] 0 7 [0,2.1] 0.09999990463256836 8 [0,2.2] 0.20000004768371582 -- ORDER-BY-type, EXPLAIN Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -41,11 +37,11 @@ Expression (Projection) Parts: 1/1 Granules: 2/4 Special cases -- ORDER-BY-type +-- Non-default metric, M, ef_construction, ef_search 6 [1,9.3] 0.005731362878640178 1 [2,3.2] 0.15200169244542905 7 [5.5,4.7] 0.3503476876550442 -- Special case: setting "max_limit_for_ann_queries" +-- Setting "max_limit_for_ann_queries" Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 486910ddbbb..2c6a7f10776 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -14,14 +14,12 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [0.0, 2.0]), (6, [0.0, 2.1]), (7, [0.0, 2.2]), (8, [0.0, 2.3]), (9, [0.0, 2.4]); -SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -SELECT '- ORDER-BY-type, EXPLAIN'; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) @@ -37,14 +35,12 @@ SELECT '12 rows, index_granularity = 3, GRANULARITY = 2 --> 4 granules, 2 indexe CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]); -SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -SELECT '- ORDER-BY-type, EXPLAIN'; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) @@ -56,19 +52,18 @@ DROP TABLE tab; SELECT 'Special cases'; -- Not a systematic test, just to check that no bad things happen. --- Test with non-default metric, M, ef_construction, ef_search +SELECT '-- Non-default metric, M, ef_construction, ef_search'; CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99, 66) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -SELECT '- ORDER-BY-type'; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, cosineDistance(vec, reference_vec) FROM tab ORDER BY cosineDistance(vec, reference_vec) LIMIT 3; -SELECT '- Special case: setting "max_limit_for_ann_queries"'; +SELECT '-- Setting "max_limit_for_ann_queries"'; EXPLAIN indexes=1 WITH [0.0, 2.0] as reference_vec SELECT id, vec, cosineDistance(vec, reference_vec) From 38a2b0dcc7c0d84136f00a3d5a7a3afbf86cccbb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 15 Aug 2024 10:42:06 +0000 Subject: [PATCH 0996/1722] Allow Array(Float64) as type of underlying column --- .../mergetree-family/annindexes.md | 8 +- .../MergeTreeIndexVectorSimilarity.cpp | 79 ++++++++++++------- .../MergeTree/VectorSimilarityCondition.cpp | 4 +- .../MergeTree/VectorSimilarityCondition.h | 4 +- ..._vector_search_index_creation_negative.sql | 2 +- .../02354_vector_search_queries.reference | 4 + .../02354_vector_search_queries.sql | 12 +++ 7 files changed, 76 insertions(+), 37 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 097b0f5850a..1057ccb5fee 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -22,10 +22,10 @@ ORDER BY Distance(vectors, Point) LIMIT N ``` -`vectors` contains N-dimensional values of type [Array(Float32)](../../../sql-reference/data-types/array.md), for example embeddings. -Function `Distance` computes the distance between two vectors. Often, the Euclidean (L2) distance is chosen as distance function but [other -distance functions](/docs/en/sql-reference/functions/distance-functions.md) are also possible. `Point` is the reference point, e.g. `(0.17, -0.33, ...)`, and `N` limits the number of search results. +`vectors` contains N-dimensional values of type [Array(Float32)](../../../sql-reference/data-types/array.md) or Array(Float64), for example +embeddings. Function `Distance` computes the distance between two vectors. Often, the Euclidean (L2) distance is chosen as distance function +but [other distance functions](/docs/en/sql-reference/functions/distance-functions.md) are also possible. `Point` is the reference point, +e.g. `(0.17, 0.33, ...)`, and `N` limits the number of search results. This query returns the top-`N` closest points to the reference point. Parameter `N` limits the number of returned values which is useful for situations where `MaxDistance` is difficult to determine in advance. diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index b215aeae495..9376cdf7562 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -248,6 +248,40 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorVectorSimilarity::getGranuleAnd return granule; } +namespace +{ + +template +void updateImpl(const ColumnArray * column_array, const ColumnArray::Offsets & column_array_offsets, USearchIndexWithSerializationPtr & index, size_t dimensions, size_t rows) +{ + const auto & column_array_data = column_array->getData(); + const auto & column_array_data_float = typeid_cast(column_array_data); + const auto & column_array_data_float_data = column_array_data_float.getData(); + + /// Check all sizes are the same + for (size_t row = 0; row < rows - 1; ++row) + if (column_array_offsets[row + 1] - column_array_offsets[row] != dimensions) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); + + /// Reserving space is mandatory + if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + rows))) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); + + for (size_t row = 0; row < rows; ++row) + { + if (auto result = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result) + throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release())); + else + { + ProfileEvents::increment(ProfileEvents::USearchAddCount); + ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members); + ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances); + } + } +} + +} + void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_t * pos, size_t limit) { if (*pos >= block.rows()) @@ -273,7 +307,7 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ const auto * column_array = typeid_cast(column_cut.get()); if (!column_array) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float32) column"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float*) column"); if (column_array->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); @@ -302,30 +336,19 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ if (index->size() + rows > std::numeric_limits::max()) throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); - const auto & column_array_data = column_array->getData(); - const auto & column_array_data_float = typeid_cast(column_array_data); - const auto & column_array_data_float_data = column_array_data_float.getData(); + DataTypePtr data_type = block.getDataTypes()[0]; + const auto * data_type_array = typeid_cast(data_type.get()); + if (!data_type_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); + const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); - /// Check all sizes are the same - for (size_t row = 0; row < rows - 1; ++row) - if (column_array_offsets[row + 1] - column_array_offsets[row] != dimensions) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); + if (WhichDataType(nested_type_index).isFloat32()) + updateImpl(column_array, column_array_offsets, index, dimensions, rows); + else if (WhichDataType(nested_type_index).isFloat64()) + updateImpl(column_array, column_array_offsets, index, dimensions, rows); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); - /// Reserving space is mandatory - if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + rows))) - throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index"); - - for (size_t row = 0; row < rows; ++row) - { - if (auto result = index->add(static_cast(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result) - throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release())); - else - { - ProfileEvents::increment(ProfileEvents::USearchAddCount); - ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members); - ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances); - } - } *pos += rows_read; } @@ -373,7 +396,7 @@ std::vector MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer "does not match the dimension in the index ({})", vector_similarity_condition.getDimensions(), index->dimensions()); - const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); + const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); auto search_result = index->search(reference_vector.data(), limit); if (!search_result) @@ -499,14 +522,14 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Vector similarity indexes must be created on a single column"); - /// Check that the data type is Array(Float32) + /// Check that the data type is Array(Float*) DataTypePtr data_type = index.sample_block.getDataTypes()[0]; const auto * data_type_array = typeid_cast(data_type.get()); if (!data_type_array) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float*)"); TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); - if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float32)"); + if (!WhichDataType(nested_type_index).isFloat()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Vector similarity indexes can only be created on columns of type Array(Float*)"); } } diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.cpp b/src/Storages/MergeTree/VectorSimilarityCondition.cpp index 2e53b4ecb3a..c8f33857640 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.cpp +++ b/src/Storages/MergeTree/VectorSimilarityCondition.cpp @@ -24,7 +24,7 @@ namespace { template -void extractReferenceVectorFromLiteral(std::vector & reference_vector, Literal literal) +void extractReferenceVectorFromLiteral(std::vector & reference_vector, Literal literal) { Float64 float_element_of_reference_vector; Int64 int_element_of_reference_vector; @@ -72,7 +72,7 @@ UInt64 VectorSimilarityCondition::getLimit() const throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } -std::vector VectorSimilarityCondition::getReferenceVector() const +std::vector VectorSimilarityCondition::getReferenceVector() const { if (index_is_useful && query_information.has_value()) return query_information->reference_vector; diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.h b/src/Storages/MergeTree/VectorSimilarityCondition.h index fd339ed715d..2380f8f46b0 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.h +++ b/src/Storages/MergeTree/VectorSimilarityCondition.h @@ -60,7 +60,7 @@ public: L2 }; - std::vector reference_vector; + std::vector reference_vector; DistanceFunction distance_function; String column_name; UInt64 limit; @@ -70,7 +70,7 @@ public: /// Returns false if query can be speeded up by an ANN index, true otherwise. bool alwaysUnknownOrTrue(String distance_function) const; - std::vector getReferenceVector() const; + std::vector getReferenceVector() const; size_t getDimensions() const; String getColumnName() const; Info::DistanceFunction getDistanceFunction() const; diff --git a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql index b39b8d3e754..e8e6aaee1b2 100644 --- a/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql +++ b/tests/queries/0_stateless/02354_vector_search_index_creation_negative.sql @@ -35,6 +35,6 @@ SELECT 'Must be created on Array(Float32) columns'; SET allow_suspicious_low_cardinality_types = 1; CREATE TABLE tab(id Int32, vec UInt64, INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, vec Float32, INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vec Array(UInt64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, vec LowCardinality(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, vec Nullable(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index 374ed1f8abd..cb3a8c801b1 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -107,3 +107,7 @@ Expression (Projection) Description: vector_similarity GRANULARITY 2 Parts: 1/1 Granules: 2/4 +-- Index on Array(Float64) column +6 [0,2] 0 +7 [0,2.1] 0.10000000000000009 +8 [0,2.2] 0.20000000000000018 diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 2c6a7f10776..fbf8427d8fe 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -124,3 +124,15 @@ LIMIT 3; DROP TABLE tab_f32; DROP TABLE tab_f16; DROP TABLE tab_i8; + +SELECT '-- Index on Array(Float64) column'; +CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +INSERT INTO tab VALUES (0, [1.0, 0.0]), (1, [1.1, 0.0]), (2, [1.2, 0.0]), (3, [1.3, 0.0]), (4, [1.4, 0.0]), (5, [1.5, 0.0]), (6, [0.0, 2.0]), (7, [0.0, 2.1]), (8, [0.0, 2.2]), (9, [0.0, 2.3]), (10, [0.0, 2.4]), (11, [0.0, 2.5]); + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +DROP TABLE tab; From 0a10f0ceb3fab80e5dcfab5ebbebbdbfcdaff6c1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 11:27:12 +0000 Subject: [PATCH 0997/1722] Update tests --- .../02325_dates_schema_inference.reference | 58 +++++++++---------- tests/queries/0_stateless/02404_data.CSV | 10 ++++ .../0_stateless/02404_data.CSVWithNames | 11 ++++ .../0_stateless/02404_data.CustomSeparated | 10 ++++ .../0_stateless/02404_data.JSONCompactEachRow | 10 ++++ .../0_stateless/02404_data.JSONEachRow | 10 ++++ tests/queries/0_stateless/02404_data.TSKV | 10 ++++ tests/queries/0_stateless/02404_data.TSV | 10 ++++ .../0_stateless/02404_data.TSVWithNames | 11 ++++ tests/queries/0_stateless/02404_data.Values | 1 + ...ce_cache_respect_format_settings.reference | 18 +++--- 11 files changed, 121 insertions(+), 38 deletions(-) create mode 100644 tests/queries/0_stateless/02404_data.CSV create mode 100644 tests/queries/0_stateless/02404_data.CSVWithNames create mode 100644 tests/queries/0_stateless/02404_data.CustomSeparated create mode 100644 tests/queries/0_stateless/02404_data.JSONCompactEachRow create mode 100644 tests/queries/0_stateless/02404_data.JSONEachRow create mode 100644 tests/queries/0_stateless/02404_data.TSKV create mode 100644 tests/queries/0_stateless/02404_data.TSV create mode 100644 tests/queries/0_stateless/02404_data.TSVWithNames create mode 100644 tests/queries/0_stateless/02404_data.Values diff --git a/tests/queries/0_stateless/02325_dates_schema_inference.reference b/tests/queries/0_stateless/02325_dates_schema_inference.reference index c8eebd3262e..124f105220d 100644 --- a/tests/queries/0_stateless/02325_dates_schema_inference.reference +++ b/tests/queries/0_stateless/02325_dates_schema_inference.reference @@ -1,29 +1,29 @@ JSONEachRow x Nullable(Date) x Nullable(DateTime64(9)) -x Nullable(DateTime64(9)) +x Nullable(DateTime) x Array(Nullable(Date)) -x Array(Nullable(DateTime64(9))) -x Array(Nullable(DateTime64(9))) -x Tuple(\n date1 Nullable(DateTime64(9)),\n date2 Nullable(Date)) -x Array(Nullable(DateTime64(9))) -x Array(Nullable(DateTime64(9))) -x Nullable(DateTime64(9)) +x Array(Nullable(DateTime)) +x Array(Nullable(DateTime)) +x Tuple(\n date1 Nullable(DateTime),\n date2 Nullable(Date)) +x Array(Nullable(DateTime)) +x Array(Nullable(DateTime)) +x Nullable(DateTime) x Array(Nullable(String)) x Nullable(String) x Array(Nullable(String)) -x Tuple(\n key1 Array(Array(Nullable(DateTime64(9)))),\n key2 Array(Array(Nullable(String)))) +x Tuple(\n key1 Array(Array(Nullable(DateTime))),\n key2 Array(Array(Nullable(String)))) CSV c1 Nullable(Date) c1 Nullable(DateTime64(9)) -c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime) c1 Array(Nullable(Date)) -c1 Array(Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Map(String, Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Nullable(DateTime64(9)) +c1 Array(Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Map(String, Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Nullable(DateTime) c1 Array(Nullable(String)) c1 Nullable(String) c1 Array(Nullable(String)) @@ -31,14 +31,14 @@ c1 Map(String, Array(Array(Nullable(String)))) TSV c1 Nullable(Date) c1 Nullable(DateTime64(9)) -c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime) c1 Array(Nullable(Date)) -c1 Array(Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Map(String, Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Nullable(DateTime64(9)) +c1 Array(Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Map(String, Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Nullable(DateTime) c1 Array(Nullable(String)) c1 Nullable(String) c1 Array(Nullable(String)) @@ -46,14 +46,14 @@ c1 Map(String, Array(Array(Nullable(String)))) Values c1 Nullable(Date) c1 Nullable(DateTime64(9)) -c1 Nullable(DateTime64(9)) +c1 Nullable(DateTime) c1 Array(Nullable(Date)) -c1 Array(Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Map(String, Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Array(Nullable(DateTime64(9))) -c1 Nullable(DateTime64(9)) +c1 Array(Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Map(String, Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Array(Nullable(DateTime)) +c1 Nullable(DateTime) c1 Array(Nullable(String)) c1 Nullable(String) c1 Array(Nullable(String)) diff --git a/tests/queries/0_stateless/02404_data.CSV b/tests/queries/0_stateless/02404_data.CSV new file mode 100644 index 00000000000..2d8b5c8daa8 --- /dev/null +++ b/tests/queries/0_stateless/02404_data.CSV @@ -0,0 +1,10 @@ +0,"1970-01-01" +1,"1970-01-02" +2,"1970-01-03" +3,"1970-01-04" +4,"1970-01-05" +5,"1970-01-06" +6,"1970-01-07" +7,"1970-01-08" +8,"1970-01-09" +9,"1970-01-10" diff --git a/tests/queries/0_stateless/02404_data.CSVWithNames b/tests/queries/0_stateless/02404_data.CSVWithNames new file mode 100644 index 00000000000..34647008916 --- /dev/null +++ b/tests/queries/0_stateless/02404_data.CSVWithNames @@ -0,0 +1,11 @@ +"number","toDate(number)" +0,"1970-01-01" +1,"1970-01-02" +2,"1970-01-03" +3,"1970-01-04" +4,"1970-01-05" +5,"1970-01-06" +6,"1970-01-07" +7,"1970-01-08" +8,"1970-01-09" +9,"1970-01-10" diff --git a/tests/queries/0_stateless/02404_data.CustomSeparated b/tests/queries/0_stateless/02404_data.CustomSeparated new file mode 100644 index 00000000000..f3ae1663536 --- /dev/null +++ b/tests/queries/0_stateless/02404_data.CustomSeparated @@ -0,0 +1,10 @@ +0 1970-01-01 +1 1970-01-02 +2 1970-01-03 +3 1970-01-04 +4 1970-01-05 +5 1970-01-06 +6 1970-01-07 +7 1970-01-08 +8 1970-01-09 +9 1970-01-10 diff --git a/tests/queries/0_stateless/02404_data.JSONCompactEachRow b/tests/queries/0_stateless/02404_data.JSONCompactEachRow new file mode 100644 index 00000000000..de2e0986aab --- /dev/null +++ b/tests/queries/0_stateless/02404_data.JSONCompactEachRow @@ -0,0 +1,10 @@ +["0", "1970-01-01"] +["1", "1970-01-02"] +["2", "1970-01-03"] +["3", "1970-01-04"] +["4", "1970-01-05"] +["5", "1970-01-06"] +["6", "1970-01-07"] +["7", "1970-01-08"] +["8", "1970-01-09"] +["9", "1970-01-10"] diff --git a/tests/queries/0_stateless/02404_data.JSONEachRow b/tests/queries/0_stateless/02404_data.JSONEachRow new file mode 100644 index 00000000000..e77256ac7fc --- /dev/null +++ b/tests/queries/0_stateless/02404_data.JSONEachRow @@ -0,0 +1,10 @@ +{"number":"0","toDate(number)":"1970-01-01"} +{"number":"1","toDate(number)":"1970-01-02"} +{"number":"2","toDate(number)":"1970-01-03"} +{"number":"3","toDate(number)":"1970-01-04"} +{"number":"4","toDate(number)":"1970-01-05"} +{"number":"5","toDate(number)":"1970-01-06"} +{"number":"6","toDate(number)":"1970-01-07"} +{"number":"7","toDate(number)":"1970-01-08"} +{"number":"8","toDate(number)":"1970-01-09"} +{"number":"9","toDate(number)":"1970-01-10"} diff --git a/tests/queries/0_stateless/02404_data.TSKV b/tests/queries/0_stateless/02404_data.TSKV new file mode 100644 index 00000000000..70f7ad33c8b --- /dev/null +++ b/tests/queries/0_stateless/02404_data.TSKV @@ -0,0 +1,10 @@ +number=0 toDate(number)=1970-01-01 +number=1 toDate(number)=1970-01-02 +number=2 toDate(number)=1970-01-03 +number=3 toDate(number)=1970-01-04 +number=4 toDate(number)=1970-01-05 +number=5 toDate(number)=1970-01-06 +number=6 toDate(number)=1970-01-07 +number=7 toDate(number)=1970-01-08 +number=8 toDate(number)=1970-01-09 +number=9 toDate(number)=1970-01-10 diff --git a/tests/queries/0_stateless/02404_data.TSV b/tests/queries/0_stateless/02404_data.TSV new file mode 100644 index 00000000000..f3ae1663536 --- /dev/null +++ b/tests/queries/0_stateless/02404_data.TSV @@ -0,0 +1,10 @@ +0 1970-01-01 +1 1970-01-02 +2 1970-01-03 +3 1970-01-04 +4 1970-01-05 +5 1970-01-06 +6 1970-01-07 +7 1970-01-08 +8 1970-01-09 +9 1970-01-10 diff --git a/tests/queries/0_stateless/02404_data.TSVWithNames b/tests/queries/0_stateless/02404_data.TSVWithNames new file mode 100644 index 00000000000..23310234a8c --- /dev/null +++ b/tests/queries/0_stateless/02404_data.TSVWithNames @@ -0,0 +1,11 @@ +number toDate(number) +0 1970-01-01 +1 1970-01-02 +2 1970-01-03 +3 1970-01-04 +4 1970-01-05 +5 1970-01-06 +6 1970-01-07 +7 1970-01-08 +8 1970-01-09 +9 1970-01-10 diff --git a/tests/queries/0_stateless/02404_data.Values b/tests/queries/0_stateless/02404_data.Values new file mode 100644 index 00000000000..d9a621d7ec9 --- /dev/null +++ b/tests/queries/0_stateless/02404_data.Values @@ -0,0 +1 @@ +(0,'1970-01-01'),(1,'1970-01-02'),(2,'1970-01-03'),(3,'1970-01-04'),(4,'1970-01-05'),(5,'1970-01-06'),(6,'1970-01-07'),(7,'1970-01-08'),(8,'1970-01-09'),(9,'1970-01-10') \ No newline at end of file diff --git a/tests/queries/0_stateless/02404_schema_inference_cache_respect_format_settings.reference b/tests/queries/0_stateless/02404_schema_inference_cache_respect_format_settings.reference index 049603328d9..3d6b1021916 100644 --- a/tests/queries/0_stateless/02404_schema_inference_cache_respect_format_settings.reference +++ b/tests/queries/0_stateless/02404_schema_inference_cache_respect_format_settings.reference @@ -4,7 +4,7 @@ c2 Nullable(Date) c1 Nullable(Float64) c2 Nullable(Date) c1 Nullable(Int64) -c2 Nullable(DateTime64(9)) +c2 Nullable(DateTime) c1 UInt8 c2 Nullable(Date) 4 @@ -14,7 +14,7 @@ toDate(number) Nullable(Date) number Nullable(Float64) toDate(number) Nullable(Date) number Nullable(Int64) -toDate(number) Nullable(DateTime64(9)) +toDate(number) Nullable(DateTime) number Nullable(Int64) toDate(number) Nullable(Date) 4 @@ -24,7 +24,7 @@ c2 Nullable(Date) c1 Nullable(Float64) c2 Nullable(Date) c1 Nullable(Int64) -c2 Nullable(DateTime64(9)) +c2 Nullable(DateTime) c1 UInt8 c2 Nullable(Date) 4 @@ -34,7 +34,7 @@ toDate(number) Nullable(Date) number Nullable(Float64) toDate(number) Nullable(Date) number Nullable(Int64) -toDate(number) Nullable(DateTime64(9)) +toDate(number) Nullable(DateTime) number Nullable(Int64) toDate(number) Nullable(Date) 4 @@ -44,7 +44,7 @@ toDate(number) Nullable(Date) number Nullable(Float64) toDate(number) Nullable(Date) number Nullable(Int64) -toDate(number) Nullable(DateTime64(9)) +toDate(number) Nullable(DateTime) number Nullable(Int64) toDate(number) Nullable(Date) 4 @@ -54,7 +54,7 @@ c2 Nullable(Date) c1 Nullable(Float64) c2 Nullable(Date) c1 Nullable(Int64) -c2 Nullable(DateTime64(9)) +c2 Nullable(DateTime) c1 UInt8 c2 Nullable(Date) 4 @@ -64,7 +64,7 @@ toDate(number) Nullable(Date) number Nullable(Float64) toDate(number) Nullable(Date) number Nullable(Int64) -toDate(number) Nullable(DateTime64(9)) +toDate(number) Nullable(DateTime) number Nullable(Int64) toDate(number) Nullable(Date) 4 @@ -74,7 +74,7 @@ c2 Nullable(Date) c1 Nullable(Float64) c2 Nullable(Date) c1 Nullable(Int64) -c2 Nullable(DateTime64(9)) +c2 Nullable(DateTime) c1 UInt8 c2 Nullable(Date) 4 @@ -84,7 +84,7 @@ c2 Nullable(Date) c1 Nullable(Float64) c2 Nullable(Date) c1 Nullable(Int64) -c2 Nullable(DateTime64(9)) +c2 Nullable(DateTime) c1 UInt8 c2 Nullable(Date) 4 From 50a8cee0c5a4cd067cee2dc5584401b15283b3cd Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 11:39:04 +0000 Subject: [PATCH 0998/1722] Update docs --- docs/en/interfaces/schema-inference.md | 93 +++++++++++++++++--------- 1 file changed, 62 insertions(+), 31 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 05fae994cbe..4afba20d76c 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -359,13 +359,14 @@ DESC format(JSONEachRow, '{"int" : 42, "float" : 42.42, "string" : "Hello, World Dates, DateTimes: ```sql -DESC format(JSONEachRow, '{"date" : "2022-01-01", "datetime" : "2022-01-01 00:00:00"}') +DESC format(JSONEachRow, '{"date" : "2022-01-01", "datetime" : "2022-01-01 00:00:00", "datetime64" : "2022-01-01 00:00:00.000"}') ``` ```response -┌─name─────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠-│ date │ Nullable(Date) │ │ │ │ │ │ -│ datetime │ Nullable(DateTime64(9)) │ │ │ │ │ │ -└──────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +┌─name───────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ date │ Nullable(Date) │ │ │ │ │ │ +│ datetime │ Nullable(DateTime) │ │ │ │ │ │ +│ datetime64 │ Nullable(DateTime64(9)) │ │ │ │ │ │ +└────────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` Arrays: @@ -759,12 +760,13 @@ DESC format(CSV, 'Hello world!,World hello!') Dates, DateTimes: ```sql -DESC format(CSV, '"2020-01-01","2020-01-01 00:00:00"') +DESC format(CSV, '"2020-01-01","2020-01-01 00:00:00","2022-01-01 00:00:00.000"') ``` ```response ┌─name─┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠│ c1 │ Nullable(Date) │ │ │ │ │ │ -│ c2 │ Nullable(DateTime64(9)) │ │ │ │ │ │ +│ c2 │ Nullable(DateTime) │ │ │ │ │ │ +│ c3 │ Nullable(DateTime64(9)) │ │ │ │ │ │ └──────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` @@ -956,12 +958,13 @@ DESC format(TSKV, 'int=42 float=42.42 bool=true string=Hello,World!\n') Dates, DateTimes: ```sql -DESC format(TSV, '2020-01-01 2020-01-01 00:00:00') +DESC format(TSV, '2020-01-01 2020-01-01 00:00:00 2022-01-01 00:00:00.000') ``` ```response ┌─name─┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠│ c1 │ Nullable(Date) │ │ │ │ │ │ -│ c2 │ Nullable(DateTime64(9)) │ │ │ │ │ │ +│ c2 │ Nullable(DateTime) │ │ │ │ │ │ +│ c3 │ Nullable(DateTime64(9)) │ │ │ │ │ │ └──────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` @@ -1126,12 +1129,13 @@ DESC format(Values, $$(42, 42.42, true, 'Hello,World!')$$) Dates, DateTimes: ```sql -DESC format(Values, $$('2020-01-01', '2020-01-01 00:00:00')$$) -``` + DESC format(Values, $$('2020-01-01', '2020-01-01 00:00:00', '2022-01-01 00:00:00.000')$$) + ``` ```response ┌─name─┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠│ c1 │ Nullable(Date) │ │ │ │ │ │ -│ c2 │ Nullable(DateTime64(9)) │ │ │ │ │ │ +│ c2 │ Nullable(DateTime) │ │ │ │ │ │ +│ c3 │ Nullable(DateTime64(9)) │ │ │ │ │ │ └──────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` @@ -1504,8 +1508,8 @@ DESC format(JSONEachRow, $$ #### input_format_try_infer_datetimes -If enabled, ClickHouse will try to infer type `DateTime64` from string fields in schema inference for text formats. -If all fields from a column in sample data were successfully parsed as datetimes, the result type will be `DateTime64(9)`, +If enabled, ClickHouse will try to infer type `DateTime` or `DateTime64` from string fields in schema inference for text formats. +If all fields from a column in sample data were successfully parsed as datetimes, the result type will be `DateTime` or `DateTime64(9)` (if any datetime had fractional part), if at least one field was not parsed as datetime, the result type will be `String`. Enabled by default. @@ -1513,39 +1517,66 @@ Enabled by default. **Examples** ```sql -SET input_format_try_infer_datetimes = 0 +SET input_format_try_infer_datetimes = 0; DESC format(JSONEachRow, $$ - {"datetime" : "2021-01-01 00:00:00.000"} - {"datetime" : "2022-01-01 00:00:00.000"} + {"datetime" : "2021-01-01 00:00:00", "datetime64" : "2021-01-01 00:00:00.000"} + {"datetime" : "2022-01-01 00:00:00", "datetime64" : "2022-01-01 00:00:00.000"} $$) ``` ```response -┌─name─────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠-│ datetime │ Nullable(String) │ │ │ │ │ │ -└──────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +┌─name───────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ datetime │ Nullable(String) │ │ │ │ │ │ +│ datetime64 │ Nullable(String) │ │ │ │ │ │ +└────────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` ```sql -SET input_format_try_infer_datetimes = 1 +SET input_format_try_infer_datetimes = 1; DESC format(JSONEachRow, $$ - {"datetime" : "2021-01-01 00:00:00.000"} - {"datetime" : "2022-01-01 00:00:00.000"} + {"datetime" : "2021-01-01 00:00:00", "datetime64" : "2021-01-01 00:00:00.000"} + {"datetime" : "2022-01-01 00:00:00", "datetime64" : "2022-01-01 00:00:00.000"} $$) ``` ```response -┌─name─────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠-│ datetime │ Nullable(DateTime64(9)) │ │ │ │ │ │ -└──────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +┌─name───────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ datetime │ Nullable(DateTime) │ │ │ │ │ │ +│ datetime64 │ Nullable(DateTime64(9)) │ │ │ │ │ │ +└────────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` ```sql DESC format(JSONEachRow, $$ - {"datetime" : "2021-01-01 00:00:00.000"} - {"datetime" : "unknown"} + {"datetime" : "2021-01-01 00:00:00", "datetime64" : "2021-01-01 00:00:00.000"} + {"datetime" : "unknown", "datetime64" : "unknown"} $$) ``` ```response -┌─name─────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠-│ datetime │ Nullable(String) │ │ │ │ │ │ -└──────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +┌─name───────┬─type─────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ datetime │ Nullable(String) │ │ │ │ │ │ +│ datetime64 │ Nullable(String) │ │ │ │ │ │ +└────────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` + +#### input_format_try_infer_datetimes_only_datetime64 + +If enabled, ClickHouse will always infer `DateTime64(9)` when `input_format_try_infer_datetimes` is enabled even if datetime values don't contain fractional part. + +Disabled by default. + +**Examples** + +```sql +SET input_format_try_infer_datetimes = 1; +SET input_format_try_infer_datetimes_only_datetime64 = 1; +DESC format(JSONEachRow, $$ + {"datetime" : "2021-01-01 00:00:00", "datetime64" : "2021-01-01 00:00:00.000"} + {"datetime" : "2022-01-01 00:00:00", "datetime64" : "2022-01-01 00:00:00.000"} + $$) +``` + +```text +┌─name───────┬─type────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┠+│ datetime │ Nullable(DateTime64(9)) │ │ │ │ │ │ +│ datetime64 │ Nullable(DateTime64(9)) │ │ │ │ │ │ +└────────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` Note: Parsing datetimes during schema inference respect setting [date_time_input_format](/docs/en/operations/settings/settings-formats.md#date_time_input_format) From 65cbe94d545b51df22d069246dafdea69d693268 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 15 Aug 2024 14:05:33 +0200 Subject: [PATCH 0999/1722] Update DeltaLakeMetadata.cpp --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 89b48f08438..ffb987bfa82 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -332,7 +332,7 @@ struct DeltaLakeMetadataImpl WhichDataType which(check_type->getTypeId()); if (which.isStringOrFixedString()) return value; - else if (isBool(data_type)) + else if (isBool(check_type)) return parse(value); else if (which.isInt8()) return parse(value); From 7e006f658127c4efe4c53c0c79880a6b09955755 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 14:09:48 +0200 Subject: [PATCH 1000/1722] Update delta lake test --- .../DataLakes/DeltaLakeMetadata.cpp | 5 +-- tests/integration/test_storage_delta/test.py | 32 +++++++++---------- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c896a760597..30001a59a59 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -262,10 +262,11 @@ struct DeltaLakeMetadataImpl partition_name, file_schema.toNamesAndTypesDescription()); } + LOG_TEST(log, "Partition {} value is {} (data type: {}, file: {})", + partition_name, value, name_and_type->type->getName(), filename); + auto field = getFieldValue(value, name_and_type->type); current_partition_columns.emplace_back(*name_and_type, field); - - LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename); } } } diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index a595d01e6b3..75a4b6cc221 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -572,7 +572,7 @@ def test_partition_columns(started_cluster): "test" + str(i), datetime.strptime(f"2000-01-0{i}", "%Y-%m-%d"), i, - False, + False if i % 2 == 0 else True, ) ] df = spark.createDataFrame(data=data, schema=schema) @@ -622,15 +622,15 @@ def test_partition_columns(started_cluster): ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')""" ) assert ( - """1 test1 2000-01-01 1 false + """1 test1 2000-01-01 1 true 2 test2 2000-01-02 2 false -3 test3 2000-01-03 3 false +3 test3 2000-01-03 3 true 4 test4 2000-01-04 4 false -5 test5 2000-01-05 5 false +5 test5 2000-01-05 5 true 6 test6 2000-01-06 6 false -7 test7 2000-01-07 7 false +7 test7 2000-01-07 7 true 8 test8 2000-01-08 8 false -9 test9 2000-01-09 9 false""" +9 test9 2000-01-09 9 true""" == instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY b").strip() ) @@ -670,7 +670,7 @@ test9 2000-01-09 9""" "test" + str(i), datetime.strptime(f"2000-01-{i}", "%Y-%m-%d"), i, - False, + False if i % 2 == 0 else True, ) ] df = spark.createDataFrame(data=data, schema=schema) @@ -696,23 +696,23 @@ test9 2000-01-09 9""" assert result == num_rows * 2 assert ( - """1 test1 2000-01-01 1 false + """1 test1 2000-01-01 1 true 2 test2 2000-01-02 2 false -3 test3 2000-01-03 3 false +3 test3 2000-01-03 3 true 4 test4 2000-01-04 4 false -5 test5 2000-01-05 5 false +5 test5 2000-01-05 5 true 6 test6 2000-01-06 6 false -7 test7 2000-01-07 7 false +7 test7 2000-01-07 7 true 8 test8 2000-01-08 8 false -9 test9 2000-01-09 9 false +9 test9 2000-01-09 9 true 10 test10 2000-01-10 10 false -11 test11 2000-01-11 11 false +11 test11 2000-01-11 11 true 12 test12 2000-01-12 12 false -13 test13 2000-01-13 13 false +13 test13 2000-01-13 13 true 14 test14 2000-01-14 14 false -15 test15 2000-01-15 15 false +15 test15 2000-01-15 15 true 16 test16 2000-01-16 16 false -17 test17 2000-01-17 17 false +17 test17 2000-01-17 17 true 18 test18 2000-01-18 18 false""" == instance.query( f""" From df07b07cb9c89200f51ffed4fffd17cb2b0307e8 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 12:18:02 +0000 Subject: [PATCH 1001/1722] Add new setting --- contrib/NuRaft | 2 +- contrib/azure | 2 +- contrib/icu | 2 +- contrib/icudata | 2 +- contrib/libprotobuf-mutator | 2 +- contrib/librdkafka | 2 +- contrib/libunwind | 2 +- contrib/numactl | 1 + contrib/qpl | 2 +- contrib/rocksdb | 2 +- contrib/usearch | 2 +- contrib/zlib-ng | 2 +- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 3 ++- src/Storages/ObjectStorage/Local/Configuration.cpp | 2 +- src/Storages/ObjectStorage/Utils.cpp | 2 +- 16 files changed, 17 insertions(+), 14 deletions(-) create mode 160000 contrib/numactl diff --git a/contrib/NuRaft b/contrib/NuRaft index cb5dc3c906e..c2b0811f164 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit cb5dc3c906e80f253e9ce9535807caef827cc2e0 +Subproject commit c2b0811f164a7948208489562dab4f186eb305ce diff --git a/contrib/azure b/contrib/azure index ea3e19a7be0..67272b7ee0a 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ea3e19a7be08519134c643177d56c7484dfec884 +Subproject commit 67272b7ee0adff6b69921b26eb071ba1a353062c diff --git a/contrib/icu b/contrib/icu index a56dde820dc..7750081bda4 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit a56dde820dc35665a66f2e9ee8ba58e75049b668 +Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 diff --git a/contrib/icudata b/contrib/icudata index c8e717892a5..4904951339a 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit c8e717892a557b4d2852317c7d628aacc0a0e5ab +Subproject commit 4904951339a70b4814d2d3723436b20d079cb01b diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index a304ec48dcf..1f95f808306 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf +Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d diff --git a/contrib/librdkafka b/contrib/librdkafka index 2d2aab6f5b7..39d4ed49ccf 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 2d2aab6f5b79db1cfca15d7bf0dee75d00d82082 +Subproject commit 39d4ed49ccf3406e2bf825d5d7b0903b5a290782 diff --git a/contrib/libunwind b/contrib/libunwind index 8f28e64d158..a89d904befe 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 +Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 diff --git a/contrib/numactl b/contrib/numactl new file mode 160000 index 00000000000..8d13d63a05f --- /dev/null +++ b/contrib/numactl @@ -0,0 +1 @@ +Subproject commit 8d13d63a05f0c3cd88bf777cbb61541202b7da08 diff --git a/contrib/qpl b/contrib/qpl index d4715e0e798..c2ced94c53c 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit d4715e0e79896b85612158e135ee1a85f3b3e04d +Subproject commit c2ced94c53c1ee22191201a59878e9280bc9b9b8 diff --git a/contrib/rocksdb b/contrib/rocksdb index be366233921..5f003e4a22d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit be366233921293bd07a84dc4ea6991858665f202 +Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 diff --git a/contrib/usearch b/contrib/usearch index 955c6f9c11a..30810452bec 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f +Subproject commit 30810452bec5d3d3aa0931bb5d761e2f09aa6356 diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 50f0eae1a41..a2fbeffdc30 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 50f0eae1a411764cd6d1e85b3ce471438acd3c1c +Subproject commit a2fbeffdc30a8b0ce6d54ee31208e2688eac4c9f diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b19ef1000c1..acc02cc4b92 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -113,6 +113,7 @@ class IColumn; M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ + M(Bool, local_truncate_on_insert, false, "Enables or disables truncate before insert in local engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7c163750a80..7d1aab939f8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,7 +81,8 @@ static std::initializer_listgetSettingsRef(); return StorageObjectStorage::QuerySettings{ - .truncate_on_insert = settings.engine_file_truncate_on_insert, + .truncate_on_insert = settings.local_truncate_on_insert, .create_new_file_on_insert = settings.local_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 6491deef440..3836e2c7ec8 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -42,7 +42,7 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", configuration.getNamespace(), key, - configuration.getTypeName() == "local" ? "engine_file" : configuration.getTypeName(), + configuration.getTypeName(), configuration.getTypeName()); } From 03bfb1562b56d96963e75bbd14b6759ae103e52a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 15 Aug 2024 14:26:01 +0200 Subject: [PATCH 1002/1722] fix overflow --- src/Common/OptimizedRegularExpression.cpp | 2 ++ src/Common/tests/gtest_optimize_re.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 04e5f846adf..2cdb3409487 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -266,6 +266,8 @@ const char * analyzeImpl( break; } pos += offset; + if (pos == end) + return pos; /// if this group only contains flags, we have nothing to do. if (*pos == ')') { diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index 0730a13f160..d6735c3ccfe 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -21,6 +21,7 @@ TEST(OptimizeRE, analyze) test_f("c([^k]*)de", ""); test_f("(?-s)bob", "bob", {}, false, true); test_f("(?s)bob", "bob", {}, false, true); + test_f("(?ssss", ""); test_f("abc(de)fg", "abcdefg", {}, false, true); test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}, false, true); test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}, false, true); From 7acf74437f26f194fd17a6f7d5dcc6a422dde722 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 15 Aug 2024 12:29:59 +0000 Subject: [PATCH 1003/1722] Fix spelling --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 9376cdf7562..27bfcbbddcf 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -55,7 +55,7 @@ const std::unordered_map quantizationToSca {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; -/// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported accross all distance functions. +/// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported across all distance functions. template concept is_set = std::same_as>; From 657bbce23f6d764dc0172f9de3d6bc7fcd06fe10 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 14:38:20 +0200 Subject: [PATCH 1004/1722] Add a test --- tests/integration/test_mask_sensitive_info/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 6f6dc4d287f..8d5345082ff 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -202,6 +202,10 @@ def test_create_table(): f"S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip') settings mode = 'ordered'", f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV') settings mode = 'ordered'", f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV', 'gzip') settings mode = 'ordered'", + ( + f"Iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + "DNS_ERROR", + ), ] def make_test_case(i): @@ -266,6 +270,7 @@ def test_create_table(): # due to sensitive data substituion the query will be normalized, so not "settings" but "SETTINGS" "CREATE TABLE table19 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV') SETTINGS mode = 'ordered'", "CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip') SETTINGS mode = 'ordered'", + "CREATE TABLE table21 (`x` int) ENGINE = Iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], must_not_contain=[password], ) @@ -387,6 +392,7 @@ def test_table_functions(): f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_15.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", f"azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')", f"azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_17.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + f"iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", ] def make_test_case(i): @@ -478,6 +484,7 @@ def test_table_functions(): f"CREATE TABLE tablefunc48 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_15.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", f"CREATE TABLE tablefunc49 (x int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')", f"CREATE TABLE tablefunc50 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_17.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc51 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], must_not_contain=[password], ) From c8e185296918b3bccad35a567139d0cff7f42f22 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 15 Aug 2024 12:40:27 +0000 Subject: [PATCH 1005/1722] Revert "Follow-up to #63898, pt. II" This reverts commit e2e9ae776d0e0795a565fa7ed6a29671fcda377e. --- .../0_stateless/00366_multi_statements.sh | 20 +++--- .../00443_preferred_block_size_bytes.sh | 4 +- ...ess_to_temporary_table_in_readonly_mode.sh | 20 +++--- ...4_performance_introspection_and_logging.sh | 4 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 6 +- ...d_optimize_skip_select_on_unused_shards.sh | 30 ++++----- ...p_select_on_unused_shards_with_prewhere.sh | 26 ++++---- ...ated_minimalistic_part_header_zookeeper.sh | 4 +- .../queries/0_stateless/00837_minmax_index.sh | 2 +- .../queries/0_stateless/00838_unique_index.sh | 2 +- .../0_stateless/00907_set_index_max_rows.sh | 2 +- .../0_stateless/00908_bloom_filter_index.sh | 10 +-- .../queries/0_stateless/00942_mutate_index.sh | 2 +- .../0_stateless/00943_materialize_index.sh | 4 +- .../00944_clear_index_in_partition.sh | 2 +- .../00964_bloom_index_string_functions.sh | 2 +- .../00965_set_index_string_functions.sh | 2 +- .../00974_primary_key_for_lowCardinality.sh | 4 +- tests/queries/0_stateless/00990_hasToken.sh | 2 +- .../01013_sync_replica_timeout_zookeeper.sh | 6 +- ...th_nondeterministic_functions_zookeeper.sh | 4 +- .../01037_polygon_dicts_correctness_all.sh | 6 +- .../01037_polygon_dicts_correctness_fast.sh | 6 +- .../01037_polygon_dicts_simple_functions.sh | 6 +- .../01055_minmax_index_compact_parts.sh | 2 +- .../01077_mutations_index_consistency.sh | 10 +-- .../01187_set_profile_as_setting.sh | 8 +-- ..._block_size_rows_for_materialized_views.sh | 2 +- .../01307_multiple_leaders_zookeeper.sh | 10 +-- .../0_stateless/01415_sticking_mutations.sh | 2 +- tests/queries/0_stateless/01451_dist_logs.sh | 2 +- .../01459_manual_write_to_replicas.sh | 4 +- .../01459_manual_write_to_replicas_quorum.sh | 4 +- ..._write_to_replicas_quorum_detach_attach.sh | 4 +- ...house_server_start_with_embedded_config.sh | 2 +- .../0_stateless/01508_format_regexp_raw.sh | 4 +- .../01509_dictionary_preallocate.sh | 2 +- ...01510_format_regexp_raw_low_cardinality.sh | 4 +- .../0_stateless/01526_initial_query_id.sh | 2 +- .../01599_mutation_query_params.sh | 2 +- .../01600_quota_by_forwarded_ip.sh | 4 +- .../01684_ssd_cache_dictionary_simple_key.sh | 2 +- .../01685_ssd_cache_dictionary_complex_key.sh | 2 +- .../01691_parser_data_type_exponential.sh | 2 +- ...ojections_optimize_aggregation_in_order.sh | 2 +- ...s_partial_optimize_aggregation_in_order.sh | 2 +- .../01753_optimize_aggregation_in_order.sh | 2 +- .../01758_optimize_skip_unused_shards_once.sh | 2 +- ...91_dist_INSERT_block_structure_mismatch.sh | 2 +- .../01814_distributed_push_down_limit.sh | 6 +- .../01853_dictionary_cache_duplicates.sh | 8 +-- .../01872_initial_query_start_time.sh | 2 +- ...75_ssd_cache_dictionary_decimal256_type.sh | 2 +- .../01890_materialized_distributed_join.sh | 2 +- .../01903_ssd_cache_dictionary_array_type.sh | 2 +- ..._cache_dictionary_default_nullable_type.sh | 2 +- ...1927_query_views_log_matview_exceptions.sh | 4 +- .../0_stateless/01947_multiple_pipe_read.sh | 4 +- .../02003_memory_limit_in_client.sh | 32 ++++----- .../02021_create_database_with_comment.sh | 2 +- .../02050_client_profile_events.sh | 8 +-- .../02221_parallel_replicas_bug.sh | 2 +- .../02221_system_zookeeper_unrestricted.sh | 4 +- ...2221_system_zookeeper_unrestricted_like.sh | 4 +- ...parallel_distributed_insert_select_view.sh | 6 +- ...arallel_reading_from_replicas_benchmark.sh | 4 +- .../02232_allow_only_replicated_engine.sh | 6 +- .../0_stateless/02250_ON_CLUSTER_grant.sh | 4 +- tests/queries/0_stateless/02262_column_ttl.sh | 4 +- .../0_stateless/02286_parallel_final.sh | 4 +- ..._distinct_in_order_optimization_explain.sh | 66 +++++++++---------- ..._column_ttl_expired_column_optimization.sh | 2 +- .../0_stateless/02361_fsync_profile_events.sh | 4 +- ...7_extend_protocol_with_query_parameters.sh | 10 +-- ...ting_by_input_stream_properties_explain.sh | 2 +- .../0_stateless/02417_load_marks_async.sh | 2 +- 76 files changed, 229 insertions(+), 229 deletions(-) diff --git a/tests/queries/0_stateless/00366_multi_statements.sh b/tests/queries/0_stateless/00366_multi_statements.sh index 8546e547581..0b2e80fe457 100755 --- a/tests/queries/0_stateless/00366_multi_statements.sh +++ b/tests/queries/0_stateless/00366_multi_statements.sh @@ -14,22 +14,22 @@ $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" -$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" -$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT -n --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2" +$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2;" +$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t_00366; CREATE TABLE t_00366 (x UInt64) ENGINE = TinyLog;" +$CLICKHOUSE_CLIENT -n --query="DROP TABLE IF EXISTS t_00366; CREATE TABLE t_00366 (x UInt64) ENGINE = TinyLog;" $CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES (1),(2),(3);" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" $CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES (1),(2),(3);" -$CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" -$CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" +$CLICKHOUSE_CLIENT -n --query="INSERT INTO t_00366 VALUES (1),(2),(3);" +$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t_00366" +$CLICKHOUSE_CLIENT -n --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" +$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t_00366" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1;" @@ -48,4 +48,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+t_00366+VALUES" -d "(7),(8),(9)" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT --query="DROP TABLE t_00366;" +$CLICKHOUSE_CLIENT -n --query="DROP TABLE t_00366;" diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 0635fbc2a57..27b9f5c00c7 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -43,10 +43,10 @@ popd > /dev/null #SCRIPTDIR=`dirname "$SCRIPTPATH"` SCRIPTDIR=$SCRIPTPATH -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED rm "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout diff --git a/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh b/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh index 5550fa69d3d..560b97a1d1b 100755 --- a/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh +++ b/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS test_readonly; CREATE TABLE test_readonly ( ID Int @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT --query=" ################ # Try to create temporary table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 1; CREATE TEMPORARY TABLE readonly ( ID Int @@ -26,7 +26,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 1; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -34,7 +34,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 1; DROP TABLE test_readonly; " 2> /dev/null; @@ -46,7 +46,7 @@ CODE=$?; ################ # Try to create temporary table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 2; CREATE TEMPORARY TABLE readonly ( ID Int @@ -58,7 +58,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 2; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -66,7 +66,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 2; DROP TABLE test_readonly; " 2> /dev/null; @@ -78,7 +78,7 @@ CODE=$?; ################ # Try to create temporary table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 0; CREATE TEMPORARY TABLE readonly ( ID Int @@ -90,7 +90,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 0; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -98,7 +98,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET readonly = 0; DROP TABLE test_readonly; " 2> /dev/null; diff --git a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index e9a4369a5bf..93fd0c4a977 100755 --- a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -19,13 +19,13 @@ settings="$server_logs --log_queries=1 --log_query_threads=1 --log_profile_event # Test insert logging on each block and checkPacket() method -$CLICKHOUSE_CLIENT $settings -q " +$CLICKHOUSE_CLIENT $settings -n -q " DROP TABLE IF EXISTS null_00634; CREATE TABLE null_00634 (i UInt8) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();" head -c 1000 /dev/zero | $CLICKHOUSE_CLIENT $settings --max_insert_block_size=10 --min_insert_block_size_rows=1 --min_insert_block_size_bytes=1 -q "INSERT INTO null_00634 FORMAT RowBinary" -$CLICKHOUSE_CLIENT $settings -q " +$CLICKHOUSE_CLIENT $settings -n -q " SELECT count() FROM null_00634; DROP TABLE null_00634;" diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index d69e14bdbb9..96d5764780f 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" DROP TABLE IF EXISTS fetches_r1 SYNC; DROP TABLE IF EXISTS fetches_r2 SYNC" @@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate SETTINGS prefer_fetch_merged_part_time_threshold=0, \ prefer_fetch_merged_part_size_threshold=0" -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); @@ -51,6 +51,6 @@ ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA fetches_r2" ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutated part ***'" ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" DROP TABLE fetches_r1 SYNC; DROP TABLE fetches_r2 SYNC" diff --git a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh index 989096a26d6..09f20284402 100755 --- a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh +++ b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh @@ -25,83 +25,83 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM distributed WHERE a = 0 AND b | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complext expressions for constant folding - all should pass. -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 1 AND a = 0 AND b = 0; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a IN (0, 1) AND b IN (0, 1); " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR a = 1 AND b = 1; " # TODO: should pass one day. -#${CLICKHOUSE_CLIENT} --query=" +#${CLICKHOUSE_CLIENT} -n --query=" # SET optimize_skip_unused_shards = 1; # SELECT count(*) FROM distributed WHERE a = 0 AND b >= 0 AND b <= 1; #" -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND c = 0; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND c != 10; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND (a+b)*b != 12; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE (a = 0 OR a = 1) AND (b = 0 OR b = 1); " # These ones should fail. -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b <= 1; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND c = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR c = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh index b3dff2ea69a..035907bddd7 100755 --- a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh +++ b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh @@ -30,73 +30,73 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM distributed_00754 PREWHERE a | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complex expressions for constant folding - all should pass. -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 1 AND a = 0 WHERE b = 0; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 1 WHERE b = 1 AND length(c) = 5; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a IN (0, 1) AND b IN (0, 1) WHERE c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a IN (0, 1) WHERE b IN (0, 1) AND c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR a = 1 AND b = 1 WHERE c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE (a = 0 OR a = 1) WHERE (b = 0 OR b = 1); " # These should fail. -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b <= 1; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 WHERE c LIKE '%l%'; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} --query=" +${CLICKHOUSE_CLIENT} -n --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR c LIKE '%l%'; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 8f7a1a9ae98..12d889a7137 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; @@ -62,7 +62,7 @@ do [[ $count1 == 1 && $count2 == 1 ]] && break done -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; diff --git a/tests/queries/0_stateless/00837_minmax_index.sh b/tests/queries/0_stateless/00837_minmax_index.sh index ff487f50ee0..e4de0b9ebfc 100755 --- a/tests/queries/0_stateless/00837_minmax_index.sh +++ b/tests/queries/0_stateless/00837_minmax_index.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00838_unique_index.sh b/tests/queries/0_stateless/00838_unique_index.sh index a3aba4f26b6..b267b6a8eb3 100755 --- a/tests/queries/0_stateless/00838_unique_index.sh +++ b/tests/queries/0_stateless/00838_unique_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE set_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00907_set_index_max_rows.sh b/tests/queries/0_stateless/00907_set_index_max_rows.sh index bdd0f36346f..3707aaf2ca6 100755 --- a/tests/queries/0_stateless/00907_set_index_max_rows.sh +++ b/tests/queries/0_stateless/00907_set_index_max_rows.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE set_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00908_bloom_filter_index.sh b/tests/queries/0_stateless/00908_bloom_filter_index.sh index 3bd169dd6df..25a6567b894 100755 --- a/tests/queries/0_stateless/00908_bloom_filter_index.sh +++ b/tests/queries/0_stateless/00908_bloom_filter_index.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx3;" # NGRAM BF -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_idx ( k UInt64, @@ -22,7 +22,7 @@ CREATE TABLE bloom_filter_idx ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_idx2 ( k UInt64, @@ -109,7 +109,7 @@ $CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT count() FROM bloom # TOKEN BF -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_idx3 ( k UInt64, @@ -147,7 +147,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx2" $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx3" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx_na;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_idx_na ( na Array(Array(String)), @@ -156,7 +156,7 @@ CREATE TABLE bloom_filter_idx_na ORDER BY na" 2>&1 | grep -c 'DB::Exception: Unexpected type Array(Array(String)) of bloom filter index' # NGRAM BF with IPv6 -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_ipv6_idx ( foo IPv6, diff --git a/tests/queries/0_stateless/00942_mutate_index.sh b/tests/queries/0_stateless/00942_mutate_index.sh index e1e23639e85..6ebb30c25b9 100755 --- a/tests/queries/0_stateless/00942_mutate_index.sh +++ b/tests/queries/0_stateless/00942_mutate_index.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00943_materialize_index.sh b/tests/queries/0_stateless/00943_materialize_index.sh index e4a585fce97..6ff7d34a9d7 100755 --- a/tests/queries/0_stateless/00943_materialize_index.sh +++ b/tests/queries/0_stateless/00943_materialize_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE minmax_idx ( u64 UInt64, @@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES $CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 SETTINGS mutations_sync = 2;" $CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1 SETTINGS mutations_sync = 2;" diff --git a/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/tests/queries/0_stateless/00944_clear_index_in_partition.sh index a12536da239..4655077960f 100755 --- a/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh index 9e410f09b13..e2ec7fd42e4 100755 --- a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh +++ b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx;" # NGRAM BF -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_idx ( k UInt64, diff --git a/tests/queries/0_stateless/00965_set_index_string_functions.sh b/tests/queries/0_stateless/00965_set_index_string_functions.sh index 0f29c3dd2f2..8892fb11752 100755 --- a/tests/queries/0_stateless/00965_set_index_string_functions.sh +++ b/tests/queries/0_stateless/00965_set_index_string_functions.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE set_idx ( k UInt64, diff --git a/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh b/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh index ba260042f47..389d433c7e2 100755 --- a/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh +++ b/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS lowString;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS string;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" create table lowString ( a LowCardinality(String), @@ -18,7 +18,7 @@ ENGINE = MergeTree() PARTITION BY toYYYYMM(b) ORDER BY (a)" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" create table string ( a String, diff --git a/tests/queries/0_stateless/00990_hasToken.sh b/tests/queries/0_stateless/00990_hasToken.sh index d79472aa5a5..6a1d4ff5ccf 100755 --- a/tests/queries/0_stateless/00990_hasToken.sh +++ b/tests/queries/0_stateless/00990_hasToken.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -m +python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -nm diff --git a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh index 54f1bbe29dc..55bbfb3ff11 100755 --- a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh +++ b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) R1=table_1013_1 R2=table_1013_2 -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " DROP TABLE IF EXISTS $R1; DROP TABLE IF EXISTS $R2; @@ -19,13 +19,13 @@ ${CLICKHOUSE_CLIENT} -q " INSERT INTO $R1 VALUES (1) " -timeout 10s ${CLICKHOUSE_CLIENT} -q " +timeout 10s ${CLICKHOUSE_CLIENT} -n -q " SET receive_timeout=1; SYSTEM SYNC REPLICA $R2 " 2>&1 | grep -F -q "Code: 159. DB::Exception" && echo 'OK' || echo 'Failed!' # By dropping tables all related SYNC REPLICA queries would be terminated as well -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " DROP TABLE IF EXISTS $R2; DROP TABLE IF EXISTS $R1; " diff --git a/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh b/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh index 053fd9d9d49..4f35b69da0b 100755 --- a/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh +++ b/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh @@ -9,7 +9,7 @@ R1=table_1017_1 R2=table_1017_2 T1=table_1017_merge -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " DROP DICTIONARY IF EXISTS dict1; DROP TABLE IF EXISTS $R1; DROP TABLE IF EXISTS $R2; @@ -68,7 +68,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE dictHas('${CLICKHOUSE ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE dictHas('${CLICKHOUSE_DATABASE}.dict1', toUInt64(x))" --allow_nondeterministic_mutations=1 2>&1 \ && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " DROP DICTIONARY IF EXISTS dict1; DROP TABLE IF EXISTS $R2; DROP TABLE IF EXISTS $R1; diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh index 5c67fe08fbf..9a26f78a8ee 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh @@ -14,7 +14,7 @@ declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${DATA_DIR}" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS points; CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_si rm "${DATA_DIR}"/01037_point_data -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array @@ -43,7 +43,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT --query=" + $CLICKHOUSE_CLIENT -n --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index 591978d1129..47f7a5c1c4f 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -14,7 +14,7 @@ declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL") tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${DATA_DIR}" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " @@ -22,7 +22,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for rm "${DATA_DIR}"/01037_point_data -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array @@ -42,7 +42,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT --query=" + $CLICKHOUSE_CLIENT -n --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array diff --git a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh index ac033ff4eb8..d1ee3f283bc 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) TMP_DIR=${CLICKHOUSE_TMP}/tmp mkdir -p $TMP_DIR -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory; @@ -53,7 +53,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT --query=" + $CLICKHOUSE_CLIENT -n --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array ( @@ -106,7 +106,7 @@ do diff -q "${CURDIR}/01037_polygon_dicts_simple_functions.ans" "$outputFile" done -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP DICTIONARY dict_array; DROP DICTIONARY dict_tuple; DROP TABLE polygons_array; diff --git a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh index 29ce4da02ed..0b14ef8f6fa 100755 --- a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh +++ b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/01077_mutations_index_consistency.sh b/tests/queries/0_stateless/01077_mutations_index_consistency.sh index f103692de56..ffbe3692b64 100755 --- a/tests/queries/0_stateless/01077_mutations_index_consistency.sh +++ b/tests/queries/0_stateless/01077_mutations_index_consistency.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS movement" -$CLICKHOUSE_CLIENT --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';" +$CLICKHOUSE_CLIENT -n --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query "insert into movement select toDateTime('2020-01-22 00:00:00', 'Asia/Istanbul') + number%(23*3600) from numbers(1000000);" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE movement FINAL" -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " SELECT count(), toStartOfHour(date) AS Hour @@ -26,7 +26,7 @@ ORDER BY Hour DESC $CLICKHOUSE_CLIENT --query "alter table movement delete where date >= toDateTime('2020-01-22T16:00:00', 'Asia/Istanbul') and date < toDateTime('2020-01-22T17:00:00', 'Asia/Istanbul') SETTINGS mutations_sync = 2" -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " SELECT count(), toStartOfHour(date) AS Hour @@ -37,7 +37,7 @@ ORDER BY Hour DESC " | grep "16:00:00" | wc -l -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " SELECT count(), toStartOfHour(date) AS Hour @@ -48,7 +48,7 @@ ORDER BY Hour DESC " | grep "22:00:00" | cut -f1 -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " SELECT count(), toStartOfHour(date) AS Hour diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index f6c6fd0be34..42f596c45d6 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -7,11 +7,11 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q "select value, changed from system.settings where name='readonly';" -$CLICKHOUSE_CLIENT -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" -$CLICKHOUSE_CLIENT -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') -$CLICKHOUSE_CLIENT -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&profile=default&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh index 1ec53399958..5f82731c54e 100755 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh @@ -10,7 +10,7 @@ set -o pipefail # shellcheck disable=SC2120 function execute() { - ${CLICKHOUSE_CLIENT} "$@" + ${CLICKHOUSE_CLIENT} -n "$@" } # diff --git a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh index 02aa0f76be5..db986e74b6b 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh @@ -12,8 +12,8 @@ DATA_SIZE=200 SEQ=$(seq 0 $(($NUM_REPLICAS - 1))) -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE IF EXISTS r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done function thread() { @@ -30,6 +30,6 @@ done wait -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "SELECT count(), sum(x) FROM r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "DROP TABLE r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "SYSTEM SYNC REPLICA r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "SELECT count(), sum(x) FROM r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE r$REPLICA"; done diff --git a/tests/queries/0_stateless/01415_sticking_mutations.sh b/tests/queries/0_stateless/01415_sticking_mutations.sh index 97467c3ce9d..b7c8768a65d 100755 --- a/tests/queries/0_stateless/01415_sticking_mutations.sh +++ b/tests/queries/0_stateless/01415_sticking_mutations.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS sticking_mutations" function check_sticky_mutations() { - $CLICKHOUSE_CLIENT --query "CREATE TABLE sticking_mutations ( + $CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations ( date Date, key UInt64, value1 String, diff --git a/tests/queries/0_stateless/01451_dist_logs.sh b/tests/queries/0_stateless/01451_dist_logs.sh index e281e232bb5..23dee7a827d 100755 --- a/tests/queries/0_stateless/01451_dist_logs.sh +++ b/tests/queries/0_stateless/01451_dist_logs.sh @@ -10,4 +10,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # triggered not for the first query for _ in {1..20}; do echo "select * from remote('127.{2,3}', system.numbers) where number = 10 limit 1;" -done | ${CLICKHOUSE_CLIENT} 2>/dev/null +done | ${CLICKHOUSE_CLIENT} -n 2>/dev/null diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index cc574557438..56620d848a3 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -31,7 +31,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 24ea3ba3835..91a73471557 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -16,7 +16,7 @@ unset CLICKHOUSE_WRITE_COVERAGE NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -39,7 +39,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh index a2ef0d52328..1f76a2efc6b 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh @@ -11,7 +11,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -36,7 +36,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh index 6954fef7314..29593ea4fb5 100755 --- a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh +++ b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh @@ -34,7 +34,7 @@ done # Check access rights -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " DROP DATABASE IF EXISTS test; CREATE DATABASE test; USE test; diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.sh b/tests/queries/0_stateless/01508_format_regexp_raw.sh index 52613c28b2f..8cf1bd73566 100755 --- a/tests/queries/0_stateless/01508_format_regexp_raw.sh +++ b/tests/queries/0_stateless/01508_format_regexp_raw.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query " +${CLICKHOUSE_CLIENT} -n --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a String, b String) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a String, b String) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' -${CLICKHOUSE_CLIENT} --query " +${CLICKHOUSE_CLIENT} -n --query " SELECT * FROM t; DROP TABLE t; " diff --git a/tests/queries/0_stateless/01509_dictionary_preallocate.sh b/tests/queries/0_stateless/01509_dictionary_preallocate.sh index 0459f69b0ad..2a22a307a08 100755 --- a/tests/queries/0_stateless/01509_dictionary_preallocate.sh +++ b/tests/queries/0_stateless/01509_dictionary_preallocate.sh @@ -15,7 +15,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # PREALLOCATE attribute (and also for the history/greppability, that it was # such). -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS data_01509; DROP DICTIONARY IF EXISTS dict_01509; CREATE TABLE data_01509 diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh index dc178d081bf..594caca7d04 100755 --- a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query " +${CLICKHOUSE_CLIENT} -n --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' -${CLICKHOUSE_CLIENT} --query " +${CLICKHOUSE_CLIENT} -n --query " SELECT * FROM t; DROP TABLE t; " diff --git a/tests/queries/0_stateless/01526_initial_query_id.sh b/tests/queries/0_stateless/01526_initial_query_id.sh index 8ba27a04d60..e77764ee34e 100755 --- a/tests/queries/0_stateless/01526_initial_query_id.sh +++ b/tests/queries/0_stateless/01526_initial_query_id.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CURL} \ --get \ --data-urlencode "query=select 1 format Null" -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " system flush logs; select interface, initial_query_id = query_id from system.query_log diff --git a/tests/queries/0_stateless/01599_mutation_query_params.sh b/tests/queries/0_stateless/01599_mutation_query_params.sh index 5b604c96028..52b0131a9c2 100755 --- a/tests/queries/0_stateless/01599_mutation_query_params.sh +++ b/tests/queries/0_stateless/01599_mutation_query_params.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS test; CREATE TABLE test diff --git a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh index 834eba8f25c..1d768c8b027 100755 --- a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh +++ b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " CREATE USER quoted_by_ip_${CLICKHOUSE_DATABASE}; CREATE USER quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; @@ -57,7 +57,7 @@ ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 5.6.7.8, 1.2.3.4' -sS "${CLICKHOUSE_URL} ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4, 5.6.7.8' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" -$CLICKHOUSE_CLIENT --query " +$CLICKHOUSE_CLIENT -n --query " DROP QUOTA IF EXISTS quota_by_ip_${CLICKHOUSE_DATABASE}; DROP QUOTA IF EXISTS quota_by_forwarded_ip; diff --git a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh index 6a7eb975c87..0e5c2862066 100755 --- a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh +++ b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary; CREATE DATABASE 01684_database_for_cache_dictionary; diff --git a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh index c2d222a86ea..55061b9a643 100755 --- a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh +++ b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" CREATE TABLE complex_key_simple_attributes_source_table ( id UInt64, diff --git a/tests/queries/0_stateless/01691_parser_data_type_exponential.sh b/tests/queries/0_stateless/01691_parser_data_type_exponential.sh index 5d115e09a79..f8004f9350d 100755 --- a/tests/queries/0_stateless/01691_parser_data_type_exponential.sh +++ b/tests/queries/0_stateless/01691_parser_data_type_exponential.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Check that DataType parser does not have exponential complexity in the case found by fuzzer. -for _ in {1..10}; do ${CLICKHOUSE_CLIENT} --query "SELECT CAST(1 AS A2222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222220000000000000000000000000000000000000000000000000000000000000000000000000000002260637443813394204 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggre222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 22222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 2222222222222eFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222, 222222222222222ggregateFuncpion(groupBitmap222222222222222222222222222222222222222222222222222222222222222222222222000000000000000000001788596394540167623 222222222222222222ggregateFu22222222222222222222222222 222222222, UInt33)); -- { clientError 62 }"; done +for _ in {1..10}; do ${CLICKHOUSE_CLIENT} -n --query "SELECT CAST(1 AS A2222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222220000000000000000000000000000000000000000000000000000000000000000000000000000002260637443813394204 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggre222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 22222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 2222222222222eFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222, 222222222222222ggregateFuncpion(groupBitmap222222222222222222222222222222222222222222222222222222222222222222222222000000000000000000001788596394540167623 222222222222222222ggregateFu22222222222222222222222222 222222222, UInt33)); -- { clientError 62 }"; done diff --git a/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh index f38e53f898a..a166837e01a 100755 --- a/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS in_order_agg_01710; CREATE TABLE in_order_agg_01710 diff --git a/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh index 01537524730..ee73974e8a4 100755 --- a/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS in_order_agg_partial_01710; CREATE TABLE in_order_agg_partial_01710 diff --git a/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh index f9681ebe4f5..2a7345f4865 100755 --- a/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -m -q " +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -nm -q " drop table if exists data_01753; create table data_01753 (key Int) engine=MergeTree() order by key as select * from numbers(8); select * from data_01753 group by key settings max_block_size=1; diff --git a/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh b/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh index 3c9e12f780b..b963f3a618f 100755 --- a/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh +++ b/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --optimize_skip_unused_shards=1 -m -q " +$CLICKHOUSE_CLIENT --optimize_skip_unused_shards=1 -nm -q " create table dist_01758 as system.one engine=Distributed(test_cluster_two_shards, system, one, dummy); select * from dist_01758 where dummy = 0 format Null; " |& grep -o "StorageDistributed (dist_01758).*" diff --git a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh index ee46f8194b9..9c51b82282c 100755 --- a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh +++ b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 -m -q " +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 -nm -q " DROP TABLE IF EXISTS tmp_01683; DROP TABLE IF EXISTS dist_01683; diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh index f3e8ceffff6..4b75102e9cf 100755 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -13,7 +13,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function setup() { - $CLICKHOUSE_CLIENT -m -q " + $CLICKHOUSE_CLIENT -nm -q " drop table if exists data_01814; drop table if exists dist_01814; @@ -24,7 +24,7 @@ function setup() function cleanup() { - $CLICKHOUSE_CLIENT -m -q " + $CLICKHOUSE_CLIENT -nm -q " drop table data_01814; drop table dist_01814; " @@ -67,7 +67,7 @@ function test_distributed_push_down_limit_with_query_log() $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from $table group by key limit $offset, 10" - $CLICKHOUSE_CLIENT -m -q " + $CLICKHOUSE_CLIENT -nm -q " system flush logs; select read_rows from system.query_log where diff --git a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh index 2b122debff5..218320772c9 100755 --- a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh +++ b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_test_once() { - $CLICKHOUSE_CLIENT -m -q " + $CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS simple_key_source_table_01863; CREATE TABLE simple_key_source_table_01863 ( @@ -29,13 +29,13 @@ function run_test_once() LIFETIME(MIN 0 MAX 1000); " - prev=$($CLICKHOUSE_CLIENT -m -q "SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1") - curr=$($CLICKHOUSE_CLIENT -m -q " + prev=$($CLICKHOUSE_CLIENT -nm -q "SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1") + curr=$($CLICKHOUSE_CLIENT -nm -q " SELECT toUInt64(1) as key, dictGet('simple_key_cache_dictionary_01863', 'value', key) FORMAT Null; SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1 ") - $CLICKHOUSE_CLIENT -m -q " + $CLICKHOUSE_CLIENT -nm -q " DROP DICTIONARY simple_key_cache_dictionary_01863; DROP TABLE simple_key_source_table_01863; " diff --git a/tests/queries/0_stateless/01872_initial_query_start_time.sh b/tests/queries/0_stateless/01872_initial_query_start_time.sh index ff3e1954c75..6a935602ea4 100755 --- a/tests/queries/0_stateless/01872_initial_query_start_time.sh +++ b/tests/queries/0_stateless/01872_initial_query_start_time.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} -q "create table m (dummy UInt8) ENGINE = Distributed('test query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") ${CLICKHOUSE_CLIENT} -q "select * from m format Null" "--query_id=$query_id" -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " system flush logs; select anyIf(initial_query_start_time, is_initial_query) = anyIf(initial_query_start_time, not is_initial_query), diff --git a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh index 1294ba53e82..8336229a643 100755 --- a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh +++ b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" SET allow_experimental_bigint_types = 1; DROP TABLE IF EXISTS dictionary_decimal_source_table; diff --git a/tests/queries/0_stateless/01890_materialized_distributed_join.sh b/tests/queries/0_stateless/01890_materialized_distributed_join.sh index 5c04ee8b214..88f7dcf9a69 100755 --- a/tests/queries/0_stateless/01890_materialized_distributed_join.sh +++ b/tests/queries/0_stateless/01890_materialized_distributed_join.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table if exists test_distributed; drop table if exists test_source; drop table if exists test_shard; diff --git a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh index a44106414ea..853445daf3f 100755 --- a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh +++ b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS dictionary_array_source_table; CREATE TABLE dictionary_array_source_table ( diff --git a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh index a5c65ca87a7..0b555cf82c2 100755 --- a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh +++ b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query=" +$CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS dictionary_nullable_source_table; CREATE TABLE dictionary_nullable_source_table ( diff --git a/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh b/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh index 608107c76d6..47d5e733480 100755 --- a/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh +++ b/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - ${CLICKHOUSE_CLIENT} -q " + ${CLICKHOUSE_CLIENT} -n -q " DROP TABLE IF EXISTS matview_exception_a_to_c; DROP TABLE IF EXISTS matview_exception_a_to_b; DROP TABLE IF EXISTS table_exception_c; @@ -17,7 +17,7 @@ function cleanup() function setup() { - ${CLICKHOUSE_CLIENT} -q " + ${CLICKHOUSE_CLIENT} -n -q " CREATE TABLE table_exception_a (a String, b Int64) ENGINE = MergeTree ORDER BY b; CREATE TABLE table_exception_b (a Float64, b Int64) ENGINE = MergeTree ORDER BY tuple(); CREATE TABLE table_exception_c (a Float64) ENGINE = MergeTree ORDER BY a; diff --git a/tests/queries/0_stateless/01947_multiple_pipe_read.sh b/tests/queries/0_stateless/01947_multiple_pipe_read.sh index 51709eb574e..06a18a55e6e 100755 --- a/tests/queries/0_stateless/01947_multiple_pipe_read.sh +++ b/tests/queries/0_stateless/01947_multiple_pipe_read.sh @@ -11,7 +11,7 @@ cat "$SAMPLE_FILE" echo '******************' echo 'Read twice from a regular file' -${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' --file "$SAMPLE_FILE" +${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' --file "$SAMPLE_FILE" echo '---' ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' --file "$SAMPLE_FILE" echo '---' @@ -19,7 +19,7 @@ ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table UNI echo '******************' echo 'Read twice from file descriptor that corresponds to a regular file' -${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' < "$SAMPLE_FILE" +${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' < "$SAMPLE_FILE" echo '---' ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' < "$SAMPLE_FILE" echo '---' diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 15cacbff8c5..96028f4847a 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -4,21 +4,21 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2P' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2.1p' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10E' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10.2e' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1.1T' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1m' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='14g' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11t' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2P' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2.1p' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10E' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10.2e' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1.1T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1m' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='14g' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11t' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.sh b/tests/queries/0_stateless/02021_create_database_with_comment.sh index d87b0794c91..f77397dc482 100755 --- a/tests/queries/0_stateless/02021_create_database_with_comment.sh +++ b/tests/queries/0_stateless/02021_create_database_with_comment.sh @@ -20,7 +20,7 @@ function test_db_comments() local ENGINE_NAME="$1" echo "engine : ${ENGINE_NAME}" - $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -m <& /dev/null +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' >& /dev/null echo $? echo 'regression test for overlap profile events snapshots between queries' -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'regression test for overlap profile events snapshots between queries (clickhouse-local)' -$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'print everything' profile_events="$( @@ -35,5 +35,5 @@ profile_events="$( test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" echo 'check that ProfileEvents is new for each query' -sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') +sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') test "$sleep_function_calls" -eq 1 && echo OK || echo "FAIL ($sleep_function_calls)" diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index a382b3859f3..3c44a2a7ba7 100755 --- a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh +++ b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -m < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh index e23a272a4e8..db94c59d2de 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2" -${CLICKHOUSE_CLIENT} -q" +${CLICKHOUSE_CLIENT} -n -q" CREATE TABLE sample_table ( key UInt64 ) @@ -16,7 +16,7 @@ ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_ ORDER BY tuple(); " -${CLICKHOUSE_CLIENT} -q" +${CLICKHOUSE_CLIENT} -n -q" CREATE TABLE sample_table_2 ( key UInt64 ) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh index 6381d811d5d..c62ec14b340 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE sample_table ( +${CLICKHOUSE_CLIENT} -n --query="CREATE TABLE sample_table ( key UInt64 ) ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like', '1') @@ -16,7 +16,7 @@ ORDER BY tuple(); DROP TABLE IF EXISTS sample_table SYNC;" -${CLICKHOUSE_CLIENT} --query "CREATE TABLE sample_table_2 ( +${CLICKHOUSE_CLIENT} -n --query "CREATE TABLE sample_table_2 ( key UInt64 ) ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like_2', '1') diff --git a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh index 63111cc32e4..376a49fd820 100755 --- a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh +++ b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table if exists dst_02225; drop table if exists src_02225; create table dst_02225 (key Int) engine=Memory(); @@ -14,7 +14,7 @@ create table src_02225 (key Int) engine=Memory(); insert into src_02225 values (1); " -$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -m -q " +$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -nm -q " truncate table dst_02225; insert into function remote('127.{1,2}', currentDatabase(), dst_02225, key) select * from remote('127.{1,2}', view(select * from {database:Identifier}.src_02225), key) @@ -29,7 +29,7 @@ settings parallel_distributed_insert_select=2, max_distributed_depth=1; select * from dst_02225; " -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table src_02225; drop table dst_02225; " diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index 177b373641f..bc90f4b2c11 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table if exists data_02226; create table data_02226 (key Int) engine=MergeTree() order by key as select * from numbers(1); @@ -24,7 +24,7 @@ opts=( $CLICKHOUSE_BENCHMARK --query "select * from remote('127.1', $CLICKHOUSE_DATABASE, data_02226)" "${opts[@]}" >& /dev/null ret=$? -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table data_02226; " diff --git a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh index e47a3033681..d1a3825d286 100755 --- a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh +++ b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh @@ -12,9 +12,9 @@ ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO us ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON Memory, TABLE ENGINE ON MergeTree, TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_memory (x UInt32) engine = Memory;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" diff --git a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh index 09f9c0c8a98..66417e9694a 100755 --- a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh +++ b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - $CLICKHOUSE_CLIENT -mq " + $CLICKHOUSE_CLIENT -nmq " DROP USER IF EXISTS with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; DROP USER IF EXISTS without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; DROP DATABASE IF EXISTS db_with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; @@ -15,7 +15,7 @@ function cleanup() cleanup trap cleanup EXIT -$CLICKHOUSE_CLIENT -mq " +$CLICKHOUSE_CLIENT -nmq " CREATE USER with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; CREATE USER without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; diff --git a/tests/queries/0_stateless/02262_column_ttl.sh b/tests/queries/0_stateless/02262_column_ttl.sh index c620d3b6d9c..b5e29c9b2a1 100755 --- a/tests/queries/0_stateless/02262_column_ttl.sh +++ b/tests/queries/0_stateless/02262_column_ttl.sh @@ -14,7 +14,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # note, that this should be written in .sh since we need $CLICKHOUSE_DATABASE # not 'default' to catch text_log -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table if exists ttl_02262; drop table if exists this_text_log; @@ -31,7 +31,7 @@ $CLICKHOUSE_CLIENT -m -q " ttl_02262_uuid=$($CLICKHOUSE_CLIENT -q "select uuid from system.tables where database = '$CLICKHOUSE_DATABASE' and name = 'ttl_02262'") -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " -- OPTIMIZE TABLE x FINAL will be done in background -- attach to it's log, via table UUID in query_id (see merger/mutator code). create materialized view this_text_log engine=Memory() as diff --git a/tests/queries/0_stateless/02286_parallel_final.sh b/tests/queries/0_stateless/02286_parallel_final.sh index 47dfad42e11..0ac510208f3 100755 --- a/tests/queries/0_stateless/02286_parallel_final.sh +++ b/tests/queries/0_stateless/02286_parallel_final.sh @@ -9,7 +9,7 @@ echo "Test intersecting ranges" test_random_values() { layers=$1 - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " drop table if exists tbl_8parts_${layers}granules_rnd; create table tbl_8parts_${layers}granules_rnd (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 % 8); insert into tbl_8parts_${layers}granules_rnd select number, 1 from numbers_mt($((layers * 8 * 8192))); @@ -29,7 +29,7 @@ echo "Test non intersecting ranges" test_sequential_values() { layers=$1 - $CLICKHOUSE_CLIENT -q " + $CLICKHOUSE_CLIENT -n -q " drop table if exists tbl_8parts_${layers}granules_seq; create table tbl_8parts_${layers}granules_seq (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 / $((layers * 8192)))::UInt64; insert into tbl_8parts_${layers}granules_seq select number, 1 from numbers_mt($((layers * 8 * 8192))); diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 953485c3a1f..bd7e6be3987 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -23,99 +23,99 @@ $CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % num $CLICKHOUSE_CLIENT -q "select '-- disable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> ordinary distinct'" -$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- enable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct with all primary key columns -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by the same columns -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by columns are prefix of distinct columns -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column _not_ in distinct -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> ordinary distinct'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column in distinct -> final distinct optimization only'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column _not_ in distinct -> ordinary distinct'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by _const_ column in distinct -> ordinary distinct'" -$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT echo "-- Check reading in order for distinct" echo "-- disabled, distinct columns match sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -q "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "-- enabled, distinct columns match sorting key" # read_in_order_two_level_merge_threshold is set here to avoid repeating MergeTreeInOrder in output -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns DON't form prefix of sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "-- enabled, distinct columns contains constant columns, non-const columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns contains constant columns, non-const columns match prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, b, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, b, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, only part of distinct columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "=== disable new analyzer ===" DISABLE_ANALYZER="set enable_analyzer=0" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES echo "=== enable new analyzer ===" ENABLE_ANALYZER="set enable_analyzer=1" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" diff --git a/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh b/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh index 490f8361682..96f80d65878 100755 --- a/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh +++ b/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) data_path="$CLICKHOUSE_TMP/local" -$CLICKHOUSE_LOCAL --path "$data_path" -m -q " +$CLICKHOUSE_LOCAL --path "$data_path" -nm -q " create table ttl_02335 ( date Date, key Int, diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 73bf3fa120a..98c9cf9b7b4 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -m -q " +$CLICKHOUSE_CLIENT -nm -q " drop table if exists data_fsync_pe; create table data_fsync_pe (key Int) engine=MergeTree() @@ -27,7 +27,7 @@ for i in {1..100}; do $CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data_fsync_pe values (1)" read -r FileSync FileOpen DirectorySync FileSyncElapsedMicroseconds DirectorySyncElapsedMicroseconds <<<"$( - $CLICKHOUSE_CLIENT -m --param_query_id "$query_id" -q " + $CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " system flush logs; select diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh index 46396d38747..71e3b6961f8 100755 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT \ table_name="t_02377_extend_protocol_with_query_parameters_$RANDOM$RANDOM" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT -n -q " create table $table_name( id Int64, arr Array(UInt8), @@ -57,17 +57,17 @@ $CLICKHOUSE_CLIENT \ # it is possible to set parameter for the current session -$CLICKHOUSE_CLIENT -q "set param_n = 42; select {n: UInt8}" +$CLICKHOUSE_CLIENT -n -q "set param_n = 42; select {n: UInt8}" # and it will not be visible to other sessions -$CLICKHOUSE_CLIENT -q "select {n: UInt8} -- { serverError 456 }" +$CLICKHOUSE_CLIENT -n -q "select {n: UInt8} -- { serverError 456 }" # the same parameter could be set multiple times within one session (new value overrides the previous one) -$CLICKHOUSE_CLIENT -q "set param_n = 12; set param_n = 13; select {n: UInt8}" +$CLICKHOUSE_CLIENT -n -q "set param_n = 12; set param_n = 13; select {n: UInt8}" # multiple different parameters could be defined within each session -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT -n -q " set param_a = 13, param_b = 'str'; set param_c = '2022-08-04 18:30:53'; set param_d = '{\'10\': [11, 12], \'13\': [14, 15]}'; diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index 974f10e2f24..4b9793da5bb 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -15,7 +15,7 @@ FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES" function explain_sorting { echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -q "$1" | eval $FIND_SORTING + $CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTING } function explain_sortmode { diff --git a/tests/queries/0_stateless/02417_load_marks_async.sh b/tests/queries/0_stateless/02417_load_marks_async.sh index bcede9e4f5e..950656e7ab6 100755 --- a/tests/queries/0_stateless/02417_load_marks_async.sh +++ b/tests/queries/0_stateless/02417_load_marks_async.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test;" -${CLICKHOUSE_CLIENT} -q " +${CLICKHOUSE_CLIENT} -n -q " CREATE TABLE test ( n0 UInt64, From 3af92fc65258f117dce6156a5c60d93267a65ab1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 15 Aug 2024 12:40:52 +0000 Subject: [PATCH 1006/1722] Reduce number of changed tests --- .../01013_sync_replica_timeout_zookeeper.sh | 6 +++--- ...ations_with_nondeterministic_functions_zookeeper.sh | 4 ++-- .../0_stateless/01037_polygon_dicts_correctness_all.sh | 6 +++--- .../01037_polygon_dicts_correctness_fast.sh | 6 +++--- .../01037_polygon_dicts_simple_functions.sh | 6 +++--- .../0_stateless/01055_minmax_index_compact_parts.sh | 2 +- .../0_stateless/01077_mutations_index_consistency.sh | 10 +++++----- .../0_stateless/01187_set_profile_as_setting.sh | 8 ++++---- ...in_insert_block_size_rows_for_materialized_views.sh | 2 +- .../0_stateless/01307_multiple_leaders_zookeeper.sh | 10 +++++----- tests/queries/0_stateless/01415_sticking_mutations.sh | 2 +- tests/queries/0_stateless/01451_dist_logs.sh | 2 +- .../0_stateless/01459_manual_write_to_replicas.sh | 4 ++-- .../01459_manual_write_to_replicas_quorum.sh | 4 ++-- ...59_manual_write_to_replicas_quorum_detach_attach.sh | 4 ++-- ...507_clickhouse_server_start_with_embedded_config.sh | 2 +- tests/queries/0_stateless/01508_format_regexp_raw.sh | 4 ++-- .../0_stateless/01509_dictionary_preallocate.sh | 2 +- .../01510_format_regexp_raw_low_cardinality.sh | 4 ++-- tests/queries/0_stateless/01526_initial_query_id.sh | 2 +- .../queries/0_stateless/01599_mutation_query_params.sh | 2 +- .../queries/0_stateless/01600_quota_by_forwarded_ip.sh | 4 ++-- .../01684_ssd_cache_dictionary_simple_key.sh | 2 +- .../01685_ssd_cache_dictionary_complex_key.sh | 2 +- .../0_stateless/01691_parser_data_type_exponential.sh | 2 +- .../01710_projections_optimize_aggregation_in_order.sh | 2 +- ...rojections_partial_optimize_aggregation_in_order.sh | 2 +- .../0_stateless/01753_optimize_aggregation_in_order.sh | 2 +- .../01758_optimize_skip_unused_shards_once.sh | 2 +- .../01791_dist_INSERT_block_structure_mismatch.sh | 2 +- .../0_stateless/01814_distributed_push_down_limit.sh | 6 +++--- .../0_stateless/01853_dictionary_cache_duplicates.sh | 8 ++++---- .../0_stateless/01872_initial_query_start_time.sh | 2 +- .../01875_ssd_cache_dictionary_decimal256_type.sh | 2 +- .../0_stateless/01890_materialized_distributed_join.sh | 2 +- .../01903_ssd_cache_dictionary_array_type.sh | 2 +- ...01904_ssd_cache_dictionary_default_nullable_type.sh | 2 +- .../01927_query_views_log_matview_exceptions.sh | 4 ++-- tests/queries/0_stateless/01947_multiple_pipe_read.sh | 4 ++-- 39 files changed, 72 insertions(+), 72 deletions(-) diff --git a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh index 55bbfb3ff11..54f1bbe29dc 100755 --- a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh +++ b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) R1=table_1013_1 R2=table_1013_2 -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS $R1; DROP TABLE IF EXISTS $R2; @@ -19,13 +19,13 @@ ${CLICKHOUSE_CLIENT} -n -q " INSERT INTO $R1 VALUES (1) " -timeout 10s ${CLICKHOUSE_CLIENT} -n -q " +timeout 10s ${CLICKHOUSE_CLIENT} -q " SET receive_timeout=1; SYSTEM SYNC REPLICA $R2 " 2>&1 | grep -F -q "Code: 159. DB::Exception" && echo 'OK' || echo 'Failed!' # By dropping tables all related SYNC REPLICA queries would be terminated as well -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS $R2; DROP TABLE IF EXISTS $R1; " diff --git a/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh b/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh index 4f35b69da0b..053fd9d9d49 100755 --- a/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh +++ b/tests/queries/0_stateless/01017_mutations_with_nondeterministic_functions_zookeeper.sh @@ -9,7 +9,7 @@ R1=table_1017_1 R2=table_1017_2 T1=table_1017_merge -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP DICTIONARY IF EXISTS dict1; DROP TABLE IF EXISTS $R1; DROP TABLE IF EXISTS $R2; @@ -68,7 +68,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE dictHas('${CLICKHOUSE ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $R1 DELETE WHERE dictHas('${CLICKHOUSE_DATABASE}.dict1', toUInt64(x))" --allow_nondeterministic_mutations=1 2>&1 \ && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " DROP DICTIONARY IF EXISTS dict1; DROP TABLE IF EXISTS $R2; DROP TABLE IF EXISTS $R1; diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh index 9a26f78a8ee..5c67fe08fbf 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh @@ -14,7 +14,7 @@ declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${DATA_DIR}" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS points; CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_si rm "${DATA_DIR}"/01037_point_data -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array @@ -43,7 +43,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index 47f7a5c1c4f..591978d1129 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -14,7 +14,7 @@ declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL") tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${DATA_DIR}" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " @@ -22,7 +22,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for rm "${DATA_DIR}"/01037_point_data -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array @@ -42,7 +42,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array diff --git a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh index d1ee3f283bc..ac033ff4eb8 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) TMP_DIR=${CLICKHOUSE_TMP}/tmp mkdir -p $TMP_DIR -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS polygons_array; CREATE TABLE polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory; @@ -53,7 +53,7 @@ for type in "${SearchTypes[@]}"; do outputFile="${TMP_DIR}/results${type}.out" - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" DROP DICTIONARY IF EXISTS dict_array; CREATE DICTIONARY dict_array ( @@ -106,7 +106,7 @@ do diff -q "${CURDIR}/01037_polygon_dicts_simple_functions.ans" "$outputFile" done -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP DICTIONARY dict_array; DROP DICTIONARY dict_tuple; DROP TABLE polygons_array; diff --git a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh index 0b14ef8f6fa..29ce4da02ed 100755 --- a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh +++ b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/01077_mutations_index_consistency.sh b/tests/queries/0_stateless/01077_mutations_index_consistency.sh index ffbe3692b64..f103692de56 100755 --- a/tests/queries/0_stateless/01077_mutations_index_consistency.sh +++ b/tests/queries/0_stateless/01077_mutations_index_consistency.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS movement" -$CLICKHOUSE_CLIENT -n --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';" +$CLICKHOUSE_CLIENT --query "CREATE TABLE movement (date DateTime('Asia/Istanbul')) Engine = MergeTree ORDER BY (toStartOfHour(date)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query "insert into movement select toDateTime('2020-01-22 00:00:00', 'Asia/Istanbul') + number%(23*3600) from numbers(1000000);" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE movement FINAL" -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour @@ -26,7 +26,7 @@ ORDER BY Hour DESC $CLICKHOUSE_CLIENT --query "alter table movement delete where date >= toDateTime('2020-01-22T16:00:00', 'Asia/Istanbul') and date < toDateTime('2020-01-22T17:00:00', 'Asia/Istanbul') SETTINGS mutations_sync = 2" -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour @@ -37,7 +37,7 @@ ORDER BY Hour DESC " | grep "16:00:00" | wc -l -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour @@ -48,7 +48,7 @@ ORDER BY Hour DESC " | grep "22:00:00" | cut -f1 -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " SELECT count(), toStartOfHour(date) AS Hour diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index 42f596c45d6..f6c6fd0be34 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -7,11 +7,11 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" -$CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" -$CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT -m -q "select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') -$CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" +$CLICKHOUSE_CLIENT -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&profile=default&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh index 5f82731c54e..1ec53399958 100755 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh @@ -10,7 +10,7 @@ set -o pipefail # shellcheck disable=SC2120 function execute() { - ${CLICKHOUSE_CLIENT} -n "$@" + ${CLICKHOUSE_CLIENT} "$@" } # diff --git a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh index db986e74b6b..02aa0f76be5 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders_zookeeper.sh @@ -12,8 +12,8 @@ DATA_SIZE=200 SEQ=$(seq 0 $(($NUM_REPLICAS - 1))) -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE IF EXISTS r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "CREATE TABLE r$REPLICA (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r$REPLICA') ORDER BY x SETTINGS min_bytes_for_wide_part = '10M';"; done function thread() { @@ -30,6 +30,6 @@ done wait -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "SYSTEM SYNC REPLICA r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "SELECT count(), sum(x) FROM r$REPLICA"; done -for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT -n --query "DROP TABLE r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "SELECT count(), sum(x) FROM r$REPLICA"; done +for REPLICA in $SEQ; do $CLICKHOUSE_CLIENT --query "DROP TABLE r$REPLICA"; done diff --git a/tests/queries/0_stateless/01415_sticking_mutations.sh b/tests/queries/0_stateless/01415_sticking_mutations.sh index b7c8768a65d..97467c3ce9d 100755 --- a/tests/queries/0_stateless/01415_sticking_mutations.sh +++ b/tests/queries/0_stateless/01415_sticking_mutations.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS sticking_mutations" function check_sticky_mutations() { - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations ( + $CLICKHOUSE_CLIENT --query "CREATE TABLE sticking_mutations ( date Date, key UInt64, value1 String, diff --git a/tests/queries/0_stateless/01451_dist_logs.sh b/tests/queries/0_stateless/01451_dist_logs.sh index 23dee7a827d..e281e232bb5 100755 --- a/tests/queries/0_stateless/01451_dist_logs.sh +++ b/tests/queries/0_stateless/01451_dist_logs.sh @@ -10,4 +10,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # triggered not for the first query for _ in {1..20}; do echo "select * from remote('127.{2,3}', system.numbers) where number = 10 limit 1;" -done | ${CLICKHOUSE_CLIENT} -n 2>/dev/null +done | ${CLICKHOUSE_CLIENT} 2>/dev/null diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 56620d848a3..cc574557438 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -31,7 +31,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 91a73471557..24ea3ba3835 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -16,7 +16,7 @@ unset CLICKHOUSE_WRITE_COVERAGE NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -39,7 +39,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh index 1f76a2efc6b..a2ef0d52328 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum_detach_attach.sh @@ -11,7 +11,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " @@ -36,7 +36,7 @@ done wait for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " SYSTEM SYNC REPLICA r$i; SELECT count(), min(x), max(x), sum(x) FROM r$i;" done diff --git a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh index 29593ea4fb5..6954fef7314 100755 --- a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh +++ b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh @@ -34,7 +34,7 @@ done # Check access rights -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP DATABASE IF EXISTS test; CREATE DATABASE test; USE test; diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.sh b/tests/queries/0_stateless/01508_format_regexp_raw.sh index 8cf1bd73566..52613c28b2f 100755 --- a/tests/queries/0_stateless/01508_format_regexp_raw.sh +++ b/tests/queries/0_stateless/01508_format_regexp_raw.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a String, b String) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a String, b String) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t; DROP TABLE t; " diff --git a/tests/queries/0_stateless/01509_dictionary_preallocate.sh b/tests/queries/0_stateless/01509_dictionary_preallocate.sh index 2a22a307a08..0459f69b0ad 100755 --- a/tests/queries/0_stateless/01509_dictionary_preallocate.sh +++ b/tests/queries/0_stateless/01509_dictionary_preallocate.sh @@ -15,7 +15,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # PREALLOCATE attribute (and also for the history/greppability, that it was # such). -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS data_01509; DROP DICTIONARY IF EXISTS dict_01509; CREATE TABLE data_01509 diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh index 594caca7d04..dc178d081bf 100755 --- a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t; DROP TABLE t; " diff --git a/tests/queries/0_stateless/01526_initial_query_id.sh b/tests/queries/0_stateless/01526_initial_query_id.sh index e77764ee34e..8ba27a04d60 100755 --- a/tests/queries/0_stateless/01526_initial_query_id.sh +++ b/tests/queries/0_stateless/01526_initial_query_id.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CURL} \ --get \ --data-urlencode "query=select 1 format Null" -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " system flush logs; select interface, initial_query_id = query_id from system.query_log diff --git a/tests/queries/0_stateless/01599_mutation_query_params.sh b/tests/queries/0_stateless/01599_mutation_query_params.sh index 52b0131a9c2..5b604c96028 100755 --- a/tests/queries/0_stateless/01599_mutation_query_params.sh +++ b/tests/queries/0_stateless/01599_mutation_query_params.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS test; CREATE TABLE test diff --git a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh index 1d768c8b027..834eba8f25c 100755 --- a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh +++ b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " CREATE USER quoted_by_ip_${CLICKHOUSE_DATABASE}; CREATE USER quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}; @@ -57,7 +57,7 @@ ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 5.6.7.8, 1.2.3.4' -sS "${CLICKHOUSE_URL} ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4, 5.6.7.8' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP QUOTA IF EXISTS quota_by_ip_${CLICKHOUSE_DATABASE}; DROP QUOTA IF EXISTS quota_by_forwarded_ip; diff --git a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh index 0e5c2862066..6a7eb975c87 100755 --- a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh +++ b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary; CREATE DATABASE 01684_database_for_cache_dictionary; diff --git a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh index 55061b9a643..c2d222a86ea 100755 --- a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh +++ b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE complex_key_simple_attributes_source_table ( id UInt64, diff --git a/tests/queries/0_stateless/01691_parser_data_type_exponential.sh b/tests/queries/0_stateless/01691_parser_data_type_exponential.sh index f8004f9350d..5d115e09a79 100755 --- a/tests/queries/0_stateless/01691_parser_data_type_exponential.sh +++ b/tests/queries/0_stateless/01691_parser_data_type_exponential.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Check that DataType parser does not have exponential complexity in the case found by fuzzer. -for _ in {1..10}; do ${CLICKHOUSE_CLIENT} -n --query "SELECT CAST(1 AS A2222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222220000000000000000000000000000000000000000000000000000000000000000000000000000002260637443813394204 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggre222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 22222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 2222222222222eFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222, 222222222222222ggregateFuncpion(groupBitmap222222222222222222222222222222222222222222222222222222222222222222222222000000000000000000001788596394540167623 222222222222222222ggregateFu22222222222222222222222222 222222222, UInt33)); -- { clientError 62 }"; done +for _ in {1..10}; do ${CLICKHOUSE_CLIENT} --query "SELECT CAST(1 AS A2222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222220000000000000000000000000000000000000000000000000000000000000000000000000000002260637443813394204 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggre222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 22222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 2222222222222eFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpio22222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 00000000000000000000000000000000000000000000000000000000000000000000000000000001841416382, 222222222222222ggregateFuncpion(groupBitmap22222222222222222222222222222222222222222222222222222222222222222222222200000000000000000000178859639454016722222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmapp, 222222222222222ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateF222222222222222222222222222222222222222222222222222222222teFuncpion(groupBitmap, 222222222222223ggregateFuncpion(groupBitmap2222222222222222222222222222222222222222222222222222 222222222222222222ggregateFuncpion(groupBitmap, 22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222, 222222222222222ggregateFuncpion(groupBitmap222222222222222222222222222222222222222222222222222222222222222222222222000000000000000000001788596394540167623 222222222222222222ggregateFu22222222222222222222222222 222222222, UInt33)); -- { clientError 62 }"; done diff --git a/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh index a166837e01a..f38e53f898a 100755 --- a/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01710_projections_optimize_aggregation_in_order.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS in_order_agg_01710; CREATE TABLE in_order_agg_01710 diff --git a/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh index ee73974e8a4..01537524730 100755 --- a/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01710_projections_partial_optimize_aggregation_in_order.sh @@ -9,7 +9,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS in_order_agg_partial_01710; CREATE TABLE in_order_agg_partial_01710 diff --git a/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh b/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh index 2a7345f4865..f9681ebe4f5 100755 --- a/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh +++ b/tests/queries/0_stateless/01753_optimize_aggregation_in_order.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -nm -q " +$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -m -q " drop table if exists data_01753; create table data_01753 (key Int) engine=MergeTree() order by key as select * from numbers(8); select * from data_01753 group by key settings max_block_size=1; diff --git a/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh b/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh index b963f3a618f..3c9e12f780b 100755 --- a/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh +++ b/tests/queries/0_stateless/01758_optimize_skip_unused_shards_once.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --optimize_skip_unused_shards=1 -nm -q " +$CLICKHOUSE_CLIENT --optimize_skip_unused_shards=1 -m -q " create table dist_01758 as system.one engine=Distributed(test_cluster_two_shards, system, one, dummy); select * from dist_01758 where dummy = 0 format Null; " |& grep -o "StorageDistributed (dist_01758).*" diff --git a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh index 9c51b82282c..ee46f8194b9 100755 --- a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh +++ b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 -nm -q " +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 -m -q " DROP TABLE IF EXISTS tmp_01683; DROP TABLE IF EXISTS dist_01683; diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh index 4b75102e9cf..f3e8ceffff6 100755 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -13,7 +13,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function setup() { - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " drop table if exists data_01814; drop table if exists dist_01814; @@ -24,7 +24,7 @@ function setup() function cleanup() { - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " drop table data_01814; drop table dist_01814; " @@ -67,7 +67,7 @@ function test_distributed_push_down_limit_with_query_log() $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from $table group by key limit $offset, 10" - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " system flush logs; select read_rows from system.query_log where diff --git a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh index 218320772c9..2b122debff5 100755 --- a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh +++ b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_test_once() { - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS simple_key_source_table_01863; CREATE TABLE simple_key_source_table_01863 ( @@ -29,13 +29,13 @@ function run_test_once() LIFETIME(MIN 0 MAX 1000); " - prev=$($CLICKHOUSE_CLIENT -nm -q "SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1") - curr=$($CLICKHOUSE_CLIENT -nm -q " + prev=$($CLICKHOUSE_CLIENT -m -q "SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1") + curr=$($CLICKHOUSE_CLIENT -m -q " SELECT toUInt64(1) as key, dictGet('simple_key_cache_dictionary_01863', 'value', key) FORMAT Null; SELECT value FROM system.events WHERE event = 'DictCacheKeysRequestedMiss' SETTINGS system_events_show_zero_values=1 ") - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT -m -q " DROP DICTIONARY simple_key_cache_dictionary_01863; DROP TABLE simple_key_source_table_01863; " diff --git a/tests/queries/0_stateless/01872_initial_query_start_time.sh b/tests/queries/0_stateless/01872_initial_query_start_time.sh index 6a935602ea4..ff3e1954c75 100755 --- a/tests/queries/0_stateless/01872_initial_query_start_time.sh +++ b/tests/queries/0_stateless/01872_initial_query_start_time.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} -q "create table m (dummy UInt8) ENGINE = Distributed('test query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") ${CLICKHOUSE_CLIENT} -q "select * from m format Null" "--query_id=$query_id" -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " system flush logs; select anyIf(initial_query_start_time, is_initial_query) = anyIf(initial_query_start_time, not is_initial_query), diff --git a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh index 8336229a643..1294ba53e82 100755 --- a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh +++ b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET allow_experimental_bigint_types = 1; DROP TABLE IF EXISTS dictionary_decimal_source_table; diff --git a/tests/queries/0_stateless/01890_materialized_distributed_join.sh b/tests/queries/0_stateless/01890_materialized_distributed_join.sh index 88f7dcf9a69..5c04ee8b214 100755 --- a/tests/queries/0_stateless/01890_materialized_distributed_join.sh +++ b/tests/queries/0_stateless/01890_materialized_distributed_join.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists test_distributed; drop table if exists test_source; drop table if exists test_shard; diff --git a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh index 853445daf3f..a44106414ea 100755 --- a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh +++ b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS dictionary_array_source_table; CREATE TABLE dictionary_array_source_table ( diff --git a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh index 0b555cf82c2..a5c65ca87a7 100755 --- a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh +++ b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS dictionary_nullable_source_table; CREATE TABLE dictionary_nullable_source_table ( diff --git a/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh b/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh index 47d5e733480..608107c76d6 100755 --- a/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh +++ b/tests/queries/0_stateless/01927_query_views_log_matview_exceptions.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - ${CLICKHOUSE_CLIENT} -n -q " + ${CLICKHOUSE_CLIENT} -q " DROP TABLE IF EXISTS matview_exception_a_to_c; DROP TABLE IF EXISTS matview_exception_a_to_b; DROP TABLE IF EXISTS table_exception_c; @@ -17,7 +17,7 @@ function cleanup() function setup() { - ${CLICKHOUSE_CLIENT} -n -q " + ${CLICKHOUSE_CLIENT} -q " CREATE TABLE table_exception_a (a String, b Int64) ENGINE = MergeTree ORDER BY b; CREATE TABLE table_exception_b (a Float64, b Int64) ENGINE = MergeTree ORDER BY tuple(); CREATE TABLE table_exception_c (a Float64) ENGINE = MergeTree ORDER BY a; diff --git a/tests/queries/0_stateless/01947_multiple_pipe_read.sh b/tests/queries/0_stateless/01947_multiple_pipe_read.sh index 06a18a55e6e..51709eb574e 100755 --- a/tests/queries/0_stateless/01947_multiple_pipe_read.sh +++ b/tests/queries/0_stateless/01947_multiple_pipe_read.sh @@ -11,7 +11,7 @@ cat "$SAMPLE_FILE" echo '******************' echo 'Read twice from a regular file' -${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' --file "$SAMPLE_FILE" +${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' --file "$SAMPLE_FILE" echo '---' ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' --file "$SAMPLE_FILE" echo '---' @@ -19,7 +19,7 @@ ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table UNI echo '******************' echo 'Read twice from file descriptor that corresponds to a regular file' -${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' < "$SAMPLE_FILE" +${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' < "$SAMPLE_FILE" echo '---' ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' < "$SAMPLE_FILE" echo '---' From 47ed2204bd1e46ca6ef0556180372cc38fde5153 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 15 Aug 2024 12:38:24 +0000 Subject: [PATCH 1007/1722] Follow-up to ClickHouse#63898 --- .../0_stateless/00366_multi_statements.sh | 20 +++--- .../00443_preferred_block_size_bytes.sh | 4 +- ...ess_to_temporary_table_in_readonly_mode.sh | 20 +++--- ...4_performance_introspection_and_logging.sh | 4 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 6 +- ...d_optimize_skip_select_on_unused_shards.sh | 30 ++++----- ...p_select_on_unused_shards_with_prewhere.sh | 26 ++++---- ...ated_minimalistic_part_header_zookeeper.sh | 4 +- .../queries/0_stateless/00837_minmax_index.sh | 2 +- .../queries/0_stateless/00838_unique_index.sh | 2 +- .../0_stateless/00907_set_index_max_rows.sh | 2 +- .../0_stateless/00908_bloom_filter_index.sh | 10 +-- .../queries/0_stateless/00942_mutate_index.sh | 2 +- .../0_stateless/00943_materialize_index.sh | 4 +- .../00944_clear_index_in_partition.sh | 2 +- .../00964_bloom_index_string_functions.sh | 2 +- .../00965_set_index_string_functions.sh | 2 +- .../00974_primary_key_for_lowCardinality.sh | 4 +- tests/queries/0_stateless/00990_hasToken.sh | 2 +- .../02003_memory_limit_in_client.sh | 32 ++++----- .../02021_create_database_with_comment.sh | 2 +- .../02050_client_profile_events.sh | 8 +-- .../02221_parallel_replicas_bug.sh | 2 +- .../02221_system_zookeeper_unrestricted.sh | 4 +- ...2221_system_zookeeper_unrestricted_like.sh | 4 +- ...parallel_distributed_insert_select_view.sh | 6 +- ...arallel_reading_from_replicas_benchmark.sh | 4 +- .../02232_allow_only_replicated_engine.sh | 6 +- .../0_stateless/02250_ON_CLUSTER_grant.sh | 4 +- tests/queries/0_stateless/02262_column_ttl.sh | 4 +- .../0_stateless/02286_parallel_final.sh | 4 +- ..._distinct_in_order_optimization_explain.sh | 66 +++++++++---------- ..._column_ttl_expired_column_optimization.sh | 2 +- .../0_stateless/02361_fsync_profile_events.sh | 4 +- ...7_extend_protocol_with_query_parameters.sh | 10 +-- ...ting_by_input_stream_properties_explain.sh | 2 +- .../0_stateless/02417_load_marks_async.sh | 2 +- 37 files changed, 157 insertions(+), 157 deletions(-) diff --git a/tests/queries/0_stateless/00366_multi_statements.sh b/tests/queries/0_stateless/00366_multi_statements.sh index 0b2e80fe457..8546e547581 100755 --- a/tests/queries/0_stateless/00366_multi_statements.sh +++ b/tests/queries/0_stateless/00366_multi_statements.sh @@ -14,22 +14,22 @@ $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" $CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT -n --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2" -$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2;" -$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" +$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error' -$CLICKHOUSE_CLIENT -n --query="DROP TABLE IF EXISTS t_00366; CREATE TABLE t_00366 (x UInt64) ENGINE = TinyLog;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t_00366; CREATE TABLE t_00366 (x UInt64) ENGINE = TinyLog;" $CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES (1),(2),(3);" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" $CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT -n --query="INSERT INTO t_00366 VALUES (1),(2),(3);" -$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT -n --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" -$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t_00366" +$CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES (1),(2),(3);" +$CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" +$CLICKHOUSE_CLIENT --query="INSERT INTO t_00366 VALUES" <<< "(4),(5),(6)" +$CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1;" @@ -48,4 +48,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+t_00366+VALUES" -d "(7),(8),(9)" $CLICKHOUSE_CLIENT --query="SELECT * FROM t_00366" -$CLICKHOUSE_CLIENT -n --query="DROP TABLE t_00366;" +$CLICKHOUSE_CLIENT --query="DROP TABLE t_00366;" diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 27b9f5c00c7..0635fbc2a57 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -43,10 +43,10 @@ popd > /dev/null #SCRIPTDIR=`dirname "$SCRIPTPATH"` SCRIPTDIR=$SCRIPTPATH -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED rm "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout diff --git a/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh b/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh index 560b97a1d1b..5550fa69d3d 100755 --- a/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh +++ b/tests/queries/0_stateless/00543_access_to_temporary_table_in_readonly_mode.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS test_readonly; CREATE TABLE test_readonly ( ID Int @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT -n --query=" ################ # Try to create temporary table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 1; CREATE TEMPORARY TABLE readonly ( ID Int @@ -26,7 +26,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 1; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -34,7 +34,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 1; DROP TABLE test_readonly; " 2> /dev/null; @@ -46,7 +46,7 @@ CODE=$?; ################ # Try to create temporary table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 2; CREATE TEMPORARY TABLE readonly ( ID Int @@ -58,7 +58,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 2; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -66,7 +66,7 @@ CODE=$?; [ "$CODE" -ne "164" ] && [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 2; DROP TABLE test_readonly; " 2> /dev/null; @@ -78,7 +78,7 @@ CODE=$?; ################ # Try to create temporary table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 0; CREATE TEMPORARY TABLE readonly ( ID Int @@ -90,7 +90,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to insert into exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 0; INSERT INTO test_readonly (ID) VALUES (1); " 2> /dev/null; @@ -98,7 +98,7 @@ CODE=$?; [ "$CODE" -ne "0" ] && echo "Fail" && exit $CODE; # Try to drop exists (non temporary) table -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" SET readonly = 0; DROP TABLE test_readonly; " 2> /dev/null; diff --git a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index 93fd0c4a977..e9a4369a5bf 100755 --- a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -19,13 +19,13 @@ settings="$server_logs --log_queries=1 --log_query_threads=1 --log_profile_event # Test insert logging on each block and checkPacket() method -$CLICKHOUSE_CLIENT $settings -n -q " +$CLICKHOUSE_CLIENT $settings -q " DROP TABLE IF EXISTS null_00634; CREATE TABLE null_00634 (i UInt8) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple();" head -c 1000 /dev/zero | $CLICKHOUSE_CLIENT $settings --max_insert_block_size=10 --min_insert_block_size_rows=1 --min_insert_block_size_bytes=1 -q "INSERT INTO null_00634 FORMAT RowBinary" -$CLICKHOUSE_CLIENT $settings -n -q " +$CLICKHOUSE_CLIENT $settings -q " SELECT count() FROM null_00634; DROP TABLE null_00634;" diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index 96d5764780f..d69e14bdbb9 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS fetches_r1 SYNC; DROP TABLE IF EXISTS fetches_r2 SYNC" @@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate SETTINGS prefer_fetch_merged_part_time_threshold=0, \ prefer_fetch_merged_part_size_threshold=0" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); @@ -51,6 +51,6 @@ ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA fetches_r2" ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutated part ***'" ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE fetches_r1 SYNC; DROP TABLE fetches_r2 SYNC" diff --git a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh index 09f20284402..989096a26d6 100755 --- a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh +++ b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh @@ -25,83 +25,83 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM distributed WHERE a = 0 AND b | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complext expressions for constant folding - all should pass. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 1 AND a = 0 AND b = 0; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a IN (0, 1) AND b IN (0, 1); " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR a = 1 AND b = 1; " # TODO: should pass one day. -#${CLICKHOUSE_CLIENT} -n --query=" +#${CLICKHOUSE_CLIENT} --query=" # SET optimize_skip_unused_shards = 1; # SELECT count(*) FROM distributed WHERE a = 0 AND b >= 0 AND b <= 1; #" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND c = 0; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND c != 10; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 AND (a+b)*b != 12; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE (a = 0 OR a = 1) AND (b = 0 OR b = 1); " # These ones should fail. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b <= 1; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND c = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed WHERE a = 0 AND b = 0 OR c = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh index 035907bddd7..b3dff2ea69a 100755 --- a/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh +++ b/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere.sh @@ -30,73 +30,73 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM distributed_00754 PREWHERE a | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complex expressions for constant folding - all should pass. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 1 AND a = 0 WHERE b = 0; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 1 WHERE b = 1 AND length(c) = 5; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a IN (0, 1) AND b IN (0, 1) WHERE c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a IN (0, 1) WHERE b IN (0, 1) AND c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR a = 1 AND b = 1 WHERE c LIKE '%l%'; " -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE (a = 0 OR a = 1) WHERE (b = 0 OR b = 1); " # These should fail. -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b <= 1; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 WHERE c LIKE '%l%'; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" SET optimize_skip_unused_shards = 1; SELECT count(*) FROM distributed_00754 PREWHERE a = 0 AND b = 0 OR c LIKE '%l%'; " 2>&1 \ | grep -F -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 12d889a7137..8f7a1a9ae98 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) SHARD=$($CLICKHOUSE_CLIENT --query "Select getMacro('shard')") REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; @@ -62,7 +62,7 @@ do [[ $count1 == 1 && $count2 == 1 ]] && break done -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; diff --git a/tests/queries/0_stateless/00837_minmax_index.sh b/tests/queries/0_stateless/00837_minmax_index.sh index e4de0b9ebfc..ff487f50ee0 100755 --- a/tests/queries/0_stateless/00837_minmax_index.sh +++ b/tests/queries/0_stateless/00837_minmax_index.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00838_unique_index.sh b/tests/queries/0_stateless/00838_unique_index.sh index b267b6a8eb3..a3aba4f26b6 100755 --- a/tests/queries/0_stateless/00838_unique_index.sh +++ b/tests/queries/0_stateless/00838_unique_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE set_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00907_set_index_max_rows.sh b/tests/queries/0_stateless/00907_set_index_max_rows.sh index 3707aaf2ca6..bdd0f36346f 100755 --- a/tests/queries/0_stateless/00907_set_index_max_rows.sh +++ b/tests/queries/0_stateless/00907_set_index_max_rows.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE set_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00908_bloom_filter_index.sh b/tests/queries/0_stateless/00908_bloom_filter_index.sh index 25a6567b894..3bd169dd6df 100755 --- a/tests/queries/0_stateless/00908_bloom_filter_index.sh +++ b/tests/queries/0_stateless/00908_bloom_filter_index.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx3;" # NGRAM BF -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx ( k UInt64, @@ -22,7 +22,7 @@ CREATE TABLE bloom_filter_idx ORDER BY k SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx2 ( k UInt64, @@ -109,7 +109,7 @@ $CLICKHOUSE_CLIENT --optimize_or_like_chain 0 --query="SELECT count() FROM bloom # TOKEN BF -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx3 ( k UInt64, @@ -147,7 +147,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx2" $CLICKHOUSE_CLIENT --query="DROP TABLE bloom_filter_idx3" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx_na;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx_na ( na Array(Array(String)), @@ -156,7 +156,7 @@ CREATE TABLE bloom_filter_idx_na ORDER BY na" 2>&1 | grep -c 'DB::Exception: Unexpected type Array(Array(String)) of bloom filter index' # NGRAM BF with IPv6 -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_ipv6_idx ( foo IPv6, diff --git a/tests/queries/0_stateless/00942_mutate_index.sh b/tests/queries/0_stateless/00942_mutate_index.sh index 6ebb30c25b9..e1e23639e85 100755 --- a/tests/queries/0_stateless/00942_mutate_index.sh +++ b/tests/queries/0_stateless/00942_mutate_index.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00943_materialize_index.sh b/tests/queries/0_stateless/00943_materialize_index.sh index 6ff7d34a9d7..e4a585fce97 100755 --- a/tests/queries/0_stateless/00943_materialize_index.sh +++ b/tests/queries/0_stateless/00943_materialize_index.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, @@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES $CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 SETTINGS mutations_sync = 2;" $CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1 SETTINGS mutations_sync = 2;" diff --git a/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 4655077960f..a12536da239 100755 --- a/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE minmax_idx ( u64 UInt64, diff --git a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh index e2ec7fd42e4..9e410f09b13 100755 --- a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh +++ b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS bloom_filter_idx;" # NGRAM BF -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE bloom_filter_idx ( k UInt64, diff --git a/tests/queries/0_stateless/00965_set_index_string_functions.sh b/tests/queries/0_stateless/00965_set_index_string_functions.sh index 8892fb11752..0f29c3dd2f2 100755 --- a/tests/queries/0_stateless/00965_set_index_string_functions.sh +++ b/tests/queries/0_stateless/00965_set_index_string_functions.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS set_idx;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE set_idx ( k UInt64, diff --git a/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh b/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh index 389d433c7e2..ba260042f47 100755 --- a/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh +++ b/tests/queries/0_stateless/00974_primary_key_for_lowCardinality.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS lowString;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS string;" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" create table lowString ( a LowCardinality(String), @@ -18,7 +18,7 @@ ENGINE = MergeTree() PARTITION BY toYYYYMM(b) ORDER BY (a)" -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" create table string ( a String, diff --git a/tests/queries/0_stateless/00990_hasToken.sh b/tests/queries/0_stateless/00990_hasToken.sh index 6a1d4ff5ccf..d79472aa5a5 100755 --- a/tests/queries/0_stateless/00990_hasToken.sh +++ b/tests/queries/0_stateless/00990_hasToken.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -nm +python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -m diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh index 96028f4847a..15cacbff8c5 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.sh +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -4,21 +4,21 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5K' -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='5k' -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1M' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='23G' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11T' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2P' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2.1p' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10E' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10.2e' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1.1T' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1m' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='14g' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" -$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11t' -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2P' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='2.1p' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10E' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='10.2e' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1.1T' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='-1' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_NUMBER" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='1m' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='14g' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client='11t' -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" 2>&1 | grep -c -F "CANNOT_PARSE_INPUT_ASSERTION_FAILED" diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.sh b/tests/queries/0_stateless/02021_create_database_with_comment.sh index f77397dc482..d87b0794c91 100755 --- a/tests/queries/0_stateless/02021_create_database_with_comment.sh +++ b/tests/queries/0_stateless/02021_create_database_with_comment.sh @@ -20,7 +20,7 @@ function test_db_comments() local ENGINE_NAME="$1" echo "engine : ${ENGINE_NAME}" - $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm <& /dev/null +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' >& /dev/null echo $? echo 'regression test for overlap profile events snapshots between queries' -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'regression test for overlap profile events snapshots between queries (clickhouse-local)' -$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -n -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' +$CLICKHOUSE_LOCAL --print-profile-events --profile-events-delay-ms=-1 -q 'select 1; select 1' |& grep -F -o '[ 0 ] SelectedRows: 1 (increment)' echo 'print everything' profile_events="$( @@ -35,5 +35,5 @@ profile_events="$( test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" echo 'check that ProfileEvents is new for each query' -sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') +sleep_function_calls=$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select sleep(1); select 1' |& grep -c 'SleepFunctionCalls') test "$sleep_function_calls" -eq 1 && echo OK || echo "FAIL ($sleep_function_calls)" diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index 3c44a2a7ba7..a382b3859f3 100755 --- a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh +++ b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -m < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh index db94c59d2de..e23a272a4e8 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2" -${CLICKHOUSE_CLIENT} -n -q" +${CLICKHOUSE_CLIENT} -q" CREATE TABLE sample_table ( key UInt64 ) @@ -16,7 +16,7 @@ ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_ ORDER BY tuple(); " -${CLICKHOUSE_CLIENT} -n -q" +${CLICKHOUSE_CLIENT} -q" CREATE TABLE sample_table_2 ( key UInt64 ) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh index c62ec14b340..6381d811d5d 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" -${CLICKHOUSE_CLIENT} -n --query="CREATE TABLE sample_table ( +${CLICKHOUSE_CLIENT} --query="CREATE TABLE sample_table ( key UInt64 ) ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like', '1') @@ -16,7 +16,7 @@ ORDER BY tuple(); DROP TABLE IF EXISTS sample_table SYNC;" -${CLICKHOUSE_CLIENT} -n --query "CREATE TABLE sample_table_2 ( +${CLICKHOUSE_CLIENT} --query "CREATE TABLE sample_table_2 ( key UInt64 ) ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like_2', '1') diff --git a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh index 376a49fd820..63111cc32e4 100755 --- a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh +++ b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists dst_02225; drop table if exists src_02225; create table dst_02225 (key Int) engine=Memory(); @@ -14,7 +14,7 @@ create table src_02225 (key Int) engine=Memory(); insert into src_02225 values (1); " -$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -nm -q " +$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -m -q " truncate table dst_02225; insert into function remote('127.{1,2}', currentDatabase(), dst_02225, key) select * from remote('127.{1,2}', view(select * from {database:Identifier}.src_02225), key) @@ -29,7 +29,7 @@ settings parallel_distributed_insert_select=2, max_distributed_depth=1; select * from dst_02225; " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table src_02225; drop table dst_02225; " diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index bc90f4b2c11..177b373641f 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data_02226; create table data_02226 (key Int) engine=MergeTree() order by key as select * from numbers(1); @@ -24,7 +24,7 @@ opts=( $CLICKHOUSE_BENCHMARK --query "select * from remote('127.1', $CLICKHOUSE_DATABASE, data_02226)" "${opts[@]}" >& /dev/null ret=$? -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table data_02226; " diff --git a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh index d1a3825d286..e47a3033681 100755 --- a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh +++ b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh @@ -12,9 +12,9 @@ ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO us ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON Memory, TABLE ENGINE ON MergeTree, TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_memory (x UInt32) engine = Memory;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" diff --git a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh index 66417e9694a..09f9c0c8a98 100755 --- a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh +++ b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - $CLICKHOUSE_CLIENT -nmq " + $CLICKHOUSE_CLIENT -mq " DROP USER IF EXISTS with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; DROP USER IF EXISTS without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; DROP DATABASE IF EXISTS db_with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; @@ -15,7 +15,7 @@ function cleanup() cleanup trap cleanup EXIT -$CLICKHOUSE_CLIENT -nmq " +$CLICKHOUSE_CLIENT -mq " CREATE USER with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; CREATE USER without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME; diff --git a/tests/queries/0_stateless/02262_column_ttl.sh b/tests/queries/0_stateless/02262_column_ttl.sh index b5e29c9b2a1..c620d3b6d9c 100755 --- a/tests/queries/0_stateless/02262_column_ttl.sh +++ b/tests/queries/0_stateless/02262_column_ttl.sh @@ -14,7 +14,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # note, that this should be written in .sh since we need $CLICKHOUSE_DATABASE # not 'default' to catch text_log -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists ttl_02262; drop table if exists this_text_log; @@ -31,7 +31,7 @@ $CLICKHOUSE_CLIENT -nm -q " ttl_02262_uuid=$($CLICKHOUSE_CLIENT -q "select uuid from system.tables where database = '$CLICKHOUSE_DATABASE' and name = 'ttl_02262'") -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " -- OPTIMIZE TABLE x FINAL will be done in background -- attach to it's log, via table UUID in query_id (see merger/mutator code). create materialized view this_text_log engine=Memory() as diff --git a/tests/queries/0_stateless/02286_parallel_final.sh b/tests/queries/0_stateless/02286_parallel_final.sh index 0ac510208f3..47dfad42e11 100755 --- a/tests/queries/0_stateless/02286_parallel_final.sh +++ b/tests/queries/0_stateless/02286_parallel_final.sh @@ -9,7 +9,7 @@ echo "Test intersecting ranges" test_random_values() { layers=$1 - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " drop table if exists tbl_8parts_${layers}granules_rnd; create table tbl_8parts_${layers}granules_rnd (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 % 8); insert into tbl_8parts_${layers}granules_rnd select number, 1 from numbers_mt($((layers * 8 * 8192))); @@ -29,7 +29,7 @@ echo "Test non intersecting ranges" test_sequential_values() { layers=$1 - $CLICKHOUSE_CLIENT -n -q " + $CLICKHOUSE_CLIENT -q " drop table if exists tbl_8parts_${layers}granules_seq; create table tbl_8parts_${layers}granules_seq (key1 UInt32, sign Int8) engine = CollapsingMergeTree(sign) order by (key1) partition by (key1 / $((layers * 8192)))::UInt64; insert into tbl_8parts_${layers}granules_seq select number, 1 from numbers_mt($((layers * 8 * 8192))); diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index bd7e6be3987..953485c3a1f 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -23,99 +23,99 @@ $CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % num $CLICKHOUSE_CLIENT -q "select '-- disable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- enable optimize_distinct_in_order'" $CLICKHOUSE_CLIENT -q "select '-- distinct with all primary key columns -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by the same columns -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by columns are prefix of distinct columns -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column _not_ in distinct -> pre-distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column in distinct -> final distinct optimization only'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column _not_ in distinct -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT $CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by _const_ column in distinct -> ordinary distinct'" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT +$CLICKHOUSE_CLIENT -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT echo "-- Check reading in order for distinct" echo "-- disabled, distinct columns match sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -q "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "-- enabled, distinct columns match sorting key" # read_in_order_two_level_merge_threshold is set here to avoid repeating MergeTreeInOrder in output -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns DON't form prefix of sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "-- enabled, distinct columns contains constant columns, non-const columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, distinct columns contains constant columns, non-const columns match prefix of sorting key" -$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, b, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER +$CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct 1, b, a from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER echo "-- enabled, only part of distinct columns form prefix of sorting key" -$CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +$CLICKHOUSE_CLIENT --max_threads=0 -q "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT echo "=== disable new analyzer ===" DISABLE_ANALYZER="set enable_analyzer=0" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES echo "=== enable new analyzer ===" ENABLE_ANALYZER="set enable_analyzer=1" echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -q "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" diff --git a/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh b/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh index 96f80d65878..490f8361682 100755 --- a/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh +++ b/tests/queries/0_stateless/02335_column_ttl_expired_column_optimization.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) data_path="$CLICKHOUSE_TMP/local" -$CLICKHOUSE_LOCAL --path "$data_path" -nm -q " +$CLICKHOUSE_LOCAL --path "$data_path" -m -q " create table ttl_02335 ( date Date, key Int, diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 98c9cf9b7b4..73bf3fa120a 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data_fsync_pe; create table data_fsync_pe (key Int) engine=MergeTree() @@ -27,7 +27,7 @@ for i in {1..100}; do $CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data_fsync_pe values (1)" read -r FileSync FileOpen DirectorySync FileSyncElapsedMicroseconds DirectorySyncElapsedMicroseconds <<<"$( - $CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + $CLICKHOUSE_CLIENT -m --param_query_id "$query_id" -q " system flush logs; select diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh index 71e3b6961f8..46396d38747 100755 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT \ table_name="t_02377_extend_protocol_with_query_parameters_$RANDOM$RANDOM" -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " create table $table_name( id Int64, arr Array(UInt8), @@ -57,17 +57,17 @@ $CLICKHOUSE_CLIENT \ # it is possible to set parameter for the current session -$CLICKHOUSE_CLIENT -n -q "set param_n = 42; select {n: UInt8}" +$CLICKHOUSE_CLIENT -q "set param_n = 42; select {n: UInt8}" # and it will not be visible to other sessions -$CLICKHOUSE_CLIENT -n -q "select {n: UInt8} -- { serverError 456 }" +$CLICKHOUSE_CLIENT -q "select {n: UInt8} -- { serverError 456 }" # the same parameter could be set multiple times within one session (new value overrides the previous one) -$CLICKHOUSE_CLIENT -n -q "set param_n = 12; set param_n = 13; select {n: UInt8}" +$CLICKHOUSE_CLIENT -q "set param_n = 12; set param_n = 13; select {n: UInt8}" # multiple different parameters could be defined within each session -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " set param_a = 13, param_b = 'str'; set param_c = '2022-08-04 18:30:53'; set param_d = '{\'10\': [11, 12], \'13\': [14, 15]}'; diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index 4b9793da5bb..974f10e2f24 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -15,7 +15,7 @@ FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES" function explain_sorting { echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTING + $CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -q "$1" | eval $FIND_SORTING } function explain_sortmode { diff --git a/tests/queries/0_stateless/02417_load_marks_async.sh b/tests/queries/0_stateless/02417_load_marks_async.sh index 950656e7ab6..bcede9e4f5e 100755 --- a/tests/queries/0_stateless/02417_load_marks_async.sh +++ b/tests/queries/0_stateless/02417_load_marks_async.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test;" -${CLICKHOUSE_CLIENT} -n -q " +${CLICKHOUSE_CLIENT} -q " CREATE TABLE test ( n0 UInt64, From 0b58cbca3836a9dbf2babb9a0c2d7ed89d44b90b Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:02:25 +0000 Subject: [PATCH 1008/1722] Fix submodules --- .gitmodules | 3 +++ contrib/usearch | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/usearch diff --git a/.gitmodules b/.gitmodules index 26778cda171..cdee6a43ad8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -336,6 +336,9 @@ [submodule "contrib/incbin"] path = contrib/incbin url = https://github.com/graphitemaster/incbin.git +[submodule "contrib/usearch"] + path = contrib/usearch + url = https://github.com/ClickHouse/usearch.git [submodule "contrib/SimSIMD"] path = contrib/SimSIMD url = https://github.com/ashvardanian/SimSIMD.git diff --git a/contrib/usearch b/contrib/usearch new file mode 160000 index 00000000000..e21a5778a0d --- /dev/null +++ b/contrib/usearch @@ -0,0 +1 @@ +Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4 From 53bff26f11301781f92a3005c9a08fe176ff0bda Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:09:39 +0000 Subject: [PATCH 1009/1722] Try to fix submodule --- contrib/libprotobuf-mutator | 2 +- contrib/libunwind | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index 1f95f808306..b922c8ab900 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d +Subproject commit b922c8ab9004ef9944982e4f165e2747b13223fa diff --git a/contrib/libunwind b/contrib/libunwind index a89d904befe..601db0b0e03 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 +Subproject commit 601db0b0e03018c01710470a37703b618f9cf08b From ef1f0e2aafc68d312cfc793aa4652e81e58c199b Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:16:09 +0000 Subject: [PATCH 1010/1722] Fix typo in docs --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 78e076a7427..94468066372 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -20,7 +20,7 @@ CREATE TABLE iceberg_table_azure ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) CREATE TABLE iceberg_table_local - ENGINE = ICebergLocal(path_to_table, [,format] [,compression_method]) + ENGINE = IcebergLocal(path_to_table, [,format] [,compression_method]) ``` **Engine arguments** From 9fec833d95a8d2d2727ab7d16d4de89049db82a2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Aug 2024 13:42:57 +0000 Subject: [PATCH 1011/1722] fix test 03221_mutation_analyzer_skip_part --- .../03221_mutate_profile_events.sh | 53 +++++++++++++++++++ .../03221_mutate_profile_events.sql | 33 ------------ 2 files changed, 53 insertions(+), 33 deletions(-) create mode 100755 tests/queries/0_stateless/03221_mutate_profile_events.sh delete mode 100644 tests/queries/0_stateless/03221_mutate_profile_events.sql diff --git a/tests/queries/0_stateless/03221_mutate_profile_events.sh b/tests/queries/0_stateless/03221_mutate_profile_events.sh new file mode 100755 index 00000000000..3758db905e0 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutate_profile_events.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash +# Tags: no-random-settings, no-random-merge-tree-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query " + DROP TABLE IF EXISTS t_mutate_profile_events; + + CREATE TABLE t_mutate_profile_events (key UInt64, id UInt64, v1 UInt64, v2 UInt64) + ENGINE = MergeTree ORDER BY id PARTITION BY key + SETTINGS min_bytes_for_wide_part = 0; + + INSERT INTO t_mutate_profile_events SELECT 1, number, number, number FROM numbers(10000); + INSERT INTO t_mutate_profile_events SELECT 2, number, number, number FROM numbers(10000); + + SET mutations_sync = 2; + + ALTER TABLE t_mutate_profile_events UPDATE v1 = 1000 WHERE key = 1; + ALTER TABLE t_mutate_profile_events DELETE WHERE key = 2 AND v2 % 10 = 0; +" + +# Mutation query may return before the entry is added to part log. +# So, we may have to retry the flush of logs until all entries are actually flushed. +for _ in {1..10}; do + ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + res=$(${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.part_log WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart'") + + if [[ $res -eq 4 ]]; then + break + fi + + sleep 2.0 +done + +${CLICKHOUSE_CLIENT} --query " + SELECT + splitByChar('_', part_name)[-1] AS version, + sum(ProfileEvents['MutationTotalParts']), + sum(ProfileEvents['MutationUntouchedParts']), + sum(ProfileEvents['MutatedRows']), + sum(ProfileEvents['MutatedUncompressedBytes']), + sum(ProfileEvents['MutationAllPartColumns']), + sum(ProfileEvents['MutationSomePartColumns']), + sum(ProfileEvents['MutationTotalMilliseconds']) > 0, + sum(ProfileEvents['MutationExecuteMilliseconds']) > 0, + FROM system.part_log + WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart' + GROUP BY version ORDER BY version; + + DROP TABLE IF EXISTS t_mutate_profile_events; +" diff --git a/tests/queries/0_stateless/03221_mutate_profile_events.sql b/tests/queries/0_stateless/03221_mutate_profile_events.sql deleted file mode 100644 index e9f7f9670bd..00000000000 --- a/tests/queries/0_stateless/03221_mutate_profile_events.sql +++ /dev/null @@ -1,33 +0,0 @@ --- Tags: no-random-settings, no-random-merge-tree-settings - -DROP TABLE IF EXISTS t_mutate_profile_events; - -CREATE TABLE t_mutate_profile_events (key UInt64, id UInt64, v1 UInt64, v2 UInt64) -ENGINE = MergeTree ORDER BY id PARTITION BY key -SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO t_mutate_profile_events SELECT 1, number, number, number FROM numbers(10000); -INSERT INTO t_mutate_profile_events SELECT 2, number, number, number FROM numbers(10000); - -SET mutations_sync = 2; - -ALTER TABLE t_mutate_profile_events UPDATE v1 = 1000 WHERE key = 1; -ALTER TABLE t_mutate_profile_events DELETE WHERE key = 2 AND v2 % 10 = 0; - -SYSTEM FLUSH LOGS; - -SELECT - splitByChar('_', part_name)[-1] AS version, - sum(ProfileEvents['MutationTotalParts']), - sum(ProfileEvents['MutationUntouchedParts']), - sum(ProfileEvents['MutatedRows']), - sum(ProfileEvents['MutatedUncompressedBytes']), - sum(ProfileEvents['MutationAllPartColumns']), - sum(ProfileEvents['MutationSomePartColumns']), - sum(ProfileEvents['MutationTotalMilliseconds']) > 0, - sum(ProfileEvents['MutationExecuteMilliseconds']) > 0, -FROM system.part_log -WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart' -GROUP BY version ORDER BY version; - -DROP TABLE IF EXISTS t_mutate_profile_events From fb037bcc722939f8d01fbd63c155a9a816c83f94 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Thu, 15 Aug 2024 13:48:43 +0000 Subject: [PATCH 1012/1722] move to mergetree setting and add more info --- .../operations/settings/merge-tree-settings.md | 17 +++++++++++++++-- docs/en/sql-reference/statements/delete.md | 3 +-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index a3bd919d3ce..a13aacc76e6 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1042,10 +1042,23 @@ Compression rates of LZ4 or ZSTD improve on average by 20-40%. This setting works best for tables with no primary key or a low-cardinality primary key, i.e. a table with only few distinct primary key values. High-cardinality primary keys, e.g. involving timestamp columns of type `DateTime64`, are not expected to benefit from this setting. -### deduplicate_merge_projection_mode +## lightweight_mutation_projection_mode + +By default, lightweight delete `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation. So the default value would be `throw`. +However, this option can change the behavior. With the value either `drop` or `rebuild`, deletes will work with projections. `drop` would delete the projection so it might be fast in the current query as projection gets deleted but slow in future queries as no projection attached. +`rebuild` would rebuild the projection which might affect the performance of the current query, but might speedup for future queries. A good thing is that these options would only work in the part level, +which means projections in the part that don't get touched would stay intact instead of triggering any action like drop or rebuild. + +Possible values: + +- throw, drop, rebuild + +Default value: throw + +## deduplicate_merge_projection_mode Whether to allow create projection for the table with non-classic MergeTree, that is not (Replicated, Shared) MergeTree. If allowed, what is the action when merge projections, either drop or rebuild. So classic MergeTree would ignore this setting. -It also controls `OPTIMIZE DEDUPLICATE` as well, but has effect on all MergeTree family members. +It also controls `OPTIMIZE DEDUPLICATE` as well, but has effect on all MergeTree family members. Similar to the option `lightweight_mutation_projection_mode`, it is also part level. Possible values: diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 88a9c933519..78142f880fe 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -38,8 +38,7 @@ If you anticipate frequent deletes, consider using a [custom partitioning key](/ ### Lightweight `DELETE`s with projections -By default, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation and may require the projection to be rebuilt, negatively affecting `DELETE` performance. -However, there is an option to change this behavior. By changing setting `lightweight_mutation_projection_mode = 'drop'`, deletes will work with projections. +By default, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation. But there is a [MergeTree setting](https://clickhouse.com/docs/en/operations/settings/merge-tree-settings) `lightweight_mutation_projection_mode` can change the behavior. ## Performance considerations when using lightweight `DELETE` From bdd0e01545a93b8bda34f667ede94f0e0faaa665 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 15 Aug 2024 14:06:21 +0200 Subject: [PATCH 1013/1722] CI: Auto release workflow --- .github/actions/debug/action.yml | 18 +++++ .github/workflows/auto_releases.yml | 109 +++++++++++++++++++++++++++ .github/workflows/create_release.yml | 21 ++++++ 3 files changed, 148 insertions(+) create mode 100644 .github/actions/debug/action.yml create mode 100644 .github/workflows/auto_releases.yml diff --git a/.github/actions/debug/action.yml b/.github/actions/debug/action.yml new file mode 100644 index 00000000000..e1fe3f28024 --- /dev/null +++ b/.github/actions/debug/action.yml @@ -0,0 +1,18 @@ +name: DebugInfo +description: Prints workflow debug info + +runs: + using: "composite" + steps: + - name: Print envs + shell: bash + run: | + echo "::group::Envs" + env + echo "::endgroup::" + - name: Print Event.json + shell: bash + run: | + echo "::group::Event.json" + python3 -m json.tool "$GITHUB_EVENT_PATH" + echo "::endgroup::" diff --git a/.github/workflows/auto_releases.yml b/.github/workflows/auto_releases.yml new file mode 100644 index 00000000000..c159907187c --- /dev/null +++ b/.github/workflows/auto_releases.yml @@ -0,0 +1,109 @@ +name: AutoReleases + +env: + PYTHONUNBUFFERED: 1 + +concurrency: + group: autoreleases + +on: + # schedule: + # - cron: '0 9 * * *' + workflow_dispatch: + inputs: + dry-run: + description: 'Dry run' + required: false + default: true + type: boolean + +jobs: + AutoReleaseInfo: + runs-on: [self-hosted, style-checker-aarch64] + outputs: + data: ${{ steps.info.outputs.AUTO_RELEASE_PARAMS }} + dry_run: ${{ steps.info.outputs.DRY_RUN }} + steps: + - name: Debug Info + uses: ./.github/actions/debug + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" + - name: Check out repository code + uses: ClickHouse/checkout@v1 + - name: Prepare Info + id: info + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --prepare + echo "::group::Auto Release Info" + python3 -m json.tool /tmp/autorelease_info.json + echo "::endgroup::" + { + echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_ENV" + { + echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_OUTPUT" + echo "DRY_RUN=true" >> "$GITHUB_OUTPUT" + - name: Post Release Branch statuses + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-status + - name: Clean up + uses: ./.github/actions/clean + + Release_0: + needs: AutoReleaseInfo + name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].release_branch }} + if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].ready }} + uses: ./.github/workflows/create_release.yml + with: + ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].commit_sha }} + type: patch + dry-run: ${{ needs.AutoReleaseInfo.outputs.dry_run }} +# +# Release_1: +# needs: [AutoReleaseInfo, Release_0] +# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].release_branch }} +# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].ready }} +# uses: ./.github/workflows/create_release.yml +# with: +# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].commit_sha }} +# type: patch +# dry-run: ${{ env.DRY_RUN }} +# +# Release_2: +# needs: [AutoReleaseInfo, Release_1] +# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[2].release_branch }} +# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[2].ready }} +# uses: ./.github/workflow/create_release.yml +# with: +# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].commit_sha }} +# type: patch +# dry-run: ${{ env.DRY_RUN }} +# +# Release_3: +# needs: [AutoReleaseInfo, Release_2] +# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].release_branch }} +# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].ready }} +# uses: ./.github/workflow/create_release.yml +# with: +# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].commit_sha }} +# type: patch +# dry-run: ${{ env.DRY_RUN }} + +# - name: Post Slack Message +# if: ${{ !cancelled() }} +# run: | +# cd "$GITHUB_WORKSPACE/tests/ci" +# python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index eb16c25f604..1553d689227 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -2,6 +2,7 @@ name: CreateRelease concurrency: group: release + 'on': workflow_dispatch: inputs: @@ -26,6 +27,26 @@ concurrency: required: false default: false type: boolean + workflow_call: + inputs: + ref: + description: 'Git reference (branch or commit sha) from which to create the release' + required: true + type: string + type: + description: 'The type of release: "new" for a new release or "patch" for a patch release' + required: true + type: string + only-repo: + description: 'Run only repos updates including docker (repo-recovery, tests)' + required: false + default: false + type: boolean + dry-run: + description: 'Dry run' + required: false + default: false + type: boolean jobs: CreateRelease: From 7d01c3131265d0dfae24fbf6ab91c71073573765 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 16:01:13 +0200 Subject: [PATCH 1014/1722] Delete old code of named collections --- src/Core/PostgreSQL/PoolWithFailover.cpp | 2 +- src/Core/PostgreSQL/PoolWithFailover.h | 9 +- src/Dictionaries/HTTPDictionarySource.cpp | 26 +- src/Dictionaries/MongoDBDictionarySource.cpp | 61 ++-- .../PostgreSQLDictionarySource.cpp | 131 ++++++-- .../ExternalDataSourceConfiguration.cpp | 288 ------------------ .../ExternalDataSourceConfiguration.h | 92 ------ src/Storages/NamedCollectionsHelpers.h | 2 +- src/Storages/StorageExternalDistributed.h | 2 - src/TableFunctions/TableFunctionMongoDB.cpp | 1 - src/TableFunctions/TableFunctionRedis.cpp | 1 - 11 files changed, 170 insertions(+), 445 deletions(-) delete mode 100644 src/Storages/ExternalDataSourceConfiguration.cpp delete mode 100644 src/Storages/ExternalDataSourceConfiguration.h diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 5014564dbe0..054fc3b2226 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -23,7 +23,7 @@ namespace postgres { PoolWithFailover::PoolWithFailover( - const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority, + const ReplicasConfigurationByPriority & configurations_by_priority, size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_, diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index 502a9a9b7d7..2237c752367 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -8,7 +8,6 @@ #include "ConnectionHolder.h" #include #include -#include #include @@ -20,12 +19,12 @@ namespace postgres class PoolWithFailover { - -using RemoteDescription = std::vector>; - public: + using ReplicasConfigurationByPriority = std::map>; + using RemoteDescription = std::vector>; + PoolWithFailover( - const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority, + const ReplicasConfigurationByPriority & configurations_by_priority, size_t pool_size, size_t pool_wait_timeout, size_t max_tries_, diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 663c63dd6c6..d6df03b39df 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -8,12 +8,12 @@ #include #include #include -#include #include #include #include "DictionarySourceFactory.h" #include "DictionarySourceHelpers.h" #include "DictionaryStructure.h" +#include #include "registerDictionaries.h" @@ -223,21 +223,23 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) String endpoint; String format; - auto named_collection = created_from_ddl - ? getURLBasedDataSourceConfiguration(config, settings_config_prefix, global_context) - : std::nullopt; + auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix, global_context) : nullptr; if (named_collection) { - url = named_collection->configuration.url; - endpoint = named_collection->configuration.endpoint; - format = named_collection->configuration.format; + validateNamedCollection( + *named_collection, + /* required_keys */{}, + /* optional_keys */ValidateKeysMultiset{ + "url", "endpoint", "user", "credentials.user", "password", "credentials.password", "format", "compression_method", "structure", "name"}); - credentials.setUsername(named_collection->configuration.user); - credentials.setPassword(named_collection->configuration.password); + url = named_collection->getOrDefault("url", ""); + endpoint = named_collection->getOrDefault("endpoint", ""); + format = named_collection->getOrDefault("format", ""); - header_entries.reserve(named_collection->configuration.headers.size()); - for (const auto & [key, value] : named_collection->configuration.headers) - header_entries.emplace_back(key, value); + credentials.setUsername(named_collection->getAnyOrDefault({"user", "credentials.user"}, "")); + credentials.setPassword(named_collection->getAnyOrDefault({"password", "credentials.password"}, "")); + + header_entries = getHeadersFromNamedCollection(*named_collection); } else { diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index c30a6f90e44..7bacfdab3d2 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -1,15 +1,12 @@ #include "MongoDBDictionarySource.h" #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" -#include #include +#include namespace DB { -static const std::unordered_set dictionary_allowed_keys = { - "host", "port", "user", "password", "db", "database", "uri", "collection", "name", "method", "options"}; - void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) { auto create_mongo_db_dictionary = []( @@ -22,35 +19,53 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) bool created_from_ddl) { const auto config_prefix = root_config_prefix + ".mongodb"; - ExternalDataSourceConfiguration configuration; - auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); }; - auto named_collection = getExternalDataSourceConfiguration(config, config_prefix, context, has_config_key); + auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, config_prefix, context) : nullptr; + + String host, username, password, database, method, options, collection; + UInt16 port; if (named_collection) { - configuration = named_collection->configuration; + validateNamedCollection( + *named_collection, + /* required_keys */{"collection"}, + /* optional_keys */ValidateKeysMultiset{ + "host", "port", "user", "password", "db", "database", "uri", "name", "method", "options"}); + + host = named_collection->getOrDefault("host", ""); + port = static_cast(named_collection->getOrDefault("port", 0)); + username = named_collection->getOrDefault("user", ""); + password = named_collection->getOrDefault("password", ""); + database = named_collection->getAnyOrDefault({"db", "database"}, ""); + method = named_collection->getOrDefault("method", ""); + collection = named_collection->getOrDefault("collection", ""); + options = named_collection->getOrDefault("options", ""); } else { - configuration.host = config.getString(config_prefix + ".host", ""); - configuration.port = config.getUInt(config_prefix + ".port", 0); - configuration.username = config.getString(config_prefix + ".user", ""); - configuration.password = config.getString(config_prefix + ".password", ""); - configuration.database = config.getString(config_prefix + ".db", ""); + host = config.getString(config_prefix + ".host", ""); + port = config.getUInt(config_prefix + ".port", 0); + username = config.getString(config_prefix + ".user", ""); + password = config.getString(config_prefix + ".password", ""); + database = config.getString(config_prefix + ".db", ""); + method = config.getString(config_prefix + ".method", ""); + collection = config.getString(config_prefix + ".collection"); + options = config.getString(config_prefix + ".options", ""); } if (created_from_ddl) - context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + context->getRemoteHostFilter().checkHostAndPort(host, toString(port)); - return std::make_unique(dict_struct, + return std::make_unique( + dict_struct, config.getString(config_prefix + ".uri", ""), - configuration.host, - configuration.port, - configuration.username, - configuration.password, - config.getString(config_prefix + ".method", ""), - configuration.database, - config.getString(config_prefix + ".collection"), - config.getString(config_prefix + ".options", ""), + host, + port, + username, + password, + method, + database, + collection, + options, sample_block); }; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index f62a9a009d8..fd026a97cd4 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -13,7 +13,7 @@ #include "readInvalidateQuery.h" #include #include -#include +#include #include #endif @@ -30,7 +30,7 @@ namespace ErrorCodes static const UInt64 max_block_size = 8192; -static const std::unordered_set dictionary_allowed_keys = { +static const ValidateKeysMultiset dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", "schema", "update_field", "update_lag", "invalidate_query", "query", "where", "name", "priority"}; @@ -179,6 +179,19 @@ std::string PostgreSQLDictionarySource::toString() const #endif +static void validateConfigKeys( + const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix) +{ + Poco::Util::AbstractConfiguration::Keys config_keys; + dict_config.keys(config_prefix, config_keys); + for (const auto & config_key : config_keys) + { + if (dictionary_allowed_keys.contains(config_key) || startsWith(config_key, "replica")) + continue; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); + } +} + void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, @@ -191,38 +204,118 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) { #if USE_LIBPQXX const auto settings_config_prefix = config_prefix + ".postgresql"; - auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key) || key.starts_with("replica"); }; - auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context, has_config_key); const auto & settings = context->getSettingsRef(); + std::optional dictionary_configuration; + String database, schema, table; + postgres::PoolWithFailover::ReplicasConfigurationByPriority replicas_by_priority; + + auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix, context) : nullptr; + if (named_collection) + { + validateNamedCollection>(*named_collection, {}, dictionary_allowed_keys); + + StoragePostgreSQL::Configuration common_configuration; + common_configuration.host = named_collection->getOrDefault("host", ""); + common_configuration.port = named_collection->getOrDefault("port", 0); + common_configuration.username = named_collection->getOrDefault("user", ""); + common_configuration.password = named_collection->getOrDefault("password", ""); + common_configuration.database = named_collection->getAnyOrDefault({"database", "db"}, ""); + common_configuration.schema = named_collection->getOrDefault("schema", ""); + common_configuration.table = named_collection->getOrDefault("table", ""); + + dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration{ + .db = common_configuration.database, + .schema = common_configuration.schema, + .table = common_configuration.table, + .query = named_collection->getOrDefault("query", ""), + .where = named_collection->getOrDefault("where", ""), + .invalidate_query = named_collection->getOrDefault("invalidate_query", ""), + .update_field = named_collection->getOrDefault("update_field", ""), + .update_lag = named_collection->getOrDefault("update_lag", 1), + }); + + replicas_by_priority[0].emplace_back(common_configuration); + } + else + { + validateConfigKeys(config, settings_config_prefix); + + StoragePostgreSQL::Configuration common_configuration; + common_configuration.host = config.getString(settings_config_prefix + ".host", ""); + common_configuration.port = config.getUInt(settings_config_prefix + ".port", 0); + common_configuration.username = config.getString(settings_config_prefix + ".user", ""); + common_configuration.password = config.getString(settings_config_prefix + ".password", ""); + common_configuration.database = config.getString(fmt::format("{}.database", settings_config_prefix), config.getString(fmt::format("{}.db", settings_config_prefix), "")); + common_configuration.schema = config.getString(fmt::format("{}.schema", settings_config_prefix), ""); + common_configuration.table = config.getString(fmt::format("{}.table", settings_config_prefix), ""); + + dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration + { + .db = common_configuration.database, + .schema = common_configuration.schema, + .table = common_configuration.table, + .query = config.getString(fmt::format("{}.query", settings_config_prefix), ""), + .where = config.getString(fmt::format("{}.where", settings_config_prefix), ""), + .invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""), + .update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""), + .update_lag = config.getUInt64(fmt::format("{}.update_lag", settings_config_prefix), 1) + }); + + + if (config.has(settings_config_prefix + ".replica")) + { + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(settings_config_prefix, config_keys); + + for (const auto & config_key : config_keys) + { + if (config_key.starts_with("replica")) + { + String replica_name = settings_config_prefix + "." + config_key; + StoragePostgreSQL::Configuration replica_configuration{common_configuration}; + + size_t priority = config.getInt(replica_name + ".priority", 0); + replica_configuration.host = config.getString(replica_name + ".host", common_configuration.host); + replica_configuration.port = config.getUInt(replica_name + ".port", common_configuration.port); + replica_configuration.username = config.getString(replica_name + ".user", common_configuration.username); + replica_configuration.password = config.getString(replica_name + ".password", common_configuration.password); + + if (replica_configuration.host.empty() || replica_configuration.port == 0 + || replica_configuration.username.empty() || replica_configuration.password.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Named collection of connection parameters is missing some " + "of the parameters and no other dictionary parameters are added"); + } + + replicas_by_priority[priority].emplace_back(replica_configuration); + } + } + } + else + { + replicas_by_priority[0].emplace_back(common_configuration); + } + } if (created_from_ddl) { - for (const auto & replicas : configuration.replicas_configurations) - for (const auto & replica : replicas.second) + for (const auto & [_, replicas] : replicas_by_priority) + for (const auto & replica : replicas) context->getRemoteHostFilter().checkHostAndPort(replica.host, toString(replica.port)); } + auto pool = std::make_shared( - configuration.replicas_configurations, + replicas_by_priority, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, settings.postgresql_connection_pool_retries, settings.postgresql_connection_pool_auto_close_connection, settings.postgresql_connection_attempt_timeout); - PostgreSQLDictionarySource::Configuration dictionary_configuration - { - .db = configuration.database, - .schema = configuration.schema, - .table = configuration.table, - .query = config.getString(fmt::format("{}.query", settings_config_prefix), ""), - .where = config.getString(fmt::format("{}.where", settings_config_prefix), ""), - .invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""), - .update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""), - .update_lag = config.getUInt64(fmt::format("{}.update_lag", settings_config_prefix), 1) - }; - return std::make_unique(dict_struct, dictionary_configuration, pool, sample_block); + return std::make_unique(dict_struct, dictionary_configuration.value(), pool, sample_block); #else (void)dict_struct; (void)config; diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp deleted file mode 100644 index 41979f8d91c..00000000000 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ /dev/null @@ -1,288 +0,0 @@ -#include "ExternalDataSourceConfiguration.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) - -static const std::unordered_set dictionary_allowed_keys = { - "host", "port", "user", "password", "quota_key", "db", - "database", "table", "schema", "replica", - "update_field", "update_lag", "invalidate_query", "query", - "where", "name", "secure", "uri", "collection"}; - - -template -SettingsChanges getSettingsChangesFromConfig( - const BaseSettings & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) -{ - SettingsChanges config_settings; - for (const auto & setting : settings.all()) - { - const auto & setting_name = setting.getName(); - auto setting_value = config.getString(config_prefix + '.' + setting_name, ""); - if (!setting_value.empty()) - config_settings.emplace_back(setting_name, setting_value); - } - return config_settings; -} - - -String ExternalDataSourceConfiguration::toString() const -{ - WriteBufferFromOwnString configuration_info; - configuration_info << "username: " << username << "\t"; - if (addresses.empty()) - { - configuration_info << "host: " << host << "\t"; - configuration_info << "port: " << port << "\t"; - } - else - { - for (const auto & [replica_host, replica_port] : addresses) - { - configuration_info << "host: " << replica_host << "\t"; - configuration_info << "port: " << replica_port << "\t"; - } - } - return configuration_info.str(); -} - - -void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration & conf) -{ - host = conf.host; - port = conf.port; - username = conf.username; - password = conf.password; - quota_key = conf.quota_key; - database = conf.database; - table = conf.table; - schema = conf.schema; - addresses = conf.addresses; - addresses_expr = conf.addresses_expr; -} - - -static void validateConfigKeys( - const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, HasConfigKeyFunc has_config_key_func) -{ - Poco::Util::AbstractConfiguration::Keys config_keys; - dict_config.keys(config_prefix, config_keys); - for (const auto & config_key : config_keys) - { - if (!has_config_key_func(config_key)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); - } -} - -template -std::optional getExternalDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, - ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings & settings) -{ - validateConfigKeys(dict_config, dict_config_prefix, has_config_key); - ExternalDataSourceConfiguration configuration; - - auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); - if (!collection_name.empty()) - { - const auto & config = context->getConfigRef(); - const auto & collection_prefix = fmt::format("named_collections.{}", collection_name); - validateConfigKeys(dict_config, collection_prefix, has_config_key); - auto config_settings = getSettingsChangesFromConfig(settings, config, collection_prefix); - auto dict_settings = getSettingsChangesFromConfig(settings, dict_config, dict_config_prefix); - /// dictionary config settings override collection settings. - config_settings.insert(config_settings.end(), dict_settings.begin(), dict_settings.end()); - - if (!config.has(collection_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); - - configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(collection_prefix + ".host", "")); - configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0)); - configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", "")); - configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", "")); - configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", config.getString(collection_prefix + ".quota_key", "")); - configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database", - config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", "")))); - configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", "")); - configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", "")); - - if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.table.empty()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Named collection of connection parameters is missing some " - "of the parameters and dictionary parameters are not added"); - } - return ExternalDataSourceInfo{.configuration = configuration, .settings_changes = config_settings}; - } - return std::nullopt; -} - -std::optional getURLBasedDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) -{ - URLBasedDataSourceConfiguration configuration; - auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); - if (!collection_name.empty()) - { - const auto & config = context->getConfigRef(); - const auto & collection_prefix = fmt::format("named_collections.{}", collection_name); - - if (!config.has(collection_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); - - configuration.url = - dict_config.getString(dict_config_prefix + ".url", config.getString(collection_prefix + ".url", "")); - configuration.endpoint = - dict_config.getString(dict_config_prefix + ".endpoint", config.getString(collection_prefix + ".endpoint", "")); - configuration.format = - dict_config.getString(dict_config_prefix + ".format", config.getString(collection_prefix + ".format", "")); - configuration.compression_method = - dict_config.getString(dict_config_prefix + ".compression", config.getString(collection_prefix + ".compression_method", "")); - configuration.structure = - dict_config.getString(dict_config_prefix + ".structure", config.getString(collection_prefix + ".structure", "")); - configuration.user = - dict_config.getString(dict_config_prefix + ".credentials.user", config.getString(collection_prefix + ".credentials.user", "")); - configuration.password = - dict_config.getString(dict_config_prefix + ".credentials.password", config.getString(collection_prefix + ".credentials.password", "")); - - String headers_prefix; - const Poco::Util::AbstractConfiguration *headers_config = nullptr; - if (dict_config.has(dict_config_prefix + ".headers")) - { - headers_prefix = dict_config_prefix + ".headers"; - headers_config = &dict_config; - } - else - { - headers_prefix = collection_prefix + ".headers"; - headers_config = &config; - } - - if (headers_config) - { - Poco::Util::AbstractConfiguration::Keys header_keys; - headers_config->keys(headers_prefix, header_keys); - headers_prefix += "."; - for (const auto & header : header_keys) - { - const auto header_prefix = headers_prefix + header; - configuration.headers.emplace_back( - headers_config->getString(header_prefix + ".name"), - headers_config->getString(header_prefix + ".value")); - } - } - - return URLBasedDataSourceConfig{ .configuration = configuration }; - } - - return std::nullopt; -} - -ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key) -{ - validateConfigKeys(dict_config, dict_config_prefix, has_config_key); - ExternalDataSourceConfiguration common_configuration; - - auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context, has_config_key); - if (named_collection) - { - common_configuration = named_collection->configuration; - } - else - { - common_configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); - common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); - common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); - common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); - common_configuration.quota_key = dict_config.getString(dict_config_prefix + ".quota_key", ""); - common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", "")); - common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); - common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); - } - - ExternalDataSourcesByPriority configuration - { - .database = common_configuration.database, - .table = common_configuration.table, - .schema = common_configuration.schema, - .replicas_configurations = {} - }; - - if (dict_config.has(dict_config_prefix + ".replica")) - { - Poco::Util::AbstractConfiguration::Keys config_keys; - dict_config.keys(dict_config_prefix, config_keys); - - for (const auto & config_key : config_keys) - { - if (config_key.starts_with("replica")) - { - ExternalDataSourceConfiguration replica_configuration(common_configuration); - String replica_name = dict_config_prefix + "." + config_key; - validateConfigKeys(dict_config, replica_name, has_config_key); - - size_t priority = dict_config.getInt(replica_name + ".priority", 0); - replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); - replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port); - replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username); - replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password); - replica_configuration.quota_key = dict_config.getString(replica_name + ".quota_key", common_configuration.quota_key); - - if (replica_configuration.host.empty() || replica_configuration.port == 0 - || replica_configuration.username.empty() || replica_configuration.password.empty()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Named collection of connection parameters is missing some " - "of the parameters and no other dictionary parameters are added"); - } - - configuration.replicas_configurations[priority].emplace_back(replica_configuration); - } - } - } - else - { - configuration.replicas_configurations[0].emplace_back(common_configuration); - } - - return configuration; -} - - -void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration & conf) -{ - url = conf.url; - format = conf.format; - compression_method = conf.compression_method; - structure = conf.structure; - http_method = conf.http_method; - headers = conf.headers; -} - -template -std::optional getExternalDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, - ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings & settings); - -template -SettingsChanges getSettingsChangesFromConfig( - const BaseSettings & settings, const Poco::Util::AbstractConfiguration & config, const String & config_prefix); - -} diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h deleted file mode 100644 index c703c9ce999..00000000000 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ /dev/null @@ -1,92 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -#define EMPTY_SETTINGS(M, ALIAS) -DECLARE_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) - -struct EmptySettings : public BaseSettings {}; - -struct ExternalDataSourceConfiguration -{ - String host; - UInt16 port = 0; - String username = "default"; - String password; - String quota_key; - String database; - String table; - String schema; - - std::vector> addresses; /// Failover replicas. - String addresses_expr; - - String toString() const; - - void set(const ExternalDataSourceConfiguration & conf); -}; - - -using StorageSpecificArgs = std::vector>; - -struct ExternalDataSourceInfo -{ - ExternalDataSourceConfiguration configuration; - SettingsChanges settings_changes; -}; - -using HasConfigKeyFunc = std::function; - -template -std::optional getExternalDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, - ContextPtr context, HasConfigKeyFunc has_config_key, const BaseSettings & settings = {}); - - -/// Highest priority is 0, the bigger the number in map, the less the priority. -using ExternalDataSourcesConfigurationByPriority = std::map>; - -struct ExternalDataSourcesByPriority -{ - String database; - String table; - String schema; - ExternalDataSourcesConfigurationByPriority replicas_configurations; -}; - -ExternalDataSourcesByPriority -getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key); - -struct URLBasedDataSourceConfiguration -{ - String url; - String endpoint; - String format = "auto"; - String compression_method = "auto"; - String structure = "auto"; - - String user; - String password; - - HTTPHeaderEntries headers; - String http_method; - - void set(const URLBasedDataSourceConfiguration & conf); -}; - -struct URLBasedDataSourceConfig -{ - URLBasedDataSourceConfiguration configuration; -}; - -std::optional getURLBasedDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); - -} diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index f444a581eb6..bf2da7235a2 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -133,7 +133,7 @@ void validateNamedCollection( { throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Unexpected key {} in named collection. Required keys: {}, optional keys: {}", + "Unexpected key `{}` in named collection. Required keys: {}, optional keys: {}", backQuoteIfNeed(key), fmt::join(required_keys, ", "), fmt::join(optional_keys, ", ")); } } diff --git a/src/Storages/StorageExternalDistributed.h b/src/Storages/StorageExternalDistributed.h index c4d37c3e5cc..56c7fe86f34 100644 --- a/src/Storages/StorageExternalDistributed.h +++ b/src/Storages/StorageExternalDistributed.h @@ -8,8 +8,6 @@ namespace DB { -struct ExternalDataSourceConfiguration; - /// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas. /// This class unites multiple storages with replicas into multiple shards with replicas. /// A query to external database is passed to one replica on each shard, the result is united. diff --git a/src/TableFunctions/TableFunctionMongoDB.cpp b/src/TableFunctions/TableFunctionMongoDB.cpp index b2cf1b4675e..94279d1bf6d 100644 --- a/src/TableFunctions/TableFunctionMongoDB.cpp +++ b/src/TableFunctions/TableFunctionMongoDB.cpp @@ -1,5 +1,4 @@ #include -#include #include diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index f87ba6d1c6d..aca751c2840 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -15,7 +15,6 @@ #include #include -#include namespace DB From f17b70e3f3bea09dc09c4b22d857fbab4e1417e4 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 14:06:00 +0000 Subject: [PATCH 1015/1722] Resolve issues with settings --- src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 262 ------------------ .../ObjectStorage/Local/Configuration.cpp | 4 +- src/Storages/ObjectStorage/Utils.cpp | 5 +- 4 files changed, 3 insertions(+), 270 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 19a59bf7777..6bca8a99f02 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -116,9 +116,7 @@ class IColumn; M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ - M(Bool, local_truncate_on_insert, false, "Enables or disables truncate before insert in local engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ - M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6f73ca4059f..815880eb63e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,267 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"optimize_functions_to_subcolumns", false, true, "Enable optimization by default"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"local_create_new_file_on_insert", false, false, "Enables or disables creating a new file on each insert in local object storage engine tables"}, - {"local_truncate_on_insert", false, false, "Enables or disables truncating the file on each insert in local object storage engine tables"}, - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -350,7 +89,6 @@ static std::initializer_listgetSettingsRef(); return StorageObjectStorage::QuerySettings{ - .truncate_on_insert = settings.local_truncate_on_insert, - .create_new_file_on_insert = settings.local_create_new_file_on_insert, + .truncate_on_insert = settings.engine_file_truncate_on_insert, + .create_new_file_on_insert = false, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.engine_file_skip_empty_files, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 77725f8ed18..73410d959e0 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -40,10 +40,7 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", - configuration.getNamespace(), - key, - configuration.getTypeName(), - configuration.getTypeName()); + configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); } void resolveSchemaAndFormat( From 92a9b29b45c254e670fe9f67114b5af890bfb5cb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 15 Aug 2024 22:25:21 +0800 Subject: [PATCH 1016/1722] devirtualize format reader --- .../Formats/Impl/BinaryRowInputFormat.cpp | 4 +- .../Formats/Impl/BinaryRowInputFormat.h | 7 ++- .../Formats/Impl/CSVRowInputFormat.cpp | 2 +- .../Formats/Impl/CSVRowInputFormat.h | 7 ++- .../Impl/CustomSeparatedRowInputFormat.h | 3 +- .../Impl/JSONCompactEachRowRowInputFormat.h | 4 +- .../Impl/JSONCompactRowInputFormat.cpp | 2 +- .../Formats/Impl/JSONCompactRowInputFormat.h | 4 +- .../Formats/Impl/TabSeparatedRowInputFormat.h | 4 +- .../RowInputFormatWithNamesAndTypes.cpp | 61 +++++++++++++------ .../Formats/RowInputFormatWithNamesAndTypes.h | 5 +- 11 files changed, 67 insertions(+), 36 deletions(-) diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index c5336f3bcc7..b549f2de975 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -15,8 +15,8 @@ namespace ErrorCodes } template -BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes( +BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, const Block & header, IRowInputFormat::Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes>( header, in_, params_, diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 6f2042d1315..6a4ca8f6418 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -10,13 +10,16 @@ namespace DB class ReadBuffer; +template +class BinaryFormatReader; + /** A stream for inputting data in a binary line-by-line format. */ template -class BinaryRowInputFormat final : public RowInputFormatWithNamesAndTypes +class BinaryRowInputFormat final : public RowInputFormatWithNamesAndTypes> { public: - BinaryRowInputFormat(ReadBuffer & in_, const Block & header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); + BinaryRowInputFormat(ReadBuffer & in_, const Block & header, IRowInputFormat::Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); String getName() const override { return "BinaryRowInputFormat"; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index b7f84748f61..cf58a4057c8 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -61,7 +61,7 @@ CSVRowInputFormat::CSVRowInputFormat( bool with_names_, bool with_types_, const FormatSettings & format_settings_, - std::unique_ptr format_reader_) + std::unique_ptr format_reader_) : RowInputFormatWithNamesAndTypes( header_, *in_, diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index fe4d4e3be08..86af5028438 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -13,10 +12,12 @@ namespace DB { +class CSVFormatReader; + /** A stream for inputting data in csv format. * Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values. */ -class CSVRowInputFormat : public RowInputFormatWithNamesAndTypes +class CSVRowInputFormat : public RowInputFormatWithNamesAndTypes { public: /** with_names - in the first line the header with column names @@ -32,7 +33,7 @@ public: protected: CSVRowInputFormat(const Block & header_, std::shared_ptr in_, const Params & params_, - bool with_names_, bool with_types_, const FormatSettings & format_settings_, std::unique_ptr format_reader_); + bool with_names_, bool with_types_, const FormatSettings & format_settings_, std::unique_ptr format_reader_); CSVRowInputFormat(const Block & header_, std::shared_ptr in_buf_, const Params & params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 58f78e5af42..b1d35947ba8 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -9,7 +9,8 @@ namespace DB { -class CustomSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes +class CustomSeparatedFormatReader; +class CustomSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: CustomSeparatedRowInputFormat( diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index ebeb939e7fa..50589329073 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -11,7 +11,7 @@ namespace DB { class ReadBuffer; - +class JSONCompactEachRowFormatReader; /** A stream for reading data in a bunch of formats: * - JSONCompactEachRow @@ -20,7 +20,7 @@ class ReadBuffer; * - JSONCompactStringsEachRowWithNamesAndTypes * */ -class JSONCompactEachRowRowInputFormat final : public RowInputFormatWithNamesAndTypes +class JSONCompactEachRowRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: JSONCompactEachRowRowInputFormat( diff --git a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp index 63066fc8220..63ced05dd3a 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes JSONCompactRowInputFormat::JSONCompactRowInputFormat( const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes( + : RowInputFormatWithNamesAndTypes( header_, in_, params_, false, false, false, format_settings_, std::make_unique(in_, format_settings_)) { } diff --git a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h index 3a93e7149b0..eb70f6ec2a3 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowInputFormat.h @@ -5,8 +5,8 @@ namespace DB { - -class JSONCompactRowInputFormat final : public RowInputFormatWithNamesAndTypes +class JSONCompactFormatReader; +class JSONCompactRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: JSONCompactRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 32abd532a52..3c6efe9ac4c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -10,9 +10,11 @@ namespace DB { +class TabSeparatedFormatReader; + /** A stream to input data in tsv format. */ -class TabSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes +class TabSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes { public: /** with_names - the first line is the header with the names of the columns diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index ae30d741c2f..5701b80ecc2 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -1,14 +1,20 @@ -#include -#include -#include -#include #include #include -#include -#include -#include -#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -44,7 +50,8 @@ namespace } } -RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( +template +RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( const Block & header_, ReadBuffer & in_, const Params & params_, @@ -52,7 +59,7 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( bool with_names_, bool with_types_, const FormatSettings & format_settings_, - std::unique_ptr format_reader_, + std::unique_ptr format_reader_, bool try_detect_header_) : RowInputFormatWithDiagnosticInfo(header_, in_, params_) , format_settings(format_settings_) @@ -66,7 +73,8 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( column_indexes_by_names = getPort().getHeader().getNamesToIndexesMap(); } -void RowInputFormatWithNamesAndTypes::readPrefix() +template +void RowInputFormatWithNamesAndTypes::readPrefix() { /// Search and remove BOM only in textual formats (CSV, TSV etc), not in binary ones (RowBinary*). /// Also, we assume that column name or type cannot contain BOM, so, if format has header, @@ -138,7 +146,8 @@ void RowInputFormatWithNamesAndTypes::readPrefix() } } -void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & column_names_out, std::vector & type_names_out) +template +void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & column_names_out, std::vector & type_names_out) { auto & read_buf = getReadBuffer(); PeekableReadBuffer * peekable_buf = dynamic_cast(&read_buf); @@ -201,7 +210,8 @@ void RowInputFormatWithNamesAndTypes::tryDetectHeader(std::vector & colu peekable_buf->dropCheckpoint(); } -bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext) +template +bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext) { if (unlikely(end_of_stream)) return false; @@ -280,7 +290,8 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE return true; } -size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size) +template +size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size) { if (unlikely(end_of_stream)) return 0; @@ -304,7 +315,8 @@ size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size) return num_rows; } -void RowInputFormatWithNamesAndTypes::resetParser() +template +void RowInputFormatWithNamesAndTypes::resetParser() { RowInputFormatWithDiagnosticInfo::resetParser(); column_mapping->column_indexes_for_input_fields.clear(); @@ -313,7 +325,8 @@ void RowInputFormatWithNamesAndTypes::resetParser() end_of_stream = false; } -void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) +template +void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) { const auto & index = column_mapping->column_indexes_for_input_fields[file_column]; if (index) @@ -328,7 +341,8 @@ void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & ty } } -bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) +template +bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) { if (in->eof()) { @@ -374,12 +388,14 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu return format_reader->parseRowEndWithDiagnosticInfo(out); } -bool RowInputFormatWithNamesAndTypes::isGarbageAfterField(size_t index, ReadBuffer::Position pos) +template +bool RowInputFormatWithNamesAndTypes::isGarbageAfterField(size_t index, ReadBuffer::Position pos) { return format_reader->isGarbageAfterField(index, pos); } -void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) +template +void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) { format_reader->setReadBuffer(in_); IInputFormat::setReadBuffer(in_); @@ -582,5 +598,12 @@ void FormatWithNamesAndTypesSchemaReader::transformTypesIfNeeded(DB::DataTypePtr transformInferredTypesIfNeeded(type, new_type, format_settings); } +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes; +template class RowInputFormatWithNamesAndTypes>; +template class RowInputFormatWithNamesAndTypes>; } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index b7d9507151e..cd836cb00dc 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -26,6 +26,7 @@ class FormatWithNamesAndTypesReader; /// will be compared types from header. /// It's important that firstly this class reads/skips names and only /// then reads/skips types. So you can this invariant. +template class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo { protected: @@ -41,7 +42,7 @@ protected: bool with_names_, bool with_types_, const FormatSettings & format_settings_, - std::unique_ptr format_reader_, + std::unique_ptr format_reader_, bool try_detect_header_ = false); void resetParser() override; @@ -70,7 +71,7 @@ private: bool is_header_detected = false; protected: - std::unique_ptr format_reader; + std::unique_ptr format_reader; Block::NameMap column_indexes_by_names; }; From f6e1eb1643888c2b8bbc179899cbc4bacaee5b78 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 16:31:48 +0200 Subject: [PATCH 1017/1722] Fix style check --- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/PostgreSQLDictionarySource.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index d6df03b39df..bf19f912723 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -230,7 +230,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) *named_collection, /* required_keys */{}, /* optional_keys */ValidateKeysMultiset{ - "url", "endpoint", "user", "credentials.user", "password", "credentials.password", "format", "compression_method", "structure", "name"}); + "url", "endpoint", "user", "credentials.user", "password", "credentials.password", "format", "compression_method", "structure", "name"}); url = named_collection->getOrDefault("url", ""); endpoint = named_collection->getOrDefault("endpoint", ""); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index fd026a97cd4..fd426de126d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -24,6 +24,7 @@ namespace DB namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; + extern const int BAD_ARGUMENTS; } #if USE_LIBPQXX From 4548957f5a73600d227dcc26926c7678dd284c5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Aug 2024 15:07:07 +0000 Subject: [PATCH 1018/1722] Add default status to not-prepared processors. --- src/Processors/Executors/ExecutingGraph.cpp | 4 ++-- src/Processors/Executors/ExecutingGraph.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 2 +- src/Processors/IProcessor.cpp | 7 +++++-- src/Processors/IProcessor.h | 2 +- 5 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 6d5b60d8159..02ae8af5f52 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -279,7 +279,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue try { auto & processor = *node.processor; - IProcessor::Status last_status = node.last_processor_status; + const auto last_status = node.last_processor_status; IProcessor::Status status = processor.prepare(node.updated_input_ports, node.updated_output_ports); node.last_processor_status = status; @@ -319,7 +319,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue node.updated_input_ports.clear(); node.updated_output_ports.clear(); - switch (node.last_processor_status) + switch (*node.last_processor_status) { case IProcessor::Status::NeedData: case IProcessor::Status::PortFull: diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 71dcd360a2c..6f91c120d47 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -92,7 +92,7 @@ public: std::exception_ptr exception; /// Last state for profiling. - IProcessor::Status last_processor_status = IProcessor::Status::NeedData; + std::optional last_processor_status; /// Ports which have changed their state since last processor->prepare() call. /// They changed when neighbour processors interact with connected ports. diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 82cad471a29..d4630f21688 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -432,7 +432,7 @@ String PipelineExecutor::dumpPipeline() const } } - std::vector statuses; + std::vector> statuses; std::vector proc_list; statuses.reserve(graph->nodes.size()); proc_list.reserve(graph->nodes.size()); diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index edb4d662d8b..fc595a7b565 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -55,9 +55,12 @@ void IProcessor::dump() const } -std::string IProcessor::statusToName(Status status) +std::string IProcessor::statusToName(std::optional status) { - switch (status) + if (status == std::nullopt) + return "NotStarted"; + + switch (*status) { case Status::NeedData: return "NeedData"; diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index f1ce044d92f..02b8a3daa28 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -162,7 +162,7 @@ public: ExpandPipeline, }; - static std::string statusToName(Status status); + static std::string statusToName(std::optional status); /** Method 'prepare' is responsible for all cheap ("instantaneous": O(1) of data volume, no wait) calculations. * From 7f0406a7265058b3dd3e5fde4866126971050a8b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 15:53:09 +0000 Subject: [PATCH 1019/1722] Remove JSONEmpty function and allow JSON in empty function --- .../sql-reference/functions/json-functions.md | 32 ----- src/Functions/JSONEmpty.cpp | 126 ------------------ src/Functions/empty.cpp | 123 ++++++++++++++++- ...ead_subcolumns_2_compact_merge_tree.sql.j2 | 4 +- .../03207_json_read_subcolumns_2_memory.sql | 4 +- ...n_read_subcolumns_2_wide_merge_tree.sql.j2 | 4 +- ...ead_subcolumns_2_compact_merge_tree.sql.j2 | 4 +- ...array_of_json_read_subcolumns_2_memory.sql | 4 +- ...n_read_subcolumns_2_wide_merge_tree.sql.j2 | 4 +- .../aspell-ignore/en/aspell-dict.txt | 1 - 10 files changed, 134 insertions(+), 172 deletions(-) delete mode 100644 src/Functions/JSONEmpty.cpp diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index fd1f924c18f..26fe888ab49 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -1359,35 +1359,3 @@ SELECT json, JSONSharedDataPathsWithTypes(json) FROM test; │ {"a":["1","2","3"],"c":"2020-01-01"} │ {'c':'Date'} │ └──────────────────────────────────────┴────────────────────────────────────┘ ``` - -### JSONEmpty - -Checks whether the input [JSON](../data-types/newjson.md) object is empty. - -``` sql -JSONEmpty(json) -``` - -**Arguments** - -- `json` — [JSON](../data-types/newjson.md). - -**Returned value** - -- Returns `1` for an empty JSON object or `0` for a non-empty JSON object. [UInt8](../data-types/int-uint.md). - -**Example** - -``` sql -CREATE TABLE test (json JSON) ENGINE = Memory; -INSERT INTO test FORMAT JSONEachRow {"json" : {}}, {"json" : {"a" : [1, 2, 3], "b" : "2020-01-01"}}, {"json" : {}}, -SELECT json, JSONEmpty(json) FROM test; -``` - -```text -┌─json─────────────────────────────────┬─JSONEmpty(json)─┠-│ {} │ 1 │ -│ {"a":["1","2","3"],"b":"2020-01-01"} │ 0 │ -│ {} │ 1 │ -└──────────────────────────────────────┴─────────────────┘ -``` diff --git a/src/Functions/JSONEmpty.cpp b/src/Functions/JSONEmpty.cpp deleted file mode 100644 index cdd1ac2f237..00000000000 --- a/src/Functions/JSONEmpty.cpp +++ /dev/null @@ -1,126 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; -} - - -namespace -{ - -/// Implements the function JSONEmpty which returns true if provided JSON object is empty and false otherwise. -class FunctionJSONEmpty : public IFunction -{ -public: - static constexpr auto name = "JSONEmpty"; - - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - std::string getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForConstants() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override - { - if (data_types.size() != 1 || data_types[0]->getTypeId() != TypeIndex::Object) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single argument with type JSON", getName()); - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override - { - const ColumnWithTypeAndName & elem = arguments[0]; - const auto * object_column = typeid_cast(elem.column.get()); - if (!object_column) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected column type in function {}. Expected Object column, got {}", getName(), elem.column->getName()); - - auto res = DataTypeUInt8().createColumn(); - auto & data = typeid_cast(*res).getData(); - const auto & typed_paths = object_column->getTypedPaths(); - size_t size = object_column->size(); - /// If object column has at least 1 typed path, it will never be empty, because these paths always have values. - if (!typed_paths.empty()) - { - data.resize_fill(size, 0); - return res; - } - - const auto & dynamic_paths = object_column->getDynamicPaths(); - const auto & shared_data = object_column->getSharedDataPtr(); - data.reserve(size); - for (size_t i = 0; i != size; ++i) - { - bool empty = true; - /// Check if there is no paths in shared data. - if (!shared_data->isDefaultAt(i)) - { - empty = false; - } - /// Check that all dynamic paths have NULL value in this row. - else - { - for (const auto & [path, column] : dynamic_paths) - { - if (!column->isNullAt(i)) - { - empty = false; - break; - } - } - } - - data.push_back(empty); - } - - return res; - } -}; - -} - -REGISTER_FUNCTION(JSONEmpty) -{ - factory.registerFunction(FunctionDocumentation{ - .description = R"( -Checks whether the input JSON object is empty. -)", - .syntax = {"JSONEmpty(json)"}, - .arguments = {{"json", "JSON column"}}, - .examples = {{{ - "Example", - R"( -CREATE TABLE test (json JSON) ENGINE = Memory; -INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {}}, {"json" : {"b" : "Hello"}}, {"json" : {}} -SELECT json, JSONEmpty(json) FROM test; -)", - R"( -┌─json──────────┬─JSONEmpty(json)─┠-│ {"a":"42"} │ 0 │ -│ {} │ 1 │ -│ {"b":"Hello"} │ 0 │ -│ {} │ 1 │ -└───────────────┴─────────────────┘ - -)"}}}, - .categories{"JSON"}, - }); -} - -} diff --git a/src/Functions/empty.cpp b/src/Functions/empty.cpp index 51811d21a0c..f97e6bfb258 100644 --- a/src/Functions/empty.cpp +++ b/src/Functions/empty.cpp @@ -2,10 +2,18 @@ #include #include #include +#include namespace DB { + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + namespace { @@ -13,13 +21,126 @@ struct NameEmpty { static constexpr auto name = "empty"; }; + using FunctionEmpty = FunctionStringOrArrayToT, NameEmpty, UInt8, false>; +/// Implements the empty function for JSON type. +class ExecutableFunctionJSONEmpty : public IExecutableFunction +{ +public: + std::string getName() const override { return NameEmpty::name; } + +private: + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const ColumnWithTypeAndName & elem = arguments[0]; + const auto * object_column = typeid_cast(elem.column.get()); + if (!object_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected column type in function {}. Expected Object column, got {}", getName(), elem.column->getName()); + + auto res = DataTypeUInt8().createColumn(); + auto & data = typeid_cast(*res).getData(); + const auto & typed_paths = object_column->getTypedPaths(); + size_t size = object_column->size(); + /// If object column has at least 1 typed path, it will never be empty, because these paths always have values. + if (!typed_paths.empty()) + { + data.resize_fill(size, 0); + return res; + } + + const auto & dynamic_paths = object_column->getDynamicPaths(); + const auto & shared_data = object_column->getSharedDataPtr(); + data.reserve(size); + for (size_t i = 0; i != size; ++i) + { + bool empty = true; + /// Check if there is no paths in shared data. + if (!shared_data->isDefaultAt(i)) + { + empty = false; + } + /// Check that all dynamic paths have NULL value in this row. + else + { + for (const auto & [path, column] : dynamic_paths) + { + if (!column->isNullAt(i)) + { + empty = false; + break; + } + } + } + + data.push_back(empty); + } + + return res; + } +}; + +class FunctionEmptyJSON final : public IFunctionBase +{ +public: + FunctionEmptyJSON(const DataTypes & argument_types_, const DataTypePtr & return_type_) : argument_types(argument_types_), return_type(return_type_) {} + + String getName() const override { return NameEmpty::name; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + const DataTypes & getArgumentTypes() const override { return argument_types; } + const DataTypePtr & getResultType() const override { return return_type; } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique(); + } + +private: + DataTypes argument_types; + DataTypePtr return_type; +}; + +class FunctionEmptyOverloadResolver final : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = NameEmpty::name; + + static FunctionOverloadResolverPtr create(ContextPtr) + { + return std::make_unique(); + } + + String getName() const override { return NameEmpty::name; } + size_t getNumberOfArguments() const override { return 1; } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + DataTypes argument_types; + argument_types.reserve(arguments.size()); + for (const auto & arg : arguments) + argument_types.push_back(arg.type); + + if (argument_types.size() == 1 && isObject(argument_types[0])) + return std::make_shared(argument_types, return_type); + + return std::make_shared(std::make_shared(), argument_types, return_type); + } + + DataTypePtr getReturnTypeImpl(const DataTypes &) const override + { + return std::make_shared(); + } +}; + } REGISTER_FUNCTION(Empty) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 index 942489ec8fc..6c33044b5d8 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.sql.j2 @@ -109,12 +109,12 @@ select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.: 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 count() from test where empty(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 count() from test where empty(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; diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql index 3c791118c9c..cc646987c80 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.sql @@ -106,12 +106,12 @@ select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.: 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 count() from test where empty(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 count() from test where empty(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; diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 index 325ce2dcbf5..ab4e0437c15 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.sql.j2 @@ -109,12 +109,12 @@ select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.: 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 count() from test where empty(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 count() from test where empty(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; diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 index e1e0c07a5df..e3930165602 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_compact_merge_tree.sql.j2 @@ -40,8 +40,8 @@ select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, 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); 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 count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null; +select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(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; diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql index 1cd5e2b8d47..9274b9b9cf7 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_memory.sql @@ -37,8 +37,8 @@ select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, 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); 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 count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null; +select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(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; diff --git a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 index 89223316aa3..3010fa0e2de 100644 --- a/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 +++ b/tests/queries/0_stateless/03208_array_of_json_read_subcolumns_2_wide_merge_tree.sql.j2 @@ -40,8 +40,8 @@ select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, 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); 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 count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null; +select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(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; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 0289296c580..fd836d93143 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -411,7 +411,6 @@ JSONDynamicPaths JSONDynamicPathsWithTypes JSONEachRow JSONEachRowWithProgress -JSONEmpty JSONExtract JSONExtractArrayRaw JSONExtractBool From 65493ccf12f6a58d898544f484ad95a54724854d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 15:57:21 +0000 Subject: [PATCH 1020/1722] Remove unused error code --- src/Functions/empty.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/empty.cpp b/src/Functions/empty.cpp index f97e6bfb258..b696385e8c9 100644 --- a/src/Functions/empty.cpp +++ b/src/Functions/empty.cpp @@ -10,7 +10,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } From b8055eb346c1eabe006d33e905134c58eaac66e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Aug 2024 17:03:14 +0000 Subject: [PATCH 1021/1722] Fixing test. --- .../0_stateless/02210_processors_profile_log.reference | 4 ++-- tests/queries/0_stateless/02210_processors_profile_log.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02210_processors_profile_log.reference b/tests/queries/0_stateless/02210_processors_profile_log.reference index 035bd9897ad..12ba17103da 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log.reference @@ -6,6 +6,6 @@ ExpressionTransform ExpressionTransform 1 1 1 1 1 LazyOutputFormat 1 1 1 0 0 LimitsCheckingTransform 1 1 1 1 1 -NullSource 1 0 0 0 0 -NullSource 1 0 0 0 0 +NullSource 0 0 0 0 0 +NullSource 0 0 0 0 0 SourceFromSingleChunk 1 0 0 1 1 diff --git a/tests/queries/0_stateless/02210_processors_profile_log.sql b/tests/queries/0_stateless/02210_processors_profile_log.sql index 75e5bcbb585..a850f4312b3 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.sql +++ b/tests/queries/0_stateless/02210_processors_profile_log.sql @@ -20,8 +20,8 @@ SELECT -- SourceFromSingleChunk, that feed data to ExpressionTransform, -- will feed first block and then wait in PortFull. name = 'SourceFromSingleChunk', output_wait_elapsed_us >= 0.9e6 ? 1 : output_wait_elapsed_us, - -- NullSource/LazyOutputFormatLazyOutputFormat are the outputs - -- so they cannot starts to execute before sleep(1) will be executed. + -- LazyOutputFormatLazyOutputFormat is the output + -- so it cannot starts to execute before sleep(1) will be executed. input_wait_elapsed_us>=1e6 ? 1 : input_wait_elapsed_us) elapsed, input_rows, From 5dcf3f0c59c1db273e56eec8e8631c09252036a6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 15 Aug 2024 19:39:35 +0200 Subject: [PATCH 1022/1722] Update 03223_analyzer_with_cube_fuzz.sql --- tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql b/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql index c19d11fbe0c..f3bccc79b3f 100644 --- a/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql +++ b/tests/queries/0_stateless/03223_analyzer_with_cube_fuzz.sql @@ -1,3 +1,5 @@ +SET enable_analyzer = 1; + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 75f951dae57c4de27cdba28792622a35597cf31c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Aug 2024 17:59:31 +0000 Subject: [PATCH 1023/1722] limit task time for mutations --- src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 88 +++++++++++++--------- 3 files changed, 57 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ee15bc7f711..26cb821f33b 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -9,7 +9,6 @@ #include #include #include -#include "base/types.h" #include #include #include @@ -521,7 +520,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); do { @@ -751,7 +750,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const bool MergeTask::VerticalMergeStage::executeVerticalMergeForOneColumn() const { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); do { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 149f9a3e80b..de1f0f60cfc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,7 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, merge_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge. Can be exceeded if one step takes longer time", 0) \ + M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 14c274d7f64..40648439887 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1257,6 +1257,8 @@ public: private: void prepare(); bool mutateOriginalPartAndPrepareProjections(); + void writeTempProjectionPart(size_t projection_idx, Chunk chunk); + void finalizeTempProjections(); bool iterateThroughAllProjections(); void constructTaskForProjectionPartsMerge(); void finalize(); @@ -1307,10 +1309,22 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { - Block cur_block; - Block projection_header; - if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); + + do { + Block cur_block; + Block projection_header; + + MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry); + + if (!ctx->mutating_executor->pull(cur_block)) + { + finalizeTempProjections(); + return false; + } + if (ctx->minmax_idx) ctx->minmax_idx->update(cur_block, MergeTreeData::getMinMaxColumnsNames(ctx->metadata_snapshot->getPartitionKey())); @@ -1322,46 +1336,56 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - const auto & projection = *ctx->projections_to_build[i]; + Chunk squashed_chunk; - ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - - Chunk squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); - if (squashed_chunk) { - auto result = projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); - tmp_part.finalize(); - tmp_part.part->getDataPartStorage().commitTransaction(); - projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + ProfileEventTimeIncrement projection_watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); + Block block_to_squash = ctx->projections_to_build[i]->calculate(cur_block, ctx->context); + + projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); + squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); } + + if (squashed_chunk) + writeTempProjectionPart(i, std::move(squashed_chunk)); } (*ctx->mutate_entry)->rows_written += cur_block.rows(); (*ctx->mutate_entry)->bytes_written_uncompressed += cur_block.bytes(); + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } + /// Need execute again + return true; +} +void PartMergerWriter::writeTempProjectionPart(size_t projection_idx, Chunk chunk) +{ + const auto & projection = *ctx->projections_to_build[projection_idx]; + const auto & projection_plan = projection_squashes[projection_idx]; + + auto result = projection_plan.getHeader().cloneWithColumns(chunk.detachColumns()); + + auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( + *ctx->data, + ctx->log, + result, + projection, + ctx->new_data_part.get(), + ++block_num); + + tmp_part.finalize(); + tmp_part.part->getDataPartStorage().commitTransaction(); + projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); +} + +void PartMergerWriter::finalizeTempProjections() +{ // Write the last block for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - const auto & projection = *ctx->projections_to_build[i]; - auto & projection_squash_plan = projection_squashes[i]; - auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + auto squashed_chunk = Squashing::squash(projection_squashes[i].flush()); if (squashed_chunk) - { - auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); - temp_part.finalize(); - temp_part.part->getDataPartStorage().commitTransaction(); - projection_parts[projection.name].emplace_back(std::move(temp_part.part)); - } + writeTempProjectionPart(i, std::move(squashed_chunk)); } projection_parts_iterator = std::make_move_iterator(projection_parts.begin()); @@ -1369,12 +1393,8 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() /// Maybe there are no projections ? if (projection_parts_iterator != std::make_move_iterator(projection_parts.end())) constructTaskForProjectionPartsMerge(); - - /// Let's move on to the next stage - return false; } - void PartMergerWriter::constructTaskForProjectionPartsMerge() { auto && [name, parts] = *projection_parts_iterator; From a5cc3a3a9cb7d40cc28a00016abbc08ba433e16a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 15 Aug 2024 20:03:22 +0200 Subject: [PATCH 1024/1722] Update log message --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 286d06bc424..56bfa019819 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -1031,7 +1031,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } else LOG_TRACE(log, "No space left in cache to reserve {} bytes, reason: {}, " - "will continue without cache download", failure_reason, size); + "will continue without cache download", size, failure_reason); if (!success) { From 6c085e0a939caeaf913a98ec83ce8d62e7a1413e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 15 Aug 2024 20:51:46 +0200 Subject: [PATCH 1025/1722] Fix empty types check --- src/Functions/empty.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Functions/empty.cpp b/src/Functions/empty.cpp index b696385e8c9..4475f622b56 100644 --- a/src/Functions/empty.cpp +++ b/src/Functions/empty.cpp @@ -131,6 +131,15 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes &) const override { + if (!isStringOrFixedString(arguments[0]) + && !isArray(arguments[0]) + && !isMap(arguments[0]) + && !isUUID(arguments[0]) + && !isIPv6(arguments[0]) + && !isIPv4(arguments[0]) + && !isObject(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); + return std::make_shared(); } }; From 5445669d4e296a367f4c807b227d5ae3888f2c3e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 15 Aug 2024 20:55:36 +0200 Subject: [PATCH 1026/1722] Don't create new dynamic path for null values --- src/Formats/JSONExtractTree.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index cac4dfbb077..d8c4ed229ca 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1731,6 +1731,11 @@ private: return false; } } + /// Don't create new dynamic paths for null and don't insert null values into shared data. + /// We consider null equivalent to the absence of this path. + else if (element.isNull()) + { + } /// Try to add a new dynamic path. else if (auto * dynamic_column = column_object.tryToAddNewDynamicPath(current_path)) { @@ -1740,10 +1745,8 @@ private: return false; } } - /// Otherwise this path should go to the shared data. - /// Don't insert null values into shared data. /// We consider null equivalent to the absence of this path. - else if (!element.isNull()) + else { auto tmp_dynamic_column = ColumnDynamic::create(); tmp_dynamic_column->reserve(1); From 32203edb07a27e0218925163fb902606efaffb9f Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 19:24:31 +0000 Subject: [PATCH 1027/1722] Fix --- src/Functions/empty.cpp | 3 ++- src/Storages/AlterCommands.cpp | 15 ++++++++++++--- .../03225_alter_to_json_not_supported.reference | 0 .../03225_alter_to_json_not_supported.sql | 15 +++++++++++++++ 4 files changed, 29 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03225_alter_to_json_not_supported.reference create mode 100644 tests/queries/0_stateless/03225_alter_to_json_not_supported.sql diff --git a/src/Functions/empty.cpp b/src/Functions/empty.cpp index 4475f622b56..ddb503668cf 100644 --- a/src/Functions/empty.cpp +++ b/src/Functions/empty.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -129,7 +130,7 @@ public: return std::make_shared(std::make_shared(), argument_types, return_type); } - DataTypePtr getReturnTypeImpl(const DataTypes &) const override + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isStringOrFixedString(arguments[0]) && !isArray(arguments[0]) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d5780e32db3..8fbd6cbd29d 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -1403,14 +1404,22 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const const GetColumnsOptions options(GetColumnsOptions::All); const auto old_data_type = all_columns.getColumn(options, column_name).type; - bool new_type_has_object = command.data_type->hasDynamicSubcolumnsDeprecated(); - bool old_type_has_object = old_data_type->hasDynamicSubcolumnsDeprecated(); + bool new_type_has_deprecated_object = command.data_type->hasDynamicSubcolumnsDeprecated(); + bool old_type_has_deprecated_object = old_data_type->hasDynamicSubcolumnsDeprecated(); - if (new_type_has_object || old_type_has_object) + if (new_type_has_deprecated_object || old_type_has_deprecated_object) throw Exception( ErrorCodes::BAD_ARGUMENTS, "The change of data type {} of column {} to {} is not allowed. It has known bugs", old_data_type->getName(), backQuote(column_name), command.data_type->getName()); + + bool has_object_type = isObject(command.data_type); + command.data_type->forEachChild([&](const IDataType & type){ has_object_type |= isObject(type); }); + if (has_object_type) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The change of data type {} of column {} to {} is not supported.", + old_data_type->getName(), backQuote(column_name), command.data_type->getName()); } if (command.isRemovingProperty()) diff --git a/tests/queries/0_stateless/03225_alter_to_json_not_supported.reference b/tests/queries/0_stateless/03225_alter_to_json_not_supported.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03225_alter_to_json_not_supported.sql b/tests/queries/0_stateless/03225_alter_to_json_not_supported.sql new file mode 100644 index 00000000000..398494d56de --- /dev/null +++ b/tests/queries/0_stateless/03225_alter_to_json_not_supported.sql @@ -0,0 +1,15 @@ +set allow_experimental_json_type = 1; + +drop table if exists test; +create table test (s String) engine=MergeTree order by tuple(); +alter table test modify column s JSON; -- { serverError BAD_ARGUMENTS } +drop table test; + +create table test (s Array(String)) engine=MergeTree order by tuple(); +alter table test modify column s Array(JSON); -- { serverError BAD_ARGUMENTS } +drop table test; + +create table test (s Tuple(String, String)) engine=MergeTree order by tuple(); +alter table test modify column s Tuple(JSON, String); -- { serverError BAD_ARGUMENTS } +drop table test; + From 077f10a4ada2a561111207d8e99e22d2c8e48f40 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 18:26:48 +0200 Subject: [PATCH 1028/1722] Fix build --- src/Dictionaries/PostgreSQLDictionarySource.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index fd426de126d..8e472f85a6e 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -4,6 +4,7 @@ #include #include #include "DictionarySourceFactory.h" +#include #include "registerDictionaries.h" #if USE_LIBPQXX @@ -13,7 +14,6 @@ #include "readInvalidateQuery.h" #include #include -#include #include #endif @@ -27,14 +27,14 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -#if USE_LIBPQXX - -static const UInt64 max_block_size = 8192; - static const ValidateKeysMultiset dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", "schema", "update_field", "update_lag", "invalidate_query", "query", "where", "name", "priority"}; +#if USE_LIBPQXX + +static const UInt64 max_block_size = 8192; + namespace { ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct, const String & schema, const String & table, const String & query, const String & where) @@ -178,8 +178,6 @@ std::string PostgreSQLDictionarySource::toString() const return "PostgreSQL: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where); } -#endif - static void validateConfigKeys( const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix) { @@ -193,6 +191,8 @@ static void validateConfigKeys( } } +#endif + void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, From 2a005ee186eb71cc9e4a13f987cfda7f1a9228f1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 20:42:43 +0000 Subject: [PATCH 1029/1722] Don't allow alter update on columns with dynamic subcolumns --- src/Interpreters/MutationsInterpreter.cpp | 6 ++++++ ...226_alter_update_dynamic_json_not_supported.reference | 0 .../03226_alter_update_dynamic_json_not_supported.sql | 9 +++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.reference create mode 100644 tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 24635870e62..0b93b5989b1 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -500,6 +500,12 @@ static void validateUpdateColumns( throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", backQuote(column_name)); } } + else if (storage_columns.getColumn(GetColumnsOptions::Ordinary, column_name).type->hasDynamicSubcolumns()) + { + throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, + "Cannot update column {} with type {}: updates of columns with dynamic subcolumns are not supported", + backQuote(column_name), storage_columns.getColumn(GetColumnsOptions::Ordinary, column_name).type->getName()); + } } } diff --git a/tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.reference b/tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.sql b/tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.sql new file mode 100644 index 00000000000..720f8670c83 --- /dev/null +++ b/tests/queries/0_stateless/03226_alter_update_dynamic_json_not_supported.sql @@ -0,0 +1,9 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_json_type = 1; + +drop table if exists test; +create table test (d Dynamic, json JSON) engine=MergeTree order by tuple(); +alter table test update d = 42 where 1; -- {serverError CANNOT_UPDATE_COLUMN} +alter table test update json = '{}' where 1; -- {serverError CANNOT_UPDATE_COLUMN} +drop table test; + From bf44d17eb8999af5441bdc56c2e9047aedc3de39 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 20:55:55 +0000 Subject: [PATCH 1030/1722] Fix comment --- src/Formats/JSONExtractTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index d8c4ed229ca..5e9f8a3690b 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1726,7 +1726,7 @@ private: } } else if (!dynamic_node->insertResultToColumn(*dynamic_it->second, element, insert_settings, format_settings, error)) - { + { error += fmt::format(" (while reading path {})", current_path); return false; } @@ -1745,7 +1745,7 @@ private: return false; } } - /// We consider null equivalent to the absence of this path. + /// Otherwise this path should go to the shared data. else { auto tmp_dynamic_column = ColumnDynamic::create(); From 182c00eba71acc602c60e0cd24be5c1696dd3c42 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 15 Aug 2024 23:17:10 +0200 Subject: [PATCH 1031/1722] Fix style --- src/Formats/JSONExtractTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 5e9f8a3690b..122224535a7 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1726,7 +1726,7 @@ private: } } else if (!dynamic_node->insertResultToColumn(*dynamic_it->second, element, insert_settings, format_settings, error)) - { + { error += fmt::format(" (while reading path {})", current_path); return false; } From 624afd68c86bb4fc3eee76f3fb9779ffb2319c68 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 01:24:52 +0400 Subject: [PATCH 1032/1722] parallel attach from multiple partitions --- src/Storages/StorageReplicatedMergeTree.cpp | 112 ++++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 13 +++ 2 files changed, 95 insertions(+), 30 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a3c1ab7cdff..67f51b186c5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8030,25 +8030,84 @@ void StorageReplicatedMergeTree::replacePartitionFrom( /// First argument is true, because we possibly will add new data to current table. auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); auto lock2 = source_table->lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); - auto storage_settings_ptr = getSettings(); - auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); - auto metadata_snapshot = getInMemoryMetadataPtr(); + const auto storage_settings_ptr = getSettings(); + const auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); + const auto metadata_snapshot = getInMemoryMetadataPtr(); + const MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - Stopwatch watch; + std::unordered_set partitions; + if (partition->as()->all) + { + if (replace) + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); + + partitions = src_data.getAllPartitionIds(); + } + else + { + partitions = std::unordered_set(); + partitions.emplace(getPartitionIDFromQuery(partition, query_context)); + } + LOG_INFO(log, "Will try to attach {} partitions", partitions.size()); + + const Stopwatch watch; ProfileEventsScope profile_events_scope; + const auto zookeeper = getZooKeeper(); - MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); + const bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + try + { + std::unique_ptr entries[partitions.size()]; + size_t idx = 0; + for (const auto & partition_id : partitions) + { + entries[idx] = replacePartitionFromImpl(watch, + profile_events_scope, + metadata_snapshot, + src_data, + partition_id, + zookeeper, + replace, + zero_copy_enabled, + storage_settings_ptr->always_use_copy_instead_of_hardlinks, + query_context); + ++idx; + } + + for (const auto & entry : entries) + waitForLogEntryToBeProcessedIfNecessary(*entry, query_context); + + lock2.reset(); + lock1.reset(); + } + catch(...) + { + lock2.reset(); + lock1.reset(); + + throw; + } +} + +std::unique_ptr StorageReplicatedMergeTree::replacePartitionFromImpl( + const Stopwatch & watch, + ProfileEventsScope & profile_events_scope, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & src_data, + const String & partition_id, + const ZooKeeperPtr & zookeeper, + bool replace, + const bool & zero_copy_enabled, + const bool & always_use_copy_instead_of_hardlinks, + const ContextPtr & query_context) +{ /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); - static const String TMP_PREFIX = "tmp_replace_from_"; - auto zookeeper = getZooKeeper(); - /// Retry if alter_partition_version changes for (size_t retry = 0; retry < 1000; ++retry) { @@ -8133,11 +8192,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom( UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication - || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), + .copy_instead_of_hardlink = always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; if (replace) @@ -8145,7 +8202,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( /// Replace can only work on the same disk auto [dst_part, part_lock] = cloneAndLoadDataPart( src_part, - TMP_PREFIX, + TMP_PREFIX_REPLACE_PARTITION_FROM, dst_part_info, metadata_snapshot, clone_params, @@ -8160,7 +8217,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( /// Attach can work on another disk auto [dst_part, part_lock] = cloneAndLoadDataPart( src_part, - TMP_PREFIX, + TMP_PREFIX_REPLACE_PARTITION_FROM, dst_part_info, metadata_snapshot, clone_params, @@ -8176,15 +8233,16 @@ void StorageReplicatedMergeTree::replacePartitionFrom( part_checksums.emplace_back(hash_hex); } - ReplicatedMergeTreeLogEntryData entry; + //ReplicatedMergeTreeLogEntryData entry; + auto entry = std::make_unique(); { auto src_table_id = src_data.getStorageID(); - entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; - entry.source_replica = replica_name; - entry.create_time = time(nullptr); - entry.replace_range_entry = std::make_shared(); + entry->type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; + entry->source_replica = replica_name; + entry->create_time = time(nullptr); + entry->replace_range_entry = std::make_shared(); - auto & entry_replace = *entry.replace_range_entry; + auto & entry_replace = *entry->replace_range_entry; entry_replace.drop_range_part_name = drop_range_fake_part_name; entry_replace.from_database = src_table_id.database_name; entry_replace.from_table = src_table_id.table_name; @@ -8225,7 +8283,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); /// Just update version, because merges assignment relies on it ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry->toString(), zkutil::CreateMode::PersistentSequential)); Transaction transaction(*this, NO_TRANSACTION_RAW); { @@ -8275,14 +8333,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } String log_znode_path = dynamic_cast(*op_results.back()).path_created; - entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + entry->znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - lock2.reset(); - lock1.reset(); - /// We need to pull the REPLACE_RANGE before cleaning the replaced parts (otherwise CHeckThread may decide that parts are lost) queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); // No need to block operations further, especially that in case we have to wait for mutation to finish, the intent would block @@ -8291,10 +8346,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( parts_holder.clear(); cleanup_thread.wakeup(); - - waitForLogEntryToBeProcessedIfNecessary(entry, query_context); - - return; + return entry; } throw Exception( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2e54f17d5d5..4291aebf3e8 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -1013,6 +1014,18 @@ private: DataPartsVector::const_iterator it; }; + const String TMP_PREFIX_REPLACE_PARTITION_FROM = "tmp_replace_from_"; + std::unique_ptr replacePartitionFromImpl( + const Stopwatch & watch, + ProfileEventsScope & profile_events_scope, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & src_data, + const String & partition_id, + const zkutil::ZooKeeperPtr & zookeeper, + bool replace, + const bool & zero_copy_enabled, + const bool & always_use_copy_instead_of_hardlinks, + const ContextPtr & query_context); }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); From a49ef43286e31525f8829e3307a9231be0ce417c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 15 Aug 2024 21:33:26 +0000 Subject: [PATCH 1033/1722] Fix 01119_session_log flakiness --- ...9_session_log.sql => 01119_session_log.sh} | 37 +++++++++++++++---- 1 file changed, 30 insertions(+), 7 deletions(-) rename tests/queries/0_stateless/{01119_session_log.sql => 01119_session_log.sh} (73%) mode change 100644 => 100755 diff --git a/tests/queries/0_stateless/01119_session_log.sql b/tests/queries/0_stateless/01119_session_log.sh old mode 100644 new mode 100755 similarity index 73% rename from tests/queries/0_stateless/01119_session_log.sql rename to tests/queries/0_stateless/01119_session_log.sh index 55f6228797a..809d300fada --- a/tests/queries/0_stateless/01119_session_log.sql +++ b/tests/queries/0_stateless/01119_session_log.sh @@ -1,5 +1,20 @@ --- Tags: no-fasttest +#!/usr/bin/env bash +# Tags: no-fasttest +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +session_log_query_prefix=" +system flush logs; +select distinct type, user, auth_type, toString(client_address)!='::ffff:0.0.0.0' as a, client_port!=0 as b, interface from system.session_log +where user in ('default', 'nonexistsnt_user_1119', ' ', ' INTERSERVER SECRET ') +and interface in ('HTTP', 'TCP', 'TCP_Interserver') +and (user != 'default' or (a=1 and b=1)) -- FIXME: we should not write uninitialized address and port (but we do sometimes) +and event_time >= now() - interval 5 minute" + +$CLICKHOUSE_CLIENT -nm -q " select * from remote('127.0.0.2', system, one, 'default', ''); select * from remote('127.0.0.2', system, one, 'default', 'wrong password'); -- { serverError AUTHENTICATION_FAILED } select * from remote('127.0.0.2', system, one, 'nonexistsnt_user_1119', ''); -- { serverError AUTHENTICATION_FAILED } @@ -16,9 +31,17 @@ select * from url('http://127.0.0.1:8123/?query=select+1&user=+++', LineAsString select * from cluster('test_cluster_interserver_secret', system, one); -system flush logs; -select distinct type, user, auth_type, toString(client_address)!='::ffff:0.0.0.0' as a, client_port!=0 as b, interface from system.session_log -where user in ('default', 'nonexistsnt_user_1119', ' ', ' INTERSERVER SECRET ') -and interface in ('HTTP', 'TCP', 'TCP_Interserver') -and (user != 'default' or (a=1 and b=1)) -- FIXME: we should not write uninitialized address and port (but we do sometimes) -and event_time >= now() - interval 5 minute order by type, user, interface; +$session_log_query_prefix and type != 'Logout' order by type, user, interface; +" + +# Wait for logout events. +for attempt in {1..10} +do + if [ "`$CLICKHOUSE_CLIENT -q "$session_log_query_prefix and type = 'Logout'" | wc -l`" -eq 3 ] + then + break + fi + sleep 2 +done + +$CLICKHOUSE_CLIENT -q "$session_log_query_prefix and type = 'Logout' order by user, interface" From b1963738bdf8e02e78a0be1b2cce1b992a6d533c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 15 Aug 2024 21:41:40 +0000 Subject: [PATCH 1034/1722] Rebase --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index c2d74e21ba1..eb35c10ac57 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit c2d74e21ba1b8a27966e344693e176f927e4eb50 +Subproject commit eb35c10ac5725da7ef4be88b303895e1b5d153be From 3972991b1f364540927858e5f45bf519a1bba928 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 15 Aug 2024 23:49:49 +0200 Subject: [PATCH 1035/1722] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 1eb2e707194..19c8b2f084e 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -47,7 +47,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (!future_part->parts.empty()) { source_data_version = future_part->parts[0]->info.getDataVersion(); - is_mutation = (result_part_info.getDataVersion() != source_data_version); + is_mutation = (result_part_info.level == future_part->parts[0]->info.level); WriteBufferFromString out(partition); const auto & part = future_part->parts[0]; From c31c4154ca47c6f1313c04974b7bb95bbcb34cff Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 01:49:54 +0400 Subject: [PATCH 1036/1722] fix cs --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 67f51b186c5..38309282768 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8083,7 +8083,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( lock2.reset(); lock1.reset(); } - catch(...) + catch (...) { lock2.reset(); lock1.reset(); From db691e4cba1c5eaee10868a481ced4b729d73bc6 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Aug 2024 22:35:07 +0000 Subject: [PATCH 1037/1722] Update tests --- .../01825_new_type_json_ephemeral.reference | 2 +- ...3206_json_parsing_and_formatting.reference | 32 +-- ...bcolumns_1_compact_merge_tree.reference.j2 | 200 +++++++++--------- ...07_json_read_subcolumns_1_memory.reference | 100 ++++----- ..._subcolumns_1_wide_merge_tree.reference.j2 | 200 +++++++++--------- ...bcolumns_2_compact_merge_tree.reference.j2 | 4 +- ...07_json_read_subcolumns_2_memory.reference | 2 +- ..._subcolumns_2_wide_merge_tree.reference.j2 | 4 +- 8 files changed, 272 insertions(+), 272 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference b/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference index e8891cf6a56..7efe8cea252 100644 --- a/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference +++ b/tests/queries/0_stateless/01825_new_type_json_ephemeral.reference @@ -1 +1 @@ -PushEvent some-repo {"actor":{"avatar_url":"https:\\/\\/avatars.githubusercontent.com\\/u\\/123213213?","display_login":"github-actions","gravatar_id":"","id":"123123123","login":"github-actions[bot]","url":"https:\\/\\/api.github.com\\/users\\/github-actions[bot]"},"created_at":"2022-01-04 07:00:00.000000000","repo":{"id":"1001001010101","name":"some-repo","url":"https:\\/\\/api.github.com\\/repos\\/some-repo"},"type":"PushEvent"} +PushEvent some-repo {"actor":{"avatar_url":"https:\\/\\/avatars.githubusercontent.com\\/u\\/123213213?","display_login":"github-actions","gravatar_id":"","id":"123123123","login":"github-actions[bot]","url":"https:\\/\\/api.github.com\\/users\\/github-actions[bot]"},"created_at":"2022-01-04 07:00:00","repo":{"id":"1001001010101","name":"some-repo","url":"https:\\/\\/api.github.com\\/repos\\/some-repo"},"type":"PushEvent"} diff --git a/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference b/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference index 94777ffab1f..75e55e0376d 100644 --- a/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference +++ b/tests/queries/0_stateless/03206_json_parsing_and_formatting.reference @@ -3,17 +3,17 @@ JSON with no arguments {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','e':'String'} {} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} -{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} +{'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} {'a.b.c':'Int64','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} {} 1 2020-01-01 {"e":{"f":["s1","s2"]}} 2 [1,2,3] {"e":{"g":"43"}} 3 \N {} 4 \N {} -5 ['b1','b2'] {"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}} +5 ['b1','b2'] {"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}} JSON(a.b Tuple(c UInt32, d Array(Bool)), SKIP d.e, SKIP c, SKIP REGEXP '.*h.*') {"a":{"b":{"c":1,"d":[false,true]}},"b":"2020-01-01"} {"a":{"b":{"c":2,"d":[true,true]}},"b":["1","2","3"]} @@ -30,55 +30,55 @@ JSON(a.b.c UInt32, max_dynamic_paths=2) {"a":{"b":{"c":2,"d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":3,"d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":4,"d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.d':'Array(Nullable(Int64))','b':'Date'} {'c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))'} {'d.e.g':'Int64'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.d':'Array(Nullable(Int64))'} {'e':'String'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.d':'Array(Nullable(Int64))'} {'c':'Int64'} -{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))'} {'d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))'} {'d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} JSON(a.b.c UInt32, max_dynamic_paths=0) {"a":{"b":{"c":1,"d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":2,"d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":3,"d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":4,"d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','e':'String'} {} {'a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} {'a.b.d':'Array(Nullable(Int64))','c':'Int64'} -{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} {} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} JSON(a.b.c UInt32, max_dynamic_types=1) {"a":{"b":{"c":1,"d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":2,"d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":3,"d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":4,"d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":5,"d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {'a.b.d':'Array(Nullable(Int64))','b':'Date','c':'Int64','d.e.f':'Array(Nullable(String))'} {} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(Int64))','d.e.g':'Int64'} {} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','e':'String'} {'a.b.d':'Array(Nullable(Int64))','e':'String'} {} {'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','c':'Int64'} {'a.b.d':'Array(Nullable(Int64))','c':'Int64'} {} -{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime64(9)'} {} +{'a.b.c':'UInt32','a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} {'a.b.d':'Array(Nullable(Int64))','b':'Array(Nullable(String))','d.e.f':'Array(Nullable(String))','d.e.g':'Int64','d.e.h':'DateTime'} {} Test small max_read_buffer_size {"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} {"a":{"b":{"c":"1","d":["0","1"]}},"b":"2020-01-01","c":"42","d":{"e":{"f":["s1","s2"]}}} {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} Test PrettyJSONEachRow { "json": { @@ -176,7 +176,7 @@ Test PrettyJSONEachRow "s4" ], "g" : "44", - "h" : "2020-02-02 10:00:00.000000000" + "h" : "2020-02-02 10:00:00" } } } @@ -186,10 +186,10 @@ Test TSV {"a":{"b":{"c":"2","d":["2","3"]}},"b":["1","2","3"],"d":{"e":{"g":"43"}}} {"a":{"b":{"c":"3","d":["4","5"]}},"e":"Hello, World!"} {"a":{"b":{"c":"4","d":["6","7"]}},"c":"43"} -{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00.000000000"}}} +{"a":{"b":{"c":"5","d":["8","9"]}},"b":["b1","b2"],"d":{"e":{"f":["s3","s4"],"g":"44","h":"2020-02-02 10:00:00"}}} Test CSV "{""a"":{""b"":{""c"":""1"",""d"":[""0"",""1""]}},""b"":""2020-01-01"",""c"":""42"",""d"":{""e"":{""f"":[""s1"",""s2""]}}}" "{""a"":{""b"":{""c"":""2"",""d"":[""2"",""3""]}},""b"":[""1"",""2"",""3""],""d"":{""e"":{""g"":""43""}}}" "{""a"":{""b"":{""c"":""3"",""d"":[""4"",""5""]}},""e"":""Hello, World!""}" "{""a"":{""b"":{""c"":""4"",""d"":[""6"",""7""]}},""c"":""43""}" -"{""a"":{""b"":{""c"":""5"",""d"":[""8"",""9""]}},""b"":[""b1"",""b2""],""d"":{""e"":{""f"":[""s3"",""s4""],""g"":""44"",""h"":""2020-02-02 10:00:00.000000000""}}}" +"{""a"":{""b"":{""c"":""5"",""d"":[""8"",""9""]}},""b"":[""b1"",""b2""],""d"":{""e"":{""f"":[""s3"",""s4""],""g"":""44"",""h"":""2020-02-02 10:00:00""}}}" diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 index 972cfd9c37f..a93a2259442 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_compact_merge_tree.reference.j2 @@ -1,6 +1,6 @@ ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._25','Int64') @@ -17,7 +17,7 @@ { "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -54,16 +54,16 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -100,8 +100,8 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } @@ -116,15 +116,15 @@ "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.non.existing.path.:`Int64`": [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] } @@ -132,7 +132,7 @@ "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { @@ -148,23 +148,23 @@ "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], @@ -176,28 +176,28 @@ "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } @@ -220,19 +220,19 @@ "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -265,13 +265,13 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], @@ -280,7 +280,7 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -310,19 +310,19 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -349,19 +349,19 @@ "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -370,50 +370,50 @@ "json.b.b._26.:`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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._25','Int64') @@ -430,7 +430,7 @@ { "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -467,16 +467,16 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -513,8 +513,8 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } @@ -529,15 +529,15 @@ "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.non.existing.path.:`Int64`": [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] } @@ -545,7 +545,7 @@ "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { @@ -561,23 +561,23 @@ "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], @@ -589,28 +589,28 @@ "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } @@ -633,19 +633,19 @@ "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -678,13 +678,13 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], @@ -693,7 +693,7 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -723,19 +723,19 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -762,19 +762,19 @@ "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -783,44 +783,44 @@ "json.b.b._26.:`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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference index a7361856bc1..6276be52c0d 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_memory.reference @@ -1,6 +1,6 @@ ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._25','Int64') @@ -17,7 +17,7 @@ { "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -54,16 +54,16 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -100,8 +100,8 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } @@ -116,15 +116,15 @@ "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.non.existing.path.:`Int64`": [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] } @@ -132,7 +132,7 @@ "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { @@ -148,23 +148,23 @@ "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], @@ -176,28 +176,28 @@ "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } @@ -220,19 +220,19 @@ "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -265,13 +265,13 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], @@ -280,7 +280,7 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -310,19 +310,19 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -349,19 +349,19 @@ "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -370,44 +370,44 @@ "json.b.b._26.:`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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 index 972cfd9c37f..a93a2259442 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_1_wide_merge_tree.reference.j2 @@ -1,6 +1,6 @@ ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._25','Int64') @@ -17,7 +17,7 @@ { "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -54,16 +54,16 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -100,8 +100,8 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } @@ -116,15 +116,15 @@ "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.non.existing.path.:`Int64`": [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] } @@ -132,7 +132,7 @@ "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { @@ -148,23 +148,23 @@ "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], @@ -176,28 +176,28 @@ "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } @@ -220,19 +220,19 @@ "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -265,13 +265,13 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], @@ -280,7 +280,7 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -310,19 +310,19 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -349,19 +349,19 @@ "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -370,50 +370,50 @@ "json.b.b._26.:`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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._25','Int64') @@ -430,7 +430,7 @@ { "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -467,16 +467,16 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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.:`UUID`": [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], "json.a.b.e": [null, null, null, null, null, null, null, null, null, null, "str_10", "str_11", "str_12", "str_13", "str_14", null, null, null, null, null, "str_20", "str_21", "str_22", "str_23", "str_24", "str_25", "str_26", "str_27", "str_28", "str_29", "str_30", "str_31", "str_32", "str_33", "str_34", "str_35", "str_36", "str_37", "str_38", "str_39"], @@ -513,8 +513,8 @@ "json.d.c.:`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, "1970-01-31", "1970-02-01", "1970-02-02", "1970-02-03", "1970-02-04", null, null, null, null, null], "json.d.c.:`UUID`": [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], "json.^`n`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.^`a`.b": [{"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":5}, {"c":6}, {"c":7}, {"c":8}, {"c":9}, {"c":0,"d":"10","e":"str_10"}, {"c":0,"d":"11","e":"str_11"}, {"c":0,"d":"12","e":"str_12"}, {"c":0,"d":"13","e":"str_13"}, {"c":0,"d":"14","e":"str_14"}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":0}, {"c":20,"d":"20","e":"str_20"}, {"c":21,"d":"21","e":"str_21"}, {"c":22,"d":"22","e":"str_22"}, {"c":23,"d":"23","e":"str_23"}, {"c":24,"d":"24","e":"str_24"}, {"c":25,"d":"25","e":"str_25"}, {"c":26,"d":"26","e":"str_26"}, {"c":27,"d":"27","e":"str_27"}, {"c":28,"d":"28","e":"str_28"}, {"c":29,"d":"29","e":"str_29"}, {"c":30,"d":[],"e":"str_30"}, {"c":31,"d":[],"e":"str_31"}, {"c":32,"d":[],"e":"str_32"}, {"c":33,"d":[],"e":"str_33"}, {"c":34,"d":[],"e":"str_34"}, {"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}, {"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}, {"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}, {"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}, {"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}], "json.^`b`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"b":{"d":"15","e":"str_15"}}, {"b":{"d":"16","e":"str_16"}}, {"b":{"d":"17","e":"str_17"}}, {"b":{"d":"18","e":"str_18"}}, {"b":{"d":"19","e":"str_19"}}, {}, {}, {}, {}, {}, {"b":{"_25":"25"}}, {"b":{"_26":"26"}}, {"b":{"_27":"27"}}, {"b":{"_28":"28"}}, {"b":{"_29":"29"}}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}], "json.^`d`": [{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {"a":"30","c":"1970-01-31"}, {"a":"31","c":"1970-02-01"}, {"a":"32","c":"1970-02-02"}, {"a":"33","c":"1970-02-03"}, {"a":"34","c":"1970-02-04"}, {"a":["0"],"b":"35"}, {"a":["0","1"],"b":"36"}, {"a":["0","1","2"],"b":"37"}, {"a":["0","1","2","3"],"b":"38"}, {"a":["0","1","2","3","4"],"b":"39"}] } @@ -529,15 +529,15 @@ "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path.:`Int64`": [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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.non.existing.path": [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], "json.non.existing.path.:`Int64`": [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] } @@ -545,7 +545,7 @@ "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { @@ -561,23 +561,23 @@ "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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] } { "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], @@ -589,28 +589,28 @@ "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } @@ -633,19 +633,19 @@ "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -678,13 +678,13 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], @@ -693,7 +693,7 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.b.b.e": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`String`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, "str_15", "str_16", "str_17", "str_18", "str_19", null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b.e.:`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], @@ -723,19 +723,19 @@ "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.b": [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, "35", "36", "37", "38", "39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.d.b.:`Int64`": [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, "35", "36", "37", "38", "39"], "json.d.b.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -762,19 +762,19 @@ "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.b.b._26": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], "json.b.b._26.:`Int64`": [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, "26", null, null, null, null, null, null, null, null, null, null, null, null, null], "json.b.b._26.:`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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], "json.d.a": [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, "30", "31", "32", "33", "34", ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`Array(Nullable(Int64))`": [[], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], ["0"], ["0","1"], ["0","1","2"], ["0","1","2","3"], ["0","1","2","3","4"]], "json.d.a.:`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], @@ -783,44 +783,44 @@ "json.b.b._26.:`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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "json.a.b.c": [0, 0, 0, 0, 0, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"] + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], "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] } { - "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], - "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35.000000000","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36.000000000","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37.000000000","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38.000000000","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39.000000000","e":"str_39"}}], - "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35.000000000", "1970-01-01 00:00:36.000000000", "1970-01-01 00:00:37.000000000", "1970-01-01 00:00:38.000000000", "1970-01-01 00:00:39.000000000"], + "json": [{"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":0}}}, {"a":{"b":{"c":5}}}, {"a":{"b":{"c":6}}}, {"a":{"b":{"c":7}}}, {"a":{"b":{"c":8}}}, {"a":{"b":{"c":9}}}, {"a":{"b":{"c":0,"d":"10","e":"str_10"}}}, {"a":{"b":{"c":0,"d":"11","e":"str_11"}}}, {"a":{"b":{"c":0,"d":"12","e":"str_12"}}}, {"a":{"b":{"c":0,"d":"13","e":"str_13"}}}, {"a":{"b":{"c":0,"d":"14","e":"str_14"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"15","e":"str_15"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"16","e":"str_16"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"17","e":"str_17"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"18","e":"str_18"}}}, {"a":{"b":{"c":0}},"b":{"b":{"d":"19","e":"str_19"}}}, {"a":{"b":{"c":20,"d":"20","e":"str_20"}}}, {"a":{"b":{"c":21,"d":"21","e":"str_21"}}}, {"a":{"b":{"c":22,"d":"22","e":"str_22"}}}, {"a":{"b":{"c":23,"d":"23","e":"str_23"}}}, {"a":{"b":{"c":24,"d":"24","e":"str_24"}}}, {"a":{"b":{"c":25,"d":"25","e":"str_25"}},"b":{"b":{"_25":"25"}}}, {"a":{"b":{"c":26,"d":"26","e":"str_26"}},"b":{"b":{"_26":"26"}}}, {"a":{"b":{"c":27,"d":"27","e":"str_27"}},"b":{"b":{"_27":"27"}}}, {"a":{"b":{"c":28,"d":"28","e":"str_28"}},"b":{"b":{"_28":"28"}}}, {"a":{"b":{"c":29,"d":"29","e":"str_29"}},"b":{"b":{"_29":"29"}}}, {"a":{"b":{"c":30,"d":[],"e":"str_30"}},"d":{"a":"30","c":"1970-01-31"}}, {"a":{"b":{"c":31,"d":[],"e":"str_31"}},"d":{"a":"31","c":"1970-02-01"}}, {"a":{"b":{"c":32,"d":[],"e":"str_32"}},"d":{"a":"32","c":"1970-02-02"}}, {"a":{"b":{"c":33,"d":[],"e":"str_33"}},"d":{"a":"33","c":"1970-02-03"}}, {"a":{"b":{"c":34,"d":[],"e":"str_34"}},"d":{"a":"34","c":"1970-02-04"}}, {"a":{"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}},"d":{"a":["0"],"b":"35"}}, {"a":{"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}},"d":{"a":["0","1"],"b":"36"}}, {"a":{"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}},"d":{"a":["0","1","2"],"b":"37"}}, {"a":{"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}},"d":{"a":["0","1","2","3"],"b":"38"}}, {"a":{"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}},"d":{"a":["0","1","2","3","4"],"b":"39"}}], + "json.^`a`": [{"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":5}}, {"b":{"c":6}}, {"b":{"c":7}}, {"b":{"c":8}}, {"b":{"c":9}}, {"b":{"c":0,"d":"10","e":"str_10"}}, {"b":{"c":0,"d":"11","e":"str_11"}}, {"b":{"c":0,"d":"12","e":"str_12"}}, {"b":{"c":0,"d":"13","e":"str_13"}}, {"b":{"c":0,"d":"14","e":"str_14"}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":0}}, {"b":{"c":20,"d":"20","e":"str_20"}}, {"b":{"c":21,"d":"21","e":"str_21"}}, {"b":{"c":22,"d":"22","e":"str_22"}}, {"b":{"c":23,"d":"23","e":"str_23"}}, {"b":{"c":24,"d":"24","e":"str_24"}}, {"b":{"c":25,"d":"25","e":"str_25"}}, {"b":{"c":26,"d":"26","e":"str_26"}}, {"b":{"c":27,"d":"27","e":"str_27"}}, {"b":{"c":28,"d":"28","e":"str_28"}}, {"b":{"c":29,"d":"29","e":"str_29"}}, {"b":{"c":30,"d":[],"e":"str_30"}}, {"b":{"c":31,"d":[],"e":"str_31"}}, {"b":{"c":32,"d":[],"e":"str_32"}}, {"b":{"c":33,"d":[],"e":"str_33"}}, {"b":{"c":34,"d":[],"e":"str_34"}}, {"b":{"c":35,"d":"1970-01-01 00:00:35","e":"str_35"}}, {"b":{"c":36,"d":"1970-01-01 00:00:36","e":"str_36"}}, {"b":{"c":37,"d":"1970-01-01 00:00:37","e":"str_37"}}, {"b":{"c":38,"d":"1970-01-01 00:00:38","e":"str_38"}}, {"b":{"c":39,"d":"1970-01-01 00:00:39","e":"str_39"}}], + "json.a.b.d": [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", [], [], [], [], [], "1970-01-01 00:00:35", "1970-01-01 00:00:36", "1970-01-01 00:00:37", "1970-01-01 00:00:38", "1970-01-01 00:00:39"], "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] } diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 index 13343b21a8c..e1e69879cfb 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_compact_merge_tree.reference.j2 @@ -1,6 +1,6 @@ ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._0','Int64') @@ -33,7 +33,7 @@ 0 ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._0','Int64') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference index 6c455b1bb0d..1ef53fb5716 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_memory.reference @@ -1,6 +1,6 @@ ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._0','Int64') diff --git a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 index 13343b21a8c..e1e69879cfb 100644 --- a/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 +++ b/tests/queries/0_stateless/03207_json_read_subcolumns_2_wide_merge_tree.reference.j2 @@ -1,6 +1,6 @@ ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._0','Int64') @@ -33,7 +33,7 @@ 0 ('a.b.c','UInt32') ('a.b.d','Array(Nullable(String))') -('a.b.d','DateTime64(9)') +('a.b.d','DateTime') ('a.b.d','Int64') ('a.b.e','String') ('b.b._0','Int64') From dc2f455b810bbe3c457e294fbb4518bdb1766eca Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 00:19:39 +0000 Subject: [PATCH 1038/1722] Try to mitigate 02818_memory_profiler_sample_min_max_allocation_size flakiness --- .../02818_memory_profiler_sample_min_max_allocation_size.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh index 9234c428147..e2afc1d208c 100755 --- a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" -${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" +${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=16384 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" @@ -14,4 +14,4 @@ ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" # show wrong allocations -${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" +${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 16384 or abs(size) < 4096)" From fb7afa779c7ea7d56c67da10a634133fe8416f97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 03:59:17 +0200 Subject: [PATCH 1039/1722] Support expressions with tuples like `expr().name` --- src/Parsers/ExpressionListParsers.cpp | 17 ++++++++++++++--- .../03224_tuple_element_identifier.reference | 4 ++++ .../03224_tuple_element_identifier.sql | 13 +++++++++++++ 3 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03224_tuple_element_identifier.reference create mode 100644 tests/queries/0_stateless/03224_tuple_element_identifier.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index d38dc6d5f37..ad6b8e13ea6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2811,8 +2811,8 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po if (op.type == OperatorType::TupleElement) { ASTPtr tmp; - if (asterisk_parser.parse(pos, tmp, expected) || - columns_matcher_parser.parse(pos, tmp, expected)) + if (asterisk_parser.parse(pos, tmp, expected) + || columns_matcher_parser.parse(pos, tmp, expected)) { if (auto * asterisk = tmp->as()) { @@ -2833,6 +2833,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po layers.back()->pushOperand(std::move(tmp)); return Action::OPERATOR; } + + /// If it is an identifier, + /// replace it with literal, because an expression `expr().elem` + /// should be transformed to `tupleElement(expr(), 'elem')` for query analysis, + /// otherwise the identifier `elem` will not be found. + if (ParserIdentifier().parse(pos, tmp, expected)) + { + layers.back()->pushOperator(op); + layers.back()->pushOperand(std::make_shared(tmp->as()->name())); + return Action::OPERATOR; + } } /// isNull & isNotNull are postfix unary operators @@ -2863,7 +2874,7 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po layers.push_back(std::make_unique()); if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween) - layers.back()->between_counter++; + ++layers.back()->between_counter; return Action::OPERAND; } diff --git a/tests/queries/0_stateless/03224_tuple_element_identifier.reference b/tests/queries/0_stateless/03224_tuple_element_identifier.reference new file mode 100644 index 00000000000..0fc9e7410c1 --- /dev/null +++ b/tests/queries/0_stateless/03224_tuple_element_identifier.reference @@ -0,0 +1,4 @@ +([('wtf')]) [('wtf')] wtf +([('wtf')]) [('wtf')] wtf +Hello +('Hello') Hello Hello Hello diff --git a/tests/queries/0_stateless/03224_tuple_element_identifier.sql b/tests/queries/0_stateless/03224_tuple_element_identifier.sql new file mode 100644 index 00000000000..2a7fb9a97a3 --- /dev/null +++ b/tests/queries/0_stateless/03224_tuple_element_identifier.sql @@ -0,0 +1,13 @@ +SET enable_analyzer = 1; + +SELECT JSONExtract('{"hello":[{"world":"wtf"}]}', 'Tuple(hello Array(Tuple(world String)))') AS x, + x.hello, x.hello[1].world; + +SELECT JSONExtract('{"hello":[{" wow ":"wtf"}]}', 'Tuple(hello Array(Tuple(` wow ` String)))') AS x, + x.hello, x.hello[1].` wow `; + +SELECT JSONExtract('{"hello":[{" wow ":"wtf"}]}', 'Tuple(hello Array(Tuple(` wow ` String)))') AS x, + x.hello, x.hello[1].`wow`; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK } + +SELECT ('Hello' AS world,).world; +SELECT ('Hello' AS world,) AS t, t.world, (t).world, identity(t).world; From 1b52466cbbbbccad74ca9c92c650758c1dfdb8d2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 05:36:03 +0000 Subject: [PATCH 1040/1722] () --- src/Common/StackTrace.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 80329b4aec1..bd01b639913 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -267,7 +267,7 @@ void StackTrace::forEachFrame( /// frame is interrupted by signal or not. We could propagate this information /// from libunwind to here and avoid subtracting 1 in this case, but currently /// we don't do this. - /// But we don't do the decrement for findSymbol() below (because `call` is + /// But we don't do the decrement for findSymbol below (because `call` is /// ~never the last instruction of a function), so the function name should be /// correct for both pre-signal frames and regular frames. adjusted_addr -= 1; From 253188381759d062967c9bb3b9d0907f30eab61a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 05:39:50 +0000 Subject: [PATCH 1041/1722] she sells seashells by seashore the shells that she sells are seashells im sure --- tests/queries/0_stateless/01119_session_log.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01119_session_log.sh b/tests/queries/0_stateless/01119_session_log.sh index 809d300fada..2d17b545276 100755 --- a/tests/queries/0_stateless/01119_session_log.sh +++ b/tests/queries/0_stateless/01119_session_log.sh @@ -35,7 +35,7 @@ $session_log_query_prefix and type != 'Logout' order by type, user, interface; " # Wait for logout events. -for attempt in {1..10} +for _ in {1..10} do if [ "`$CLICKHOUSE_CLIENT -q "$session_log_query_prefix and type = 'Logout'" | wc -l`" -eq 3 ] then From b3a3d1e7202837e2da45b6775b4ea8c389c881a7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 05:53:17 +0000 Subject: [PATCH 1042/1722] another rebase --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index eb35c10ac57..73752937366 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit eb35c10ac5725da7ef4be88b303895e1b5d153be +Subproject commit 737529373665bc067971ba098a12d6928580a0ae From 7c691cbb797594d0c6649b36be13ff99c3c4664b Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 16 Aug 2024 10:44:11 +0200 Subject: [PATCH 1043/1722] style --- tests/ci/integration_tests_runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index f63afc297e6..006958abe6d 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -774,10 +774,8 @@ class ClickhouseIntegrationTestsRunner: logging.info("Found '%s' tests to run", " ".join(tests_to_run)) result_state = "success" description_prefix = "No flaky tests: " - start = time.time() logging.info("Starting check with retries") final_retry = 0 - logs = [] counters = { "ERROR": [], "PASSED": [], @@ -858,7 +856,7 @@ class ClickhouseIntegrationTestsRunner: ] ) - return result_state, status_text, test_result, logs + return result_state, status_text, test_result, tests_log_paths def run_impl(self, repo_path, build_path): if self.flaky_check or self.bugfix_validate_check: From a85f544205f2e782d4f6c16c0622728475db3571 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Aug 2024 08:47:28 +0000 Subject: [PATCH 1044/1722] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index bd92465e1aa..c8edbdc5932 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,4 +1,3 @@ 01624_soft_constraints -02354_vector_search_queries # Check after ConstantNode refactoring 02944_variant_as_common_type From 1b49e2492521c54b0e6240d412af847d3fa21221 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 16 Aug 2024 11:26:31 +0200 Subject: [PATCH 1045/1722] Fix clang-tidy --- src/Dictionaries/PostgreSQLDictionarySource.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 8e472f85a6e..b1bab17e2e9 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -208,7 +208,6 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const auto & settings = context->getSettingsRef(); std::optional dictionary_configuration; - String database, schema, table; postgres::PoolWithFailover::ReplicasConfigurationByPriority replicas_by_priority; auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix, context) : nullptr; From 60a6e893a40761eb46655e76cb6a3fe5f177019c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 16 Aug 2024 17:56:12 +0800 Subject: [PATCH 1046/1722] first commit --- src/Common/examples/CMakeLists.txt | 5 + src/Common/examples/utf8_upper_lower.cpp | 27 ++ src/Functions/LowerUpperImpl.h | 1 - src/Functions/LowerUpperUTF8Impl.h | 283 +++--------------- src/Functions/initcapUTF8.cpp | 3 +- src/Functions/lowerUTF8.cpp | 25 +- src/Functions/upperUTF8.cpp | 24 +- .../00170_lower_upper_utf8.reference | 4 + .../0_stateless/00170_lower_upper_utf8.sql | 11 + .../00233_position_function_family.sql | 3 + .../0_stateless/00761_lower_utf8_bug.sql | 3 + .../0_stateless/01278_random_string_utf8.sql | 3 + .../0_stateless/01431_utf8_ubsan.reference | 4 +- .../queries/0_stateless/01431_utf8_ubsan.sql | 3 + .../0_stateless/01590_countSubstrings.sql | 3 + ...71_lower_upper_utf8_row_overlaps.reference | 4 +- .../02071_lower_upper_utf8_row_overlaps.sql | 3 + ...new_functions_must_be_documented.reference | 2 - .../02514_if_with_lazy_low_cardinality.sql | 3 + .../0_stateless/02807_lower_utf8_msan.sql | 3 + tests/queries/0_stateless/03015_peder1001.sql | 3 + 21 files changed, 159 insertions(+), 261 deletions(-) create mode 100644 src/Common/examples/utf8_upper_lower.cpp diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 69580d4ad0e..8383e80d09d 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -92,3 +92,8 @@ endif() clickhouse_add_executable (check_pointer_valid check_pointer_valid.cpp) target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io clickhouse_common_config) + +if (TARGET ch_contrib::icu) + clickhouse_add_executable (utf8_upper_lower utf8_upper_lower.cpp) + target_link_libraries (utf8_upper_lower PRIVATE ch_contrib::icu) +endif () diff --git a/src/Common/examples/utf8_upper_lower.cpp b/src/Common/examples/utf8_upper_lower.cpp new file mode 100644 index 00000000000..826e1763105 --- /dev/null +++ b/src/Common/examples/utf8_upper_lower.cpp @@ -0,0 +1,27 @@ +#include +#include + +std::string utf8_to_lower(const std::string & input) +{ + icu::UnicodeString unicodeInput(input.c_str(), "UTF-8"); + unicodeInput.toLower(); + std::string output; + unicodeInput.toUTF8String(output); + return output; +} + +std::string utf8_to_upper(const std::string & input) +{ + icu::UnicodeString unicodeInput(input.c_str(), "UTF-8"); + unicodeInput.toUpper(); + std::string output; + unicodeInput.toUTF8String(output); + return output; +} + +int main() +{ + std::string input = "ır"; + std::cout << "upper:" << utf8_to_upper(input) << std::endl; + return 0; +} diff --git a/src/Functions/LowerUpperImpl.h b/src/Functions/LowerUpperImpl.h index d463ef96e16..a52703d10c8 100644 --- a/src/Functions/LowerUpperImpl.h +++ b/src/Functions/LowerUpperImpl.h @@ -1,7 +1,6 @@ #pragma once #include - namespace DB { diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index eedabca5b22..5da085f48e5 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -1,15 +1,14 @@ #pragma once + +#include "config.h" + +#if USE_ICU + #include #include -#include -#include +#include +#include #include -#include - -#ifdef __SSE2__ -#include -#endif - namespace DB { @@ -19,71 +18,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -/// xor or do nothing -template -UInt8 xor_or_identity(const UInt8 c, const int mask) -{ - return c ^ mask; -} - -template <> -inline UInt8 xor_or_identity(const UInt8 c, const int) -{ - return c; -} - -/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array -template -inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) -{ - if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// ЀÐЂЃЄЅІЇЈЉЊЋЌÐÐŽÐ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// ÑёђѓєѕіїјљњћќÑўџ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// Ð-П - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) - { - /// а-п - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) - { - /// Р-Я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// Ñ€-Ñ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } -} - - -/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case. - * Note: It is assumed that after the character is converted to another case, - * the length of its multibyte sequence in UTF-8 does not change. - * Otherwise, the behavior is undefined. - */ -template +template struct LowerUpperUTF8Impl { static void vector( @@ -103,180 +38,46 @@ struct LowerUpperUTF8Impl return; } - res_data.resize_exact(data.size()); - res_offsets.assign(offsets); - array(data.data(), data.data() + data.size(), offsets, res_data.data()); + res_data.resize(data.size()); + res_offsets.resize_exact(offsets.size()); + + String output; + size_t curr_offset = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); + size_t size = offsets[i] - offsets[i - 1]; + + icu::UnicodeString input(data_start, static_cast(size), "UTF-8"); + if constexpr (upper) + input.toUpper(); + else + input.toLower(); + + output.clear(); + input.toUTF8String(output); + + /// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first + /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this + /// case, the behavior is also reasonable. + const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size()); + size_t valid_size = res_end ? res_end - output.data() + 1 : 0; + + res_data.resize(curr_offset + valid_size + 1); + memcpy(&res_data[curr_offset], output.data(), valid_size); + res_data[curr_offset + valid_size] = 0; + + curr_offset += valid_size + 1; + res_offsets[i] = curr_offset; + } } static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument"); } - - /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. - * `src` and `dst` are incremented by corresponding sequence lengths. */ - static bool toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst, bool partial) - { - if (src[0] <= ascii_upper_bound) - { - if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) - *dst++ = *src++ ^ flip_case_mask; - else - *dst++ = *src++; - } - else if (src + 1 < src_end - && ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu)))) - { - cyrillic_to_case(src, dst); - } - else if (src + 1 < src_end && src[0] == 0xC2u) - { - /// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF - *dst++ = *src++; - *dst++ = *src++; - } - else if (src + 2 < src_end && src[0] == 0xE2u) - { - /// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF - *dst++ = *src++; - *dst++ = *src++; - *dst++ = *src++; - } - else - { - size_t src_sequence_length = UTF8::seqLength(*src); - /// In case partial buffer was passed (due to SSE optimization) - /// we cannot convert it with current src_end, but we may have more - /// bytes to convert and eventually got correct symbol. - if (partial && src_sequence_length > static_cast(src_end - src)) - return false; - - auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src); - if (src_code_point) - { - int dst_code_point = to_case(*src_code_point); - if (dst_code_point > 0) - { - size_t dst_sequence_length = UTF8::convertCodePointToUTF8(dst_code_point, dst, src_end - src); - assert(dst_sequence_length <= 4); - - /// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8. - /// As an example, this happens for ß and ẞ. - if (dst_sequence_length == src_sequence_length) - { - src += dst_sequence_length; - dst += dst_sequence_length; - return true; - } - } - } - - *dst = *src; - ++dst; - ++src; - } - - return true; - } - -private: - static constexpr auto ascii_upper_bound = '\x7f'; - static constexpr auto flip_case_mask = 'A' ^ 'a'; - - static void array(const UInt8 * src, const UInt8 * src_end, const ColumnString::Offsets & offsets, UInt8 * dst) - { - const auto * offset_it = offsets.begin(); - const UInt8 * begin = src; - -#ifdef __SSE2__ - static constexpr auto bytes_sse = sizeof(__m128i); - - /// If we are before this position, we can still read at least bytes_sse. - const auto * src_end_sse = src_end - bytes_sse + 1; - - /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) - const auto v_zero = _mm_setzero_si128(); - const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); - const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); - const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); - - while (src < src_end_sse) - { - const auto chars = _mm_loadu_si128(reinterpret_cast(src)); - - /// check for ASCII - const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero); - const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii); - - /// ASCII - if (mask_is_not_ascii == 0) - { - const auto is_not_case - = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); - const auto mask_is_not_case = _mm_movemask_epi8(is_not_case); - - /// everything in correct case ASCII - if (mask_is_not_case == 0) - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars); - else - { - /// ASCII in mixed case - /// keep `flip_case_mask` only where necessary, zero out elsewhere - const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); - - /// flip case by applying calculated mask - const auto cased_chars = _mm_xor_si128(chars, xor_mask); - - /// store result back to destination - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); - } - - src += bytes_sse; - dst += bytes_sse; - } - else - { - /// UTF-8 - - /// Find the offset of the next string after src - size_t offset_from_begin = src - begin; - while (offset_from_begin >= *offset_it) - ++offset_it; - - /// Do not allow one row influence another (since row may have invalid sequence, and break the next) - const UInt8 * row_end = begin + *offset_it; - chassert(row_end >= src); - const UInt8 * expected_end = std::min(src + bytes_sse, row_end); - - while (src < expected_end) - { - if (!toCase(src, expected_end, dst, /* partial= */ true)) - { - /// Fallback to handling byte by byte. - src_end_sse = src; - break; - } - } - } - } - - /// Find the offset of the next string after src - size_t offset_from_begin = src - begin; - while (offset_it != offsets.end() && offset_from_begin >= *offset_it) - ++offset_it; -#endif - - /// handle remaining symbols, row by row (to avoid influence of bad UTF8 symbols from one row, to another) - while (src < src_end) - { - const UInt8 * row_end = begin + *offset_it; - chassert(row_end >= src); - - while (src < row_end) - toCase(src, row_end, dst, /* partial= */ false); - ++offset_it; - } - } }; } + +#endif diff --git a/src/Functions/initcapUTF8.cpp b/src/Functions/initcapUTF8.cpp index 282d846094e..004586dce26 100644 --- a/src/Functions/initcapUTF8.cpp +++ b/src/Functions/initcapUTF8.cpp @@ -1,9 +1,8 @@ #include #include -#include #include #include - +#include namespace DB { diff --git a/src/Functions/lowerUTF8.cpp b/src/Functions/lowerUTF8.cpp index 7adb0069121..e2f7cb84730 100644 --- a/src/Functions/lowerUTF8.cpp +++ b/src/Functions/lowerUTF8.cpp @@ -1,9 +1,10 @@ -#include +#include "config.h" + +#if USE_ICU + +#include #include #include -#include -#include - namespace DB { @@ -15,13 +16,25 @@ struct NameLowerUTF8 static constexpr auto name = "lowerUTF8"; }; -using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; +using FunctionLowerUTF8 = FunctionStringToString, NameLowerUTF8>; } REGISTER_FUNCTION(LowerUTF8) { - factory.registerFunction(); + FunctionDocumentation::Description description + = R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)"; + FunctionDocumentation::Syntax syntax = "lowerUTF8(input)"; + FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}}; + FunctionDocumentation::ReturnedValue returned_value = "A String data type value"; + FunctionDocumentation::Examples examples = { + {"first", "SELECT lowerUTF8('München') as Lowerutf8;", "münchen"}, + }; + FunctionDocumentation::Categories categories = {"String"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); } } + +#endif diff --git a/src/Functions/upperUTF8.cpp b/src/Functions/upperUTF8.cpp index 659e67f0ef3..ef26430331f 100644 --- a/src/Functions/upperUTF8.cpp +++ b/src/Functions/upperUTF8.cpp @@ -1,8 +1,10 @@ +#include "config.h" + +#if USE_ICU + +#include #include #include -#include -#include - namespace DB { @@ -14,13 +16,25 @@ struct NameUpperUTF8 static constexpr auto name = "upperUTF8"; }; -using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; +using FunctionUpperUTF8 = FunctionStringToString, NameUpperUTF8>; } REGISTER_FUNCTION(UpperUTF8) { - factory.registerFunction(); + FunctionDocumentation::Description description + = R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)"; + FunctionDocumentation::Syntax syntax = "upperUTF8(input)"; + FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}}; + FunctionDocumentation::ReturnedValue returned_value = "A String data type value"; + FunctionDocumentation::Examples examples = { + {"first", "SELECT upperUTF8('München') as Upperutf8;", "MÃœNCHEN"}, + }; + FunctionDocumentation::Categories categories = {"String"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); } } + +#endif diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.reference b/tests/queries/0_stateless/00170_lower_upper_utf8.reference index f202cb75513..3c644f22b9b 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.reference +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.reference @@ -22,3 +22,7 @@ 1 1 1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.sql b/tests/queries/0_stateless/00170_lower_upper_utf8.sql index 4caba2033ff..85b6c5c6095 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.sql +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + select lower('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str; select lowerUTF8('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str; select lower('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'aaaaaaaaaaaaaaa012345789,.!aaaa'; @@ -27,3 +30,11 @@ select sum(lower(materialize('aaaaÐБВГAAAAaaAA')) = materialize('aaaaÐБВ select sum(upper(materialize('aaaaÐБВГAAAAaaAA')) = materialize('AAAAÐБВГAAAAAAAA')) = count() from system.one array join range(16384) as n; select sum(lowerUTF8(materialize('aaaaÐБВГAAAAaaAA')) = materialize('aaaaабвгaaaaaaaa')) = count() from system.one array join range(16384) as n; select sum(upperUTF8(materialize('aaaaÐБВГAAAAaaAA')) = materialize('AAAAÐБВГAAAAAAAA')) = count() from system.one array join range(16384) as n; + +-- Turkish language +select upperUTF8('ır') = 'IR'; +select lowerUTF8('ır') = 'ır'; + +-- German language +select upper('öäüß') = 'öäüß'; +select lower('ÖÄÜẞ') = 'ÖÄÜẞ'; diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index dd7394bc39a..d6668cb7ba4 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SET send_logs_level = 'fatal'; select 1 = position('', ''); diff --git a/tests/queries/0_stateless/00761_lower_utf8_bug.sql b/tests/queries/0_stateless/00761_lower_utf8_bug.sql index de20b894331..a0ab55edc15 100644 --- a/tests/queries/0_stateless/00761_lower_utf8_bug.sql +++ b/tests/queries/0_stateless/00761_lower_utf8_bug.sql @@ -1 +1,4 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT lowerUTF8('\xF0') = lowerUTF8('\xF0'); diff --git a/tests/queries/0_stateless/01278_random_string_utf8.sql b/tests/queries/0_stateless/01278_random_string_utf8.sql index da2dc48c3e1..290d6a0c759 100644 --- a/tests/queries/0_stateless/01278_random_string_utf8.sql +++ b/tests/queries/0_stateless/01278_random_string_utf8.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT randomStringUTF8('string'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT lengthUTF8(randomStringUTF8(100)); SELECT toTypeName(randomStringUTF8(10)); diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.reference b/tests/queries/0_stateless/01431_utf8_ubsan.reference index c98c950d535..dc785e57851 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.reference +++ b/tests/queries/0_stateless/01431_utf8_ubsan.reference @@ -1,2 +1,2 @@ -FF -FF +EFBFBD +EFBFBD diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.sql b/tests/queries/0_stateless/01431_utf8_ubsan.sql index d6a299225b1..3a28e023805 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.sql +++ b/tests/queries/0_stateless/01431_utf8_ubsan.sql @@ -1,2 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT hex(lowerUTF8('\xFF')); SELECT hex(upperUTF8('\xFF')); diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index b38cbb7d188..5ec4f412d7f 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + -- -- countSubstrings -- diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index a3bac432482..deabef61a88 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo -- MONOGRAM FOR YANG with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV; -1,"0xE2","0xE2" +1,"�","�" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -1,"0xE2","0xE2" +1,"�","�" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -- NOTE: regression test for introduced bug -- https://github.com/ClickHouse/ClickHouse/issues/42756 diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql index 8ca0a3f5f75..d175e0659d0 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + drop table if exists utf8_overlap; create table utf8_overlap (str String) engine=Memory(); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index c39f1fb1ce9..0980e25b70f 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -416,7 +416,6 @@ logTrace lowCardinalityIndices lowCardinalityKeys lower -lowerUTF8 makeDate makeDate32 makeDateTime @@ -897,7 +896,6 @@ tupleToNameValuePairs unbin unhex upper -upperUTF8 uptime validateNestedArraySizes version diff --git a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql index 80e3c0a9ece..b169cfd0ab9 100644 --- a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql +++ b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + create table if not exists t (`arr.key` Array(LowCardinality(String)), `arr.value` Array(LowCardinality(String))) engine = Memory; insert into t (`arr.key`, `arr.value`) values (['a'], ['b']); select if(true, if(lowerUTF8(arr.key) = 'a', 1, 2), 3) as x from t left array join arr; diff --git a/tests/queries/0_stateless/02807_lower_utf8_msan.sql b/tests/queries/0_stateless/02807_lower_utf8_msan.sql index e9eb18bf615..95f224577f7 100644 --- a/tests/queries/0_stateless/02807_lower_utf8_msan.sql +++ b/tests/queries/0_stateless/02807_lower_utf8_msan.sql @@ -1,2 +1,5 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + SELECT lowerUTF8(arrayJoin(['©--------------------------------------', '©--------------------'])) ORDER BY 1; SELECT upperUTF8(materialize('aaaaÐБВГaaaaaaaaaaaaÐБВГAAAAaaAA')) FROM numbers(2); diff --git a/tests/queries/0_stateless/03015_peder1001.sql b/tests/queries/0_stateless/03015_peder1001.sql index 810503207f2..df8e4db1536 100644 --- a/tests/queries/0_stateless/03015_peder1001.sql +++ b/tests/queries/0_stateless/03015_peder1001.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest: upper/lowerUTF8 use ICU + DROP TABLE IF EXISTS test_data; CREATE TABLE test_data From 4600b270dafec20b276ab83eb557270c24cb4169 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 16 Aug 2024 17:58:54 +0800 Subject: [PATCH 1047/1722] remote icu contrib --- .gitmodules | 3 --- contrib/icu | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/icu diff --git a/.gitmodules b/.gitmodules index 7fdfb1103c5..164da311930 100644 --- a/.gitmodules +++ b/.gitmodules @@ -106,9 +106,6 @@ [submodule "contrib/icudata"] path = contrib/icudata url = https://github.com/ClickHouse/icudata -[submodule "contrib/icu"] - path = contrib/icu - url = https://github.com/unicode-org/icu [submodule "contrib/flatbuffers"] path = contrib/flatbuffers url = https://github.com/ClickHouse/flatbuffers diff --git a/contrib/icu b/contrib/icu deleted file mode 160000 index 7750081bda4..00000000000 --- a/contrib/icu +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 From 3ee741bd5e33d16b2f5711a8f2b06fca1a64b7bc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 16 Aug 2024 18:04:15 +0800 Subject: [PATCH 1048/1722] add submodule contrib/icu from clickhouse --- .gitmodules | 4 ++++ contrib/icu | 1 + 2 files changed, 5 insertions(+) create mode 160000 contrib/icu diff --git a/.gitmodules b/.gitmodules index 164da311930..a8cc6a07caf 100644 --- a/.gitmodules +++ b/.gitmodules @@ -372,3 +372,7 @@ [submodule "contrib/numactl"] path = contrib/numactl url = https://github.com/ClickHouse/numactl.git +[submodule "contrib/icu"] + path = contrib/icu + url = https://github.com/ClickHouse/icu + branch = ClickHouse/release-75-1 diff --git a/contrib/icu b/contrib/icu new file mode 160000 index 00000000000..4216173eeeb --- /dev/null +++ b/contrib/icu @@ -0,0 +1 @@ +Subproject commit 4216173eeeb39c1d4caaa54a68860e800412d273 From b76277ca00b3d91052bf90b83b20f730dcf96b2e Mon Sep 17 00:00:00 2001 From: neoman36 Date: Fri, 16 Aug 2024 12:23:20 +0200 Subject: [PATCH 1049/1722] named-collections.md is updated --- docs/en/operations/named-collections.md | 22 +++++++++++++++++-- docs/ru/operations/named-collections.md | 29 ++++++++++++++++++++++++- 2 files changed, 48 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 59ee05d1f9e..16c75814e5a 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -307,8 +307,22 @@ SELECT dictGet('dict', 'B', 2); ## Named collections for accessing PostgreSQL database -The description of parameters see [postgresql](../sql-reference/table-functions/postgresql.md). +The description of parameters see [postgresql](../sql-reference/table-functions/postgresql.md). Additionally, there are aliases: +- `username` for `user` +- `db` for `database`. + +Parameter `addresses_expr` is used in a collection instead of `host:port`. The parameter is optional, because there are other optional ones: `host`, `hostname`, `port`. The following pseudocode explains the priority: + +```sql +CASE + WHEN collection['addresses_expr'] != '' THEN collection['addresses_expr'] + WHEN collection['host'] != '' THEN collection['host'] || ':' || if(collection['port'] != '', collection['port'], '5432') + WHEN collection['hostname'] != '' THEN collection['hostname'] || ':' || if(collection['port'] != '', collection['port'], '5432') +END +``` + +Example of creation: ```sql CREATE NAMED COLLECTION mypg AS user = 'pguser', @@ -316,7 +330,7 @@ password = 'jw8s0F4', host = '127.0.0.1', port = 5432, database = 'test', -schema = 'test_schema', +schema = 'test_schema' ``` Example of configuration: @@ -369,6 +383,10 @@ SELECT * FROM mypgtable; └───┘ ``` +:::note +PostgreSQL copies data from the named collection when the table is being created. A change in the collection does not affect the existing tables. +::: + ### Example of using named collections with database with engine PostgreSQL ```sql diff --git a/docs/ru/operations/named-collections.md b/docs/ru/operations/named-collections.md index 48ee7c9f15d..67656f24ba3 100644 --- a/docs/ru/operations/named-collections.md +++ b/docs/ru/operations/named-collections.md @@ -146,7 +146,30 @@ SELECT dictGet('dict', 'B', 2); ## Пример иÑÐ¿Ð¾Ð»ÑŒÐ·Ð¾Ð²Ð°Ð½Ð¸Ñ Ð¸Ð¼ÐµÐ½Ð¾Ð²Ð°Ð½Ð½Ñ‹Ñ… Ñоединений Ñ Ð±Ð°Ð·Ð¾Ð¹ данных PostgreSQL -ОпиÑание параметров Ñмотрите [postgresql](../sql-reference/table-functions/postgresql.md). +ОпиÑание параметров Ñмотрите [postgresql](../sql-reference/table-functions/postgresql.md). Дополнительно еÑÑ‚ÑŒ алиаÑÑ‹: +- `username` Ð´Ð»Ñ `user` +- `db` Ð´Ð»Ñ `database`. + +Параметр `addresses_expr` иÑпользуетÑÑ Ð² коллекции вмеÑто `host:port`. Параметр опционален, потому что еÑÑ‚ÑŒ так же другие: `host`, `hostname`, `port`. Следующий пÑевдокод показывает приоритет: + +```sql +CASE + WHEN collection['addresses_expr'] != '' THEN collection['addresses_expr'] + WHEN collection['host'] != '' THEN collection['host'] || ':' || if(collection['port'] != '', collection['port'], '5432') + WHEN collection['hostname'] != '' THEN collection['hostname'] || ':' || if(collection['port'] != '', collection['port'], '5432') +END +``` + +Пример ÑозданиÑ: +```sql +CREATE NAMED COLLECTION mypg AS +user = 'pguser', +password = 'jw8s0F4', +host = '127.0.0.1', +port = 5432, +database = 'test', +schema = 'test_schema' +``` Пример конфигурации: ```xml @@ -199,6 +222,10 @@ SELECT * FROM mypgtable; └───┘ ``` +:::note +PostgreSQL копирует данные из named collection при Ñоздании таблицы. Ð˜Ð·Ð¼ÐµÐ½ÐµÐ½Ð¸Ñ Ð² коллекции не влиÑÑŽÑ‚ на ÑущеÑтвующие таблицы. +::: + ### Пример иÑÐ¿Ð¾Ð»ÑŒÐ·Ð¾Ð²Ð°Ð½Ð¸Ñ Ð¸Ð¼ÐµÐ½Ð¾Ð²Ð°Ð½Ð½Ñ‹Ñ… Ñоединений базой данных Ñ Ð´Ð²Ð¸Ð¶ÐºÐ¾Ð¼ PostgreSQL ```sql From 5ff4d990e189dfee42eb57f567a5ff6313cfa8d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 16 Aug 2024 11:11:11 +0200 Subject: [PATCH 1050/1722] CI: Auto Releases in prod --- .github/workflows/auto_releases.yml | 76 ++++++++++------------------ .github/workflows/create_release.yml | 2 + tests/ci/auto_release.py | 9 ++++ tests/ci/ci_utils.py | 24 ++++++--- 4 files changed, 54 insertions(+), 57 deletions(-) diff --git a/.github/workflows/auto_releases.yml b/.github/workflows/auto_releases.yml index c159907187c..28483ea136f 100644 --- a/.github/workflows/auto_releases.yml +++ b/.github/workflows/auto_releases.yml @@ -19,13 +19,11 @@ on: jobs: AutoReleaseInfo: - runs-on: [self-hosted, style-checker-aarch64] + runs-on: [self-hosted, release-maker] outputs: data: ${{ steps.info.outputs.AUTO_RELEASE_PARAMS }} dry_run: ${{ steps.info.outputs.DRY_RUN }} steps: - - name: Debug Info - uses: ./.github/actions/debug - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' @@ -36,6 +34,10 @@ jobs: echo "DRY_RUN=true" >> "$GITHUB_ENV" - name: Check out repository code uses: ClickHouse/checkout@v1 + with: + fetch-depth: 0 # full history needed + - name: Debug Info + uses: ./.github/actions/debug - name: Prepare Info id: info run: | @@ -46,12 +48,7 @@ jobs: echo "::endgroup::" { echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_ENV" - { - echo 'AUTO_RELEASE_PARAMS<> "$GITHUB_OUTPUT" echo "DRY_RUN=true" >> "$GITHUB_OUTPUT" @@ -62,48 +59,29 @@ jobs: - name: Clean up uses: ./.github/actions/clean - Release_0: + Releases: needs: AutoReleaseInfo - name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].release_branch }} - if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].ready }} + strategy: + matrix: + release_params: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases }} + max-parallel: 1 + name: Release ${{ matrix.release_params.release_branch }} uses: ./.github/workflows/create_release.yml with: - ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].commit_sha }} + ref: ${{ matrix.release_params.commit_sha }} type: patch - dry-run: ${{ needs.AutoReleaseInfo.outputs.dry_run }} -# -# Release_1: -# needs: [AutoReleaseInfo, Release_0] -# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].release_branch }} -# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].ready }} -# uses: ./.github/workflows/create_release.yml -# with: -# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[1].commit_sha }} -# type: patch -# dry-run: ${{ env.DRY_RUN }} -# -# Release_2: -# needs: [AutoReleaseInfo, Release_1] -# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[2].release_branch }} -# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[2].ready }} -# uses: ./.github/workflow/create_release.yml -# with: -# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[0].commit_sha }} -# type: patch -# dry-run: ${{ env.DRY_RUN }} -# -# Release_3: -# needs: [AutoReleaseInfo, Release_2] -# name: Release ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].release_branch }} -# if: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3] && fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].ready }} -# uses: ./.github/workflow/create_release.yml -# with: -# ref: ${{ fromJson(needs.AutoReleaseInfo.outputs.data).releases[3].commit_sha }} -# type: patch -# dry-run: ${{ env.DRY_RUN }} + dry-run: ${{ fromJson(needs.AutoReleaseInfo.outputs.dry_run) }} + secrets: + ROBOT_CLICKHOUSE_COMMIT_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} -# - name: Post Slack Message -# if: ${{ !cancelled() }} -# run: | -# cd "$GITHUB_WORKSPACE/tests/ci" -# python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} + PostSlackMessage: + needs: [AutoReleaseInfo] + runs-on: [self-hosted, release-maker] + if: ${{ !cancelled() }} + steps: + - name: Check out repository code + uses: ClickHouse/checkout@v1 + - name: Post + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 1553d689227..1fb6cb60e96 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -47,6 +47,8 @@ concurrency: required: false default: false type: boolean + secrets: + ROBOT_CLICKHOUSE_COMMIT_TOKEN: jobs: CreateRelease: diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 3cc88634004..58cfc833afe 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,4 +1,5 @@ import argparse +import copy import dataclasses import json import os @@ -46,6 +47,7 @@ def parse_args(): MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE = 5 AUTORELEASE_INFO_FILE = "/tmp/autorelease_info.json" +AUTORELEASE_MATRIX_PARAMS = "/tmp/autorelease_params.json" @dataclasses.dataclass @@ -74,6 +76,12 @@ class AutoReleaseInfo: with open(AUTORELEASE_INFO_FILE, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + # dump file for GH action matrix that is similar to the file above but with dropped not ready release branches + params = copy.deepcopy(self) + params.releases = [release for release in params.releases if release.ready] + with open(AUTORELEASE_MATRIX_PARAMS, "w", encoding="utf-8") as f: + print(json.dumps(params, indent=2), file=f) + @staticmethod def from_file() -> "AutoReleaseInfo": with open(AUTORELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: @@ -136,6 +144,7 @@ def _prepare(token): commit_ci_status = CI.GH.get_commit_status_by_name( token=token, commit_sha=commit, + # handle old name for old releases status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), ) commit_sha = commit diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index b8778e0cc50..97ab10f1b58 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -102,21 +102,29 @@ class GH: assert len(commit_sha) == 40 assert Utils.is_hex(commit_sha) assert not Utils.is_hex(token) - url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses?per_page={200}" + + url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses" headers = { "Authorization": f"token {token}", "Accept": "application/vnd.github.v3+json", } - response = requests.get(url, headers=headers, timeout=5) if isinstance(status_name, str): status_name = (status_name,) - if response.status_code == 200: - assert "next" not in response.links, "Response truncated" - statuses = response.json() - for status in statuses: - if status["context"] in status_name: - return status["state"] # type: ignore + + while url: + response = requests.get(url, headers=headers, timeout=5) + if response.status_code == 200: + statuses = response.json() + for status in statuses: + if status["context"] in status_name: + return status["state"] + + # Check if there is a next page + url = response.links.get("next", {}).get("url") + else: + break + return "" @staticmethod From f17655f13fcadb9babbc859e45e9f38cb32ad9e3 Mon Sep 17 00:00:00 2001 From: Dani Pozo Date: Thu, 1 Aug 2024 11:01:27 +0200 Subject: [PATCH 1051/1722] Load filesystem cache metadata asynchronously --- src/Common/StatusFile.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 35 +++--- .../Cached/CachedObjectStorage.cpp | 5 +- src/Interpreters/Cache/FileCache.cpp | 71 +++++++++--- src/Interpreters/Cache/FileCache.h | 10 ++ src/Interpreters/Cache/FileCacheSettings.cpp | 3 + src/Interpreters/Cache/FileCacheSettings.h | 1 + .../InterpreterDescribeCacheQuery.cpp | 2 + src/Interpreters/TemporaryDataOnDisk.cpp | 4 +- src/Interpreters/tests/gtest_filecache.cpp | 11 ++ .../System/StorageSystemFilesystemCache.cpp | 3 + .../StorageSystemFilesystemCacheSettings.cpp | 2 + tests/config/config.d/storage_conf.xml | 2 + tests/config/config.d/storage_conf_02944.xml | 1 + .../integration/test_filesystem_cache/test.py | 106 +++++++++++++++--- .../02344_describe_cache.reference | 2 +- .../0_stateless/02344_describe_cache.sh | 2 +- ...8_filesystem_cache_as_collection.reference | 4 +- .../02908_filesystem_cache_as_collection.sql | 4 +- ...ge_cache_setting_without_restart.reference | 14 +-- ...lly_change_filesystem_cache_size.reference | 10 +- 21 files changed, 227 insertions(+), 67 deletions(-) diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 80464f38082..0bbb7ff411d 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -51,7 +51,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_) std::string contents; { ReadBufferFromFile in(path, 1024); - LimitReadBuffer limit_in(in, 1024, /* trow_exception */ false, /* exact_limit */ {}); + LimitReadBuffer limit_in(in, 1024, /* throw_exception */ false, /* exact_limit */ {}); readStringUntilEOF(contents, limit_in); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index bb9761a3905..c96f5f0c931 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -80,20 +80,27 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c if (with_file_cache) { - auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); - buf = std::make_unique( - object_path, - cache_key, - settings.remote_fs_cache, - FileCache::getCommonUser(), - [=, this]() { return read_buffer_creator(/* restricted_seek */true, object); }, - settings, - query_id, - object.bytes_size, - /* allow_seeks */false, - /* use_external_buffer */true, - /* read_until_position */std::nullopt, - cache_log); + if (settings.remote_fs_cache->isInitialized()) + { + auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); + buf = std::make_unique( + object_path, + cache_key, + settings.remote_fs_cache, + FileCache::getCommonUser(), + [=, this]() { return read_buffer_creator(/* restricted_seek */true, object); }, + settings, + query_id, + object.bytes_size, + /* allow_seeks */false, + /* use_external_buffer */true, + /* read_until_position */std::nullopt, + cache_log); + } + else + { + settings.remote_fs_cache->throwInitExceptionIfNeeded(); + } } /// Can't wrap CachedOnDiskReadBufferFromFile in CachedInMemoryReadBufferFromFile because the diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index fb817005399..ab0d357119c 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -99,7 +99,7 @@ std::unique_ptr CachedObjectStorage::writeObject( /// N /// Need to remove even if cache_on_write == false. removeCacheIfExists(object.remote_path); - if (cache_on_write) + if (cache_on_write && cache->isInitialized()) { auto key = getCacheKey(object.remote_path); return std::make_unique( @@ -122,7 +122,8 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c return; /// Add try catch? - cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id); + if (cache->isInitialized()) + cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id); } void CachedObjectStorage::removeObject(const StoredObject & object) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e3925163362..4c35c0f7f4c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -11,11 +11,15 @@ #include #include #include +#include +#include #include #include #include +#include #include +#include namespace fs = std::filesystem; @@ -88,6 +92,7 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s , bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0) , boundary_alignment(settings.boundary_alignment) , load_metadata_threads(settings.load_metadata_threads) + , load_metadata_asynchronously(settings.load_metadata_asynchronously) , write_cache_per_user_directory(settings.write_cache_per_user_id_directory) , keep_current_size_to_max_ratio(1 - settings.keep_free_space_size_ratio) , keep_current_elements_to_max_ratio(1 - settings.keep_free_space_elements_ratio) @@ -136,7 +141,17 @@ const FileCache::UserInfo & FileCache::getInternalUser() bool FileCache::isInitialized() const { - return is_initialized.load(std::memory_order_seq_cst); + return is_initialized; +} + +void FileCache::throwInitExceptionIfNeeded() +{ + if (load_metadata_asynchronously) + return; + + std::lock_guard lock(init_mutex); + if (init_exception) + std::rethrow_exception(init_exception); } const String & FileCache::getBasePath() const @@ -170,6 +185,35 @@ void FileCache::assertInitialized() const } void FileCache::initialize() +{ + // Prevent initialize() from running twice. This may be caused by two cache disks being created with the same path (see integration/test_filesystem_cache). + callOnce(initialize_called, [&] { + bool need_to_load_metadata = fs::exists(getBasePath()); + try + { + if (!need_to_load_metadata) + fs::create_directories(getBasePath()); + status_file = make_unique(fs::path(getBasePath()) / "status", StatusFile::write_full_info); + } + catch (...) + { + init_exception = std::current_exception(); + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; + } + + if (load_metadata_asynchronously) + { + load_metadata_main_thread = ThreadFromGlobalPool([this, need_to_load_metadata] { initializeImpl(need_to_load_metadata); }); + } + else + { + initializeImpl(need_to_load_metadata); + } + }); +} + +void FileCache::initializeImpl(bool load_metadata) { std::lock_guard lock(init_mutex); @@ -178,16 +222,10 @@ void FileCache::initialize() try { - if (fs::exists(getBasePath())) - { + if (load_metadata) loadMetadata(); - } - else - { - fs::create_directories(getBasePath()); - } - status_file = make_unique(fs::path(getBasePath()) / "status", StatusFile::write_full_info); + metadata.startup(); } catch (...) { @@ -196,8 +234,6 @@ void FileCache::initialize() throw; } - metadata.startup(); - if (keep_current_size_to_max_ratio != 1 || keep_current_elements_to_max_ratio != 1) { keep_up_free_space_ratio_task = Context::getGlobalContextInstance()->getSchedulePool().createTask(log->name(), [this] { freeSpaceRatioKeepingThreadFunc(); }); @@ -205,6 +241,7 @@ void FileCache::initialize() } is_initialized = true; + LOG_TEST(log, "Initialized cache from {}", metadata.getBaseDirectory()); } CachePriorityGuard::Lock FileCache::lockCache() const @@ -1185,7 +1222,6 @@ void FileCache::loadMetadataImpl() std::vector loading_threads; std::exception_ptr first_exception; std::mutex set_exception_mutex; - std::atomic stop_loading = false; LOG_INFO(log, "Loading filesystem cache with {} threads from {}", load_metadata_threads, metadata.getBaseDirectory()); @@ -1195,7 +1231,7 @@ void FileCache::loadMetadataImpl() { loading_threads.emplace_back([&] { - while (!stop_loading) + while (!stop_loading_metadata) { try { @@ -1212,7 +1248,7 @@ void FileCache::loadMetadataImpl() if (!first_exception) first_exception = std::current_exception(); } - stop_loading = true; + stop_loading_metadata = true; return; } } @@ -1225,7 +1261,7 @@ void FileCache::loadMetadataImpl() if (!first_exception) first_exception = std::current_exception(); } - stop_loading = true; + stop_loading_metadata = true; break; } } @@ -1412,6 +1448,11 @@ FileCache::~FileCache() void FileCache::deactivateBackgroundOperations() { shutdown.store(true); + + stop_loading_metadata = true; + if (load_metadata_main_thread.joinable()) + load_metadata_main_thread.join(); + metadata.shutdown(); if (keep_up_free_space_ratio_task) keep_up_free_space_ratio_task->deactivate(); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 07be802a940..579472eb824 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -8,6 +8,7 @@ #include +#include #include #include #include @@ -82,6 +83,9 @@ public: bool isInitialized() const; + /// Throws if `!load_metadata_asynchronously` and there is an exception in `init_exception` + void throwInitExceptionIfNeeded(); + const String & getBasePath() const; static Key createKeyForPath(const String & path); @@ -198,6 +202,9 @@ private: const size_t bypass_cache_threshold; const size_t boundary_alignment; size_t load_metadata_threads; + const bool load_metadata_asynchronously; + std::atomic stop_loading_metadata = false; + ThreadFromGlobalPool load_metadata_main_thread; const bool write_cache_per_user_directory; BackgroundSchedulePool::TaskHolder keep_up_free_space_ratio_task; @@ -209,6 +216,7 @@ private: std::exception_ptr init_exception; std::atomic is_initialized = false; + OnceFlag initialize_called; mutable std::mutex init_mutex; std::unique_ptr status_file; std::atomic shutdown = false; @@ -246,6 +254,8 @@ private: */ FileCacheQueryLimitPtr query_limit; + void initializeImpl(bool load_metadata); + void assertInitialized() const; void assertCacheCorrectness(); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index c68ff3183c6..e162d6b7551 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -65,6 +65,9 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin if (has("load_metadata_threads")) load_metadata_threads = get_uint("load_metadata_threads"); + if (has("load_metadata_asynchronously")) + load_metadata_asynchronously = get_uint("load_metadata_asynchronously"); + if (boundary_alignment > max_file_segment_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `boundary_alignment` cannot exceed `max_file_segment_size`"); diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 93ded202947..72a2b6c3369 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -32,6 +32,7 @@ struct FileCacheSettings size_t background_download_queue_size_limit = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_QUEUE_SIZE_LIMIT; size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS; + bool load_metadata_asynchronously = false; bool write_cache_per_user_id_directory = false; diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index c7e863bf260..c7464dc6b77 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -20,6 +20,7 @@ static Block getSampleBlock() ColumnWithTypeAndName{std::make_shared(), "max_size"}, ColumnWithTypeAndName{std::make_shared(), "max_elements"}, ColumnWithTypeAndName{std::make_shared(), "max_file_segment_size"}, + ColumnWithTypeAndName{std::make_shared(), "is_initialized"}, ColumnWithTypeAndName{std::make_shared(), "boundary_alignment"}, ColumnWithTypeAndName{std::make_shared>(), "cache_on_write_operations"}, ColumnWithTypeAndName{std::make_shared>(), "cache_hits_threshold"}, @@ -50,6 +51,7 @@ BlockIO InterpreterDescribeCacheQuery::execute() res_columns[i++]->insert(settings.max_size); res_columns[i++]->insert(settings.max_elements); res_columns[i++]->insert(settings.max_file_segment_size); + res_columns[i++]->insert(cache->isInitialized()); res_columns[i++]->insert(settings.boundary_alignment); res_columns[i++]->insert(settings.cache_on_write_operations); res_columns[i++]->insert(settings.cache_hits_threshold); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 7f0fb8cd6ca..3259d7b67d6 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -65,7 +65,7 @@ TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, Cu std::unique_ptr TemporaryDataOnDisk::createRawStream(size_t max_file_size) { - if (file_cache) + if (file_cache && file_cache->isInitialized()) { auto holder = createCacheFile(max_file_size); return std::make_unique(std::move(holder)); @@ -81,7 +81,7 @@ std::unique_ptr TemporaryDataOnDisk::createRawStream(si TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) { - if (file_cache) + if (file_cache && file_cache->isInitialized()) { auto holder = createCacheFile(max_file_size); diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 36acc319f4e..5e2d3ee8219 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -42,6 +43,7 @@ #include #include +using namespace std::chrono_literals; namespace fs = std::filesystem; using namespace DB; @@ -358,9 +360,11 @@ TEST_F(FileCacheTest, LRUPolicy) settings.max_size = 30; settings.max_elements = 5; settings.boundary_alignment = 1; + settings.load_metadata_asynchronously = false; const size_t file_size = INT_MAX; // the value doesn't really matter because boundary_alignment == 1. + const auto user = FileCache::getCommonUser(); { std::cerr << "Step 1\n"; @@ -815,6 +819,7 @@ TEST_F(FileCacheTest, writeBuffer) settings.max_elements = 5; settings.max_file_segment_size = 5; settings.base_path = cache_base_path; + settings.load_metadata_asynchronously = false; FileCache cache("6", settings); cache.initialize(); @@ -946,6 +951,7 @@ TEST_F(FileCacheTest, temporaryData) settings.max_size = 10_KiB; settings.max_file_segment_size = 1_KiB; settings.base_path = cache_base_path; + settings.load_metadata_asynchronously = false; DB::FileCache file_cache("7", settings); file_cache.initialize(); @@ -1073,6 +1079,7 @@ TEST_F(FileCacheTest, CachedReadBuffer) settings.max_size = 30; settings.max_elements = 10; settings.boundary_alignment = 1; + settings.load_metadata_asynchronously = false; ReadSettings read_settings; read_settings.enable_filesystem_cache = true; @@ -1092,6 +1099,7 @@ TEST_F(FileCacheTest, CachedReadBuffer) auto cache = std::make_shared("8", settings); cache->initialize(); + auto key = cache->createKeyForPath(file_path); const auto user = FileCache::getCommonUser(); @@ -1132,6 +1140,7 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize) settings.max_size = 10_KiB; settings.max_file_segment_size = 1_KiB; settings.base_path = cache_base_path; + settings.load_metadata_asynchronously = false; DB::FileCache file_cache("cache", settings); file_cache.initialize(); @@ -1195,6 +1204,7 @@ TEST_F(FileCacheTest, SLRUPolicy) settings.max_size = 40; settings.max_elements = 6; settings.boundary_alignment = 1; + settings.load_metadata_asynchronously = false; settings.cache_policy = "SLRU"; settings.slru_size_ratio = 0.5; @@ -1307,6 +1317,7 @@ TEST_F(FileCacheTest, SLRUPolicy) settings2.boundary_alignment = 1; settings2.cache_policy = "SLRU"; settings2.slru_size_ratio = 0.5; + settings.load_metadata_asynchronously = false; auto cache = std::make_shared("slru_2", settings2); cache->initialize(); diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index cfb388bc232..0e972d8411b 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -47,6 +47,9 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & [cache_name, cache_data] : caches) { const auto & cache = cache_data->cache; + if (!cache->isInitialized()) + continue; + cache->iterate([&](const FileSegment::Info & file_segment) { size_t i = 0; diff --git a/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp b/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp index 8915032baf7..c6bba6b8598 100644 --- a/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp +++ b/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp @@ -21,6 +21,7 @@ ColumnsDescription StorageSystemFilesystemCacheSettings::getColumnsDescription() {"path", std::make_shared(), "Cache directory"}, {"max_size", std::make_shared(), "Cache size limit by the number of bytes"}, {"max_elements", std::make_shared(), "Cache size limit by the number of elements"}, + {"is_initialized", std::make_shared(), "Whether the cache is initialized and ready to be used"}, {"current_size", std::make_shared(), "Current cache size by the number of bytes"}, {"current_elements", std::make_shared(), "Current cache size by the number of elements"}, {"max_file_segment_size", std::make_shared(), "Maximum allowed file segment size"}, @@ -56,6 +57,7 @@ void StorageSystemFilesystemCacheSettings::fillData( res_columns[i++]->insert(settings.base_path); res_columns[i++]->insert(settings.max_size); res_columns[i++]->insert(settings.max_elements); + res_columns[i++]->insert(cache->isInitialized()); res_columns[i++]->insert(cache->getUsedCacheSize()); res_columns[i++]->insert(cache->getFileSegmentsNum()); res_columns[i++]->insert(settings.max_file_segment_size); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index e106e3a0e6b..091071f0637 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -27,6 +27,7 @@ 0.3 0.15 0.15 + 0 cache @@ -37,6 +38,7 @@ 100 0 0 + 0 diff --git a/tests/config/config.d/storage_conf_02944.xml b/tests/config/config.d/storage_conf_02944.xml index 5f45640a923..08d78900229 100644 --- a/tests/config/config.d/storage_conf_02944.xml +++ b/tests/config/config.d/storage_conf_02944.xml @@ -19,6 +19,7 @@ 10 100 0 + 0 diff --git a/tests/integration/test_filesystem_cache/test.py b/tests/integration/test_filesystem_cache/test.py index 17a8dd8b6e1..aee8bd25c2e 100644 --- a/tests/integration/test_filesystem_cache/test.py +++ b/tests/integration/test_filesystem_cache/test.py @@ -1,6 +1,7 @@ import logging import time import os +import random import pytest from helpers.cluster import ClickHouseCluster @@ -30,14 +31,6 @@ def cluster(): "config.d/storage_conf_2.xml", ], ) - cluster.add_instance( - "node_no_filesystem_caches_path", - main_configs=[ - "config.d/storage_conf.xml", - "config.d/remove_filesystem_caches_path.xml", - ], - stay_alive=True, - ) cluster.add_instance( "node_force_read_through_cache_on_merge", main_configs=[ @@ -59,6 +52,51 @@ def cluster(): cluster.shutdown() +@pytest.fixture(scope="function") +def non_shared_cluster(): + """ + For tests that cannot run in parallel against the same node/cluster (see test_custom_cached_disk, which relies on + changing server settings at runtime) + """ + try: + # Randomize the cluster name + cluster = ClickHouseCluster(f"{__file__}_non_shared_{random.randint(0, 10**7)}") + cluster.add_instance( + "node_no_filesystem_caches_path", + main_configs=[ + "config.d/storage_conf.xml", + "config.d/remove_filesystem_caches_path.xml", + ], + stay_alive=True, + ) + + logging.info("Starting test-exclusive cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def wait_for_cache_initialized(node, cache_path, max_attempts=50): + initialized = False + attempts = 0 + while not initialized: + query_result = node.query( + "SELECT path FROM system.filesystem_cache_settings WHERE is_initialized" + ) + initialized = cache_path in query_result + + if initialized: + break + + time.sleep(0.1) + attempts += 1 + if attempts >= max_attempts: + raise "Stopped waiting for cache to be initialized" + + @pytest.mark.parametrize("node_name", ["node"]) def test_parallel_cache_loading_on_startup(cluster, node_name): node = cluster.instances[node_name] @@ -71,14 +109,21 @@ def test_parallel_cache_loading_on_startup(cluster, node_name): ORDER BY value SETTINGS disk = disk( type = cache, - path = 'paralel_loading_test', + name = 'parallel_loading_test', + path = 'parallel_loading_test', disk = 'hdd_blob', max_file_segment_size = '1Ki', boundary_alignment = '1Ki', max_size = '1Gi', max_elements = 10000000, load_metadata_threads = 30); + """ + ) + wait_for_cache_initialized(node, "parallel_loading_test") + + node.query( + """ SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT * FROM generateRandom('a Int32, b String') LIMIT 1000000; SELECT * FROM test FORMAT Null; @@ -103,6 +148,7 @@ def test_parallel_cache_loading_on_startup(cluster, node_name): ) node.restart_clickhouse() + wait_for_cache_initialized(node, "parallel_loading_test") # < because of additional files loaded into cache on server startup. assert cache_count <= int(node.query("SELECT count() FROM system.filesystem_cache")) @@ -131,7 +177,7 @@ def test_caches_with_the_same_configuration(cluster, node_name): node = cluster.instances[node_name] cache_path = "cache1" - node.query(f"SYSTEM DROP FILESYSTEM CACHE;") + node.query("SYSTEM DROP FILESYSTEM CACHE;") for table in ["test", "test2"]: node.query( f""" @@ -142,14 +188,20 @@ def test_caches_with_the_same_configuration(cluster, node_name): ORDER BY value SETTINGS disk = disk( type = cache, - name = {table}, + name = '{table}', path = '{cache_path}', disk = 'hdd_blob', max_file_segment_size = '1Ki', boundary_alignment = '1Ki', cache_on_write_operations=1, max_size = '1Mi'); + """ + ) + wait_for_cache_initialized(node, cache_path) + + node.query( + f""" SET enable_filesystem_cache_on_write_operations=1; INSERT INTO {table} SELECT * FROM generateRandom('a Int32, b String') LIMIT 1000; @@ -195,9 +247,8 @@ def test_caches_with_the_same_configuration(cluster, node_name): @pytest.mark.parametrize("node_name", ["node_caches_with_same_path"]) def test_caches_with_the_same_configuration_2(cluster, node_name): node = cluster.instances[node_name] - cache_path = "cache1" - node.query(f"SYSTEM DROP FILESYSTEM CACHE;") + node.query("SYSTEM DROP FILESYSTEM CACHE;") for table in ["cache1", "cache2"]: node.query( f""" @@ -207,7 +258,13 @@ def test_caches_with_the_same_configuration_2(cluster, node_name): Engine=MergeTree() ORDER BY value SETTINGS disk = '{table}'; + """ + ) + wait_for_cache_initialized(node, "cache1") + + node.query( + f""" SET enable_filesystem_cache_on_write_operations=1; INSERT INTO {table} SELECT * FROM generateRandom('a Int32, b String') LIMIT 1000; @@ -227,8 +284,8 @@ def test_caches_with_the_same_configuration_2(cluster, node_name): ) -def test_custom_cached_disk(cluster): - node = cluster.instances["node_no_filesystem_caches_path"] +def test_custom_cached_disk(non_shared_cluster): + node = non_shared_cluster.instances["node_no_filesystem_caches_path"] assert "Cannot create cached custom disk without" in node.query_and_get_error( f""" @@ -377,6 +434,7 @@ def test_force_filesystem_cache_on_merges(cluster): ORDER BY value SETTINGS disk = disk( type = cache, + name = 'force_cache_on_merges', path = 'force_cache_on_merges', disk = 'hdd_blob', max_file_segment_size = '1Ki', @@ -385,7 +443,13 @@ def test_force_filesystem_cache_on_merges(cluster): max_size = '10Gi', max_elements = 10000000, load_metadata_threads = 30); + """ + ) + wait_for_cache_initialized(node, "force_cache_on_merges") + + node.query( + """ SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT * FROM generateRandom('a Int32, b String') LIMIT 1000000; INSERT INTO test SELECT * FROM generateRandom('a Int32, b String') LIMIT 1000000; @@ -441,7 +505,13 @@ SETTINGS disk = disk(type = cache, path = "test_system_sync_filesystem_cache", delayed_cleanup_interval_ms = 10000000, disk = hdd_blob), min_bytes_for_wide_part = 10485760; + """ + ) + wait_for_cache_initialized(node, "test_system_sync_filesystem_cache") + + node.query( + """ INSERT INTO test SELECT 1, 'test'; """ ) @@ -525,7 +595,13 @@ SETTINGS disk = disk(type = cache, keep_free_space_elements_ratio = {elements_ratio}, disk = hdd_blob), min_bytes_for_wide_part = 10485760; + """ + ) + wait_for_cache_initialized(node, "test_keep_up_size_ratio") + + node.query( + """ INSERT INTO test SELECT randomString(200); """ ) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 6895606eb2b..13429b14866 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ 1 -102400 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/02344_describe_cache_test 0 5000 0 16 +102400 10000000 33554432 1 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/02344_describe_cache_test 0 5000 0 16 diff --git a/tests/queries/0_stateless/02344_describe_cache.sh b/tests/queries/0_stateless/02344_describe_cache.sh index d91661db9bc..c5373b4d7e3 100755 --- a/tests/queries/0_stateless/02344_describe_cache.sh +++ b/tests/queries/0_stateless/02344_describe_cache.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = '$disk_name', disk = 's3_disk'); +SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = '$disk_name', disk = 's3_disk', load_metadata_asynchronously = 0); """ $CLICKHOUSE_CLIENT -nm --query """ diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference index d4191af1594..41a60204eab 100644 --- a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.reference @@ -1,2 +1,2 @@ -1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection_sql 0 5000 0 16 -1048576 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection 0 5000 0 16 +1048576 10000000 33554432 1 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection_sql 0 5000 0 16 +1048576 10000000 33554432 1 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/collection 0 5000 0 16 diff --git a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql index c7216833bc9..127baa8304e 100644 --- a/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql +++ b/tests/queries/0_stateless/02908_filesystem_cache_as_collection.sql @@ -3,8 +3,8 @@ CREATE NAMED COLLECTION IF NOT EXISTS cache_collection_sql AS path = 'collection_sql', max_size = '1Mi'; DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) -ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 'local_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME', cache_name='cache_collection_sql'); +ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 'local_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME', cache_name='cache_collection_sql', load_metadata_asynchronously = 0); DESCRIBE FILESYSTEM CACHE '$CLICHOUSE_TEST_UNIQUE_NAME'; CREATE TABLE test2 (a Int32, b String) -ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 'local_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME_2', cache_name='cache_collection'); +ENGINE = MergeTree() ORDER BY a SETTINGS disk = disk(type = cache, disk = 'local_disk', name = '$CLICHOUSE_TEST_UNIQUE_NAME_2', cache_name='cache_collection', load_metadata_asynchronously = 0); DESCRIBE FILESYSTEM CACHE '$CLICHOUSE_TEST_UNIQUE_NAME_2'; diff --git a/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference b/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference index 17a25d82824..0f64d0393b2 100644 --- a/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference +++ b/tests/queries/0_stateless/02933_change_cache_setting_without_restart.reference @@ -1,7 +1,7 @@ -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 16 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 10 1000 0 16 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 5 1000 0 16 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 15 1000 0 16 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 2 1000 0 16 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 1000 0 16 -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 10 1000 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 5 1000 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 15 1000 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 2 1000 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 1000 0 16 +134217728 10000000 33554432 1 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02933/ 0 0 0 16 diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference index 298cc908178..c6bbcdc20c2 100644 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.reference @@ -1,20 +1,20 @@ -100 10 10 10 0 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 +100 10 10 1 10 0 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 0 10 98 set max_size from 100 to 10 -10 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 +10 10 10 1 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 1 8 set max_size from 10 to 100 -100 10 10 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 +100 10 10 1 10 0 0 8 1 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 10 98 set max_elements from 10 to 2 -100 2 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 +100 2 10 1 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 2 18 set max_elements from 2 to 10 -100 10 10 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 +100 10 10 1 10 0 0 18 2 /var/lib/clickhouse/filesystem_caches/s3_cache_02944/ 0 5000 0 16 10 98 From c542fb57cdbd1d2a17544476f924fc3e13f56ebf Mon Sep 17 00:00:00 2001 From: neoman36 Date: Fri, 16 Aug 2024 13:01:23 +0200 Subject: [PATCH 1052/1722] named-collections.md is updated 2 --- docs/en/operations/named-collections.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 16c75814e5a..340a5a8f87a 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -312,7 +312,7 @@ The description of parameters see [postgresql](../sql-reference/table-functions/ - `username` for `user` - `db` for `database`. -Parameter `addresses_expr` is used in a collection instead of `host:port`. The parameter is optional, because there are other optional ones: `host`, `hostname`, `port`. The following pseudocode explains the priority: +Parameter `addresses_expr` is used in a collection instead of `host:port`. The parameter is optional, because there are other optional ones: `host`, `hostname`, `port`. The following pseudo code explains the priority: ```sql CASE From 46f27b03f9dd2f122da6cfcb9905f5bb7ec944e7 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 15:48:28 +0400 Subject: [PATCH 1053/1722] fix transaction --- src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 38309282768..4915bf9f366 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8108,6 +8108,13 @@ std::unique_ptr StorageReplicatedMergeTree::rep DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); + std::optional txn; + if (auto query_txn = query_context->getZooKeeperMetadataTransaction()) + txn.emplace(query_txn->getZooKeeper(), + query_txn->getDatabaseZooKeeperPath(), + query_txn->isInitialQuery(), + query_txn->getTaskZooKeeperPath()); + /// Retry if alter_partition_version changes for (size_t retry = 0; retry < 1000; ++retry) { @@ -8275,7 +8282,7 @@ std::unique_ptr StorageReplicatedMergeTree::rep ephemeral_locks[i].getUnlockOp(ops); } - if (auto txn = query_context->getZooKeeperMetadataTransaction()) + if (txn) txn->moveOpsTo(ops); delimiting_block_lock->getUnlockOp(ops); From dfd17cc2d71555de9c42ad6085c35bb3f1372dd1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:23:57 +0000 Subject: [PATCH 1054/1722] Check for invalid regexp in JSON SKIP REGEXP section --- src/DataTypes/DataTypeObject.cpp | 11 +++++++++++ .../0_stateless/03227_json_invalid_regexp.reference | 0 .../queries/0_stateless/03227_json_invalid_regexp.sql | 4 ++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03227_json_invalid_regexp.reference create mode 100644 tests/queries/0_stateless/03227_json_invalid_regexp.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index d6395155397..11fffd8769b 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -49,6 +49,17 @@ DataTypeObject::DataTypeObject( , max_dynamic_paths(max_dynamic_paths_) , max_dynamic_types(max_dynamic_types_) { + /// Check if regular expressions are valid. + for (const auto & regexp_str : path_regexps_to_skip) + { + re2::RE2::Options options; + /// Don't log errors to stderr. + options.set_log_errors(false); + auto regexp = re2::RE2(regexp_str, options); + if (!regexp.error().empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid regexp '{}': {}", regexp_str, regexp.error()); + } + for (const auto & [typed_path, type] : typed_paths) { for (const auto & path_to_skip : paths_to_skip) diff --git a/tests/queries/0_stateless/03227_json_invalid_regexp.reference b/tests/queries/0_stateless/03227_json_invalid_regexp.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03227_json_invalid_regexp.sql b/tests/queries/0_stateless/03227_json_invalid_regexp.sql new file mode 100644 index 00000000000..734dea1aac6 --- /dev/null +++ b/tests/queries/0_stateless/03227_json_invalid_regexp.sql @@ -0,0 +1,4 @@ +set allow_experimental_json_type = 1; +create table test (json JSON(SKIP REGEXP '[]')) engine=Memory(); -- {serverError BAD_ARGUMENTS} +create table test (json JSON(SKIP REGEXP '+')) engine=Memory(); -- {serverError BAD_ARGUMENTS}; + From b1c18e2dd42051fb045431278eceae9dd22563bd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:29:25 +0000 Subject: [PATCH 1055/1722] Fix check for allowed date symbols in date time best effort schema infernece --- src/IO/parseDateTimeBestEffort.cpp | 4 ++-- tests/queries/0_stateless/03222_date_time_inference.reference | 1 + tests/queries/0_stateless/03222_date_time_inference.sql | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 8d798bf725c..03429b46b2a 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -350,7 +350,7 @@ ReturnType parseDateTimeBestEffortImpl( if (month && !day_of_month) day_of_month = hour_or_day_of_month_or_month; } - else if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) + else if ((!in.eof() && isSymbolIn(*in.position(), allowed_date_delimiters)) && (checkChar('/', in) || checkChar('.', in) || checkChar('-', in))) { if (day_of_month) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: day of month is duplicated"); @@ -399,7 +399,7 @@ ReturnType parseDateTimeBestEffortImpl( if (month > 12) std::swap(month, day_of_month); - if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) + if ((!in.eof() && isSymbolIn(*in.position(), allowed_date_delimiters)) && (checkChar('/', in) || checkChar('.', in) || checkChar('-', in))) { if (year) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: year component is duplicated"); diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference index 3288308a1d0..838b103f106 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.reference +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -251,3 +251,4 @@ Mar 2000 00:00:00.000 String 2000 00:00:00.000 String Mar 2000-01-01 00:00:00 String Mar 2000-01-01 00:00:00.000 String +1.7.10 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index ebd472294be..b0c4df530cf 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -265,5 +265,5 @@ select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00.000"}'); - +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "1.7.10"}'); From 370b6bdc7b6d97f0e697e99ccd06a25e97651406 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:38:30 +0000 Subject: [PATCH 1056/1722] Update tests --- tests/queries/0_stateless/02497_schema_inference_nulls.sql | 4 ++-- .../0_stateless/02784_schema_inference_null_as_default.sql | 4 ++-- ..._max_bytes_to_read_for_schema_inference_in_cache.reference | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02497_schema_inference_nulls.sql b/tests/queries/0_stateless/02497_schema_inference_nulls.sql index b78b5709dbb..5670b031e8b 100644 --- a/tests/queries/0_stateless/02497_schema_inference_nulls.sql +++ b/tests/queries/0_stateless/02497_schema_inference_nulls.sql @@ -40,7 +40,7 @@ desc format(JSONCompactEachRow, '[[[], [null], [1, 2, 3]]]'); desc format(JSONCompactEachRow, '[[{"a" : null}, {"b" : 1}]]'); desc format(JSONCompactEachRow, '[[["2020-01-01", null, "1234"], ["abcd"]]]'); -set schema_inference_make_columns_nullable=0; +set schema_inference_make_columns_nullable='auto'; desc format(JSONCompactEachRow, '[[1, 2]]'); desc format(JSONCompactEachRow, '[[null, 1]]'); desc format(JSONCompactEachRow, '[[1, 2]], [[3]]'); @@ -59,7 +59,7 @@ desc format(CSV, '"[[], [null], [1, 2, 3]]"'); desc format(CSV, '"[{\'a\' : null}, {\'b\' : 1}]"'); desc format(CSV, '"[[\'2020-01-01\', null, \'1234\'], [\'abcd\']]"'); -set schema_inference_make_columns_nullable=0; +set schema_inference_make_columns_nullable='auto'; desc format(CSV, '"[1,2]"'); desc format(CSV, '"[NULL, 1]"'); desc format(CSV, '"[1, 2]"\n"[3]"'); diff --git a/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql b/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql index 9c9f99d8283..571e3ab4f25 100644 --- a/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql +++ b/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql @@ -1,7 +1,7 @@ desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=1; select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=1; -desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=0; -select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=0; +desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable='auto', input_format_null_as_default=0; +select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable='auto', input_format_null_as_default=0; desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=1; select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=1; diff --git a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference index cd109daac52..3b9d88edc19 100644 --- a/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference +++ b/tests/queries/0_stateless/03212_max_bytes_to_read_for_schema_inference_in_cache.reference @@ -1,2 +1,2 @@ x Nullable(Int64) -schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=true, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false +schema_inference_hints=, max_rows_to_read_for_schema_inference=25000, max_bytes_to_read_for_schema_inference=1000, schema_inference_make_columns_nullable=1, try_infer_integers=true, try_infer_dates=true, try_infer_datetimes=true, try_infer_numbers_from_strings=false, read_bools_as_numbers=true, read_bools_as_strings=true, read_objects_as_strings=true, read_numbers_as_strings=true, read_arrays_as_strings=true, try_infer_objects_as_tuples=true, infer_incomplete_types_as_strings=true, try_infer_objects=false, use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects=false From 4975264c9d39d560c32799dcf14bd3f9d40fcfa4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 14:08:07 +0000 Subject: [PATCH 1057/1722] More strict parsing --- src/IO/parseDateTimeBestEffort.cpp | 23 ++++--- .../03222_date_time_inference.reference | 67 ++++++++++++------- .../0_stateless/03222_date_time_inference.sql | 19 ++++++ 3 files changed, 75 insertions(+), 34 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 03429b46b2a..52bcdc6bbb4 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -180,8 +180,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 10 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow timestamps"); /// This is unix timestamp. readDecimalNumber<10>(res, digits); @@ -189,8 +189,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 9 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow timestamps"); /// This is unix timestamp. readDecimalNumber<9>(res, digits); @@ -198,8 +198,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 14 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); /// This is YYYYMMDDhhmmss readDecimalNumber<4>(year, digits); @@ -212,8 +212,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 8 && !year) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); /// This is YYYYMMDD readDecimalNumber<4>(year, digits); @@ -222,6 +222,9 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 6) { + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); + /// This is YYYYMM or hhmmss if (!year && !month) { @@ -593,8 +596,8 @@ ReturnType parseDateTimeBestEffortImpl( else return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected word"); -// while (!in.eof() && isAlphaASCII(*in.position())) -// ++in.position(); + while (!in.eof() && isAlphaASCII(*in.position())) + ++in.position(); /// For RFC 2822 if (has_day_of_week) diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference index 838b103f106..221ab1fe5f5 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.reference +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -45,34 +45,34 @@ DateTime/DateTime64 best effort 2000-01-01 00:00:00 DateTime 2000-01-01 01:00:00 DateTime 2000-01-01 01:00:00.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) -2017-01-01 21:02:03 DateTime -2017-01-01 21:02:03.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) -2017-01-02 01:02:03 DateTime -2017-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) +02/01/17 010203 MSK String +02/01/17 010203.000 MSK String +02/01/17 010203 MSK+0100 String +02/01/17 010203.000 MSK+0100 String +02/01/17 010203 UTC+0300 String +02/01/17 010203.000 UTC+0300 String +02/01/17 010203Z String +02/01/17 010203.000Z String +02/01/1970 010203Z String +02/01/1970 010203.000Z String +02/01/70 010203Z String +02/01/70 010203.000Z String 2018-02-11 03:40:50 DateTime 2018-02-11 03:40:50.000000000 DateTime64(9) 2000-04-17 01:02:03 DateTime 2000-04-17 01:02:03.000000000 DateTime64(9) -1970-01-02 01:00:00 DateTime -1970-01-02 01:00:00.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) +19700102 01:00:00 String +19700102 01:00:00.000 String +19700102010203Z String +19700102010203Z.000 String +1970/01/02 010203Z String +1970/01/02 010203.000Z String 2015-12-31 20:00:00 DateTime 2015-12-31 20:00:00 DateTime 2016-01-01 00:00:00 DateTime 2016-01-01 00:00:00 DateTime -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) +201701 02 010203 UTC+0300 String +201701 02 010203.000 UTC+0300 String 2017-01-02 03:04:05 DateTime 2017-01-02 03:04:05.000000000 DateTime64(9) 2017-01-02 03:04:05 DateTime @@ -117,8 +117,8 @@ DateTime/DateTime64 best effort 2017-01-02 03:04:05.000000000 DateTime64(9) 2017-04-01 11:22:33 DateTime 2017-04-01 11:22:33.000000000 DateTime64(9) -2017-04-01 22:02:03 DateTime -2017-04-01 22:02:03.000000000 DateTime64(9) +2017 Apr 02 010203 UTC+0300 String +2017 Apr 02 010203.000 UTC+0300 String 2017-04-01 22:02:03 DateTime 2017-04-01 22:02:03.000000000 DateTime64(9) 2017-04-02 01:02:03 DateTime @@ -143,8 +143,8 @@ DateTime/DateTime64 best effort 2017-04-01 21:02:03.000000000 DateTime64(9) 2017-04-02 01:02:03 DateTime 2017-04-02 01:02:03.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) +2017 Jan 02 010203 UTC+0300 String +2017 Jan 02 010203.000 UTC+0300 String 2017-04-25 01:02:03 DateTime 2017-04-25 01:02:03.000000000 DateTime64(9) 2017-04-25 01:02:03 DateTime @@ -231,6 +231,25 @@ Mar01012020010101 String Mar 01012020010101 String Mar01012020010101.000 String Mar 0101202001010101.000 String +Sun String +Sun1 String +Sun 1 String +Sun01 String +Sun 01 String +Sun2020 String +Sun 2020 String +Sun012020 String +Sun 012020 String +Sun01012020 String +Sun 01012020 String +Sun0101202001 String +Sun 0101202001 String +Sun010120200101 String +Sun 010120200101 String +Sun01012020010101 String +Sun 01012020010101 String +Sun01012020010101.000 String +Sun 0101202001010101.000 String 2000 01 01 01:00:00 String 2000 01 01 01:00:00.000 String 2000a01a01 01:00:00 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index b0c4df530cf..b16f72c72f4 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -245,6 +245,25 @@ select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 01012020010101"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 0101202001010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 0101202001010101.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000a01a01 01:00:00"}'); From 6bd65dbfa5c5d450e355dc64c110db65d2f56cbb Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 16 Aug 2024 15:07:53 +0000 Subject: [PATCH 1058/1722] Use HTTP/1.1 for external HTTP authentication --- src/Access/HTTPAuthClient.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Access/HTTPAuthClient.h b/src/Access/HTTPAuthClient.h index a8b56cf05a7..a1b97a729a3 100644 --- a/src/Access/HTTPAuthClient.h +++ b/src/Access/HTTPAuthClient.h @@ -82,7 +82,8 @@ public: Result authenticate(const String & user_name, const String & password) const { - Poco::Net::HTTPRequest request{Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery()}; + Poco::Net::HTTPRequest request{ + Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1}; Poco::Net::HTTPBasicCredentials basic_credentials{user_name, password}; basic_credentials.authenticate(request); From 0dba461d7f9228fa81facdeabeffc983e181d9e0 Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Fri, 16 Aug 2024 16:46:57 +0100 Subject: [PATCH 1059/1722] Update gui.md: Links, formatting, spelling --- docs/en/interfaces/third-party/gui.md | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index 5b7615485ca..8d9dce983bc 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -10,7 +10,7 @@ sidebar_label: Visual Interfaces ### ch-ui {#ch-ui} -[ch-ui](https://github.com/caioricciuti/ch-ui) is a simple React.js app interface for ClickHouse databases, designed for executing queries and visualizing data. Built with React and the ClickHouse client for web, it offers a sleek and user-friendly UI for easy database interactions. +[ch-ui](https://github.com/caioricciuti/ch-ui) is a simple React.js app interface for ClickHouse databases designed for executing queries and visualizing data. Built with React and the ClickHouse client for web, it offers a sleek and user-friendly UI for easy database interactions. Features: @@ -25,7 +25,7 @@ Web interface for ClickHouse in the [Tabix](https://github.com/tabixio/tabix) pr Features: -- Works with ClickHouse directly from the browser, without the need to install additional software. +- Works with ClickHouse directly from the browser without the need to install additional software. - Query editor with syntax highlighting. - Auto-completion of commands. - Tools for graphical analysis of query execution. @@ -63,7 +63,7 @@ Features: - Table list with filtering and metadata. - Table preview with filtering and sorting. -- Read-only queries execution. +- Read-only query execution. ### Redash {#redash} @@ -75,23 +75,23 @@ Features: - Powerful editor of queries. - Database explorer. -- Visualization tools, that allow you to represent data in different forms. +- Visualization tool that allows you to represent data in different forms. ### Grafana {#grafana} [Grafana](https://grafana.com/grafana/plugins/grafana-clickhouse-datasource/) is a platform for monitoring and visualization. -"Grafana allows you to query, visualize, alert on and understand your metrics no matter where they are stored. Create, explore, and share dashboards with your team and foster a data driven culture. Trusted and loved by the community" — grafana.com. +"Grafana allows you to query, visualize, alert on and understand your metrics no matter where they are stored. Create, explore, and share dashboards with your team and foster a data-driven culture. Trusted and loved by the community" — grafana.com. -ClickHouse datasource plugin provides a support for ClickHouse as a backend database. +ClickHouse data source plugin provides support for ClickHouse as a backend database. -### qryn (#qryn) +### qryn {#qryn} [qryn](https://metrico.in) is a polyglot, high-performance observability stack for ClickHouse _(formerly cLoki)_ with native Grafana integrations allowing users to ingest and analyze logs, metrics and telemetry traces from any agent supporting Loki/LogQL, Prometheus/PromQL, OTLP/Tempo, Elastic, InfluxDB and many more. Features: -- Built in Explore UI and LogQL CLI for querying, extracting and visualizing data +- Built-in Explore UI and LogQL CLI for querying, extracting and visualizing data - Native Grafana APIs support for querying, processing, ingesting, tracing and alerting without plugins - Powerful pipeline to dynamically search, filter and extract data from logs, events, traces and beyond - Ingestion and PUSH APIs transparently compatible with LogQL, PromQL, InfluxDB, Elastic and many more @@ -139,7 +139,7 @@ Features: ### DBM {#dbm} -[DBM](https://dbm.incubator.edurt.io/) DBM is a visual management tool for ClickHouse! +[DBM](https://github.com/devlive-community/dbm) DBM is a visual management tool for ClickHouse! Features: @@ -151,7 +151,7 @@ Features: - Support custom query - Support multiple data sources management(connection test, monitoring) - Support monitor (processor, connection, query) -- Support migrate data +- Support migrating data ### Bytebase {#bytebase} @@ -169,7 +169,7 @@ Features: ### Zeppelin-Interpreter-for-ClickHouse {#zeppelin-interpreter-for-clickhouse} -[Zeppelin-Interpreter-for-ClickHouse](https://github.com/SiderZhang/Zeppelin-Interpreter-for-ClickHouse) is a [Zeppelin](https://zeppelin.apache.org) interpreter for ClickHouse. Compared with JDBC interpreter, it can provide better timeout control for long running queries. +[Zeppelin-Interpreter-for-ClickHouse](https://github.com/SiderZhang/Zeppelin-Interpreter-for-ClickHouse) is a [Zeppelin](https://zeppelin.apache.org) interpreter for ClickHouse. Compared with the JDBC interpreter, it can provide better timeout control for long-running queries. ### ClickCat {#clickcat} @@ -179,7 +179,7 @@ Features: - An online SQL editor which can run your SQL code without any installing. - You can observe all processes and mutations. For those unfinished processes, you can kill them in ui. -- The Metrics contains Cluster Analysis,Data Analysis,Query Analysis. +- The Metrics contain Cluster Analysis, Data Analysis, and Query Analysis. ### ClickVisual {#clickvisual} @@ -332,7 +332,7 @@ Learn more about the product at [TABLUM.IO](https://tablum.io/) ### CKMAN {#ckman} -[CKMAN] (https://www.github.com/housepower/ckman) is a tool for managing and monitoring ClickHouse clusters! +[CKMAN](https://www.github.com/housepower/ckman) is a tool for managing and monitoring ClickHouse clusters! Features: From 45e06de3267486296cc1452c981a78688a2193ae Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 16 Aug 2024 18:01:43 +0200 Subject: [PATCH 1060/1722] Minor update in Dynamic/JSON serializations --- src/DataTypes/Serializations/SerializationObject.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 2dd25e540cc..0042aa6d89d 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -199,7 +199,7 @@ void SerializationObject::serializeBinaryBulkStatePrefix( auto object_state = std::make_shared(serialization_version); object_state->max_dynamic_paths = column_object.getMaxDynamicPaths(); /// Write max_dynamic_paths parameter. - writeBinaryLittleEndian(object_state->max_dynamic_paths, *stream); + writeVarUInt(object_state->max_dynamic_paths, *stream); /// Write all dynamic paths in sorted order. object_state->sorted_dynamic_paths.reserve(dynamic_paths.size()); for (const auto & [path, _] : dynamic_paths) @@ -354,7 +354,7 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationObject::deserializeOb readBinaryLittleEndian(serialization_version, *structure_stream); auto structure_state = std::make_shared(serialization_version); /// Read max_dynamic_paths parameter. - readBinaryLittleEndian(structure_state->max_dynamic_paths, *structure_stream); + readVarUInt(structure_state->max_dynamic_paths, *structure_stream); /// Read the sorted list of dynamic paths. size_t dynamic_paths_size; readVarUInt(dynamic_paths_size, *structure_stream); From c85d5e753899503f93a8f9ca7b67776d386d9130 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 16 Aug 2024 18:02:51 +0200 Subject: [PATCH 1061/1722] Update Dynamic serialization --- src/DataTypes/Serializations/SerializationDynamic.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 6bba87c40fa..ab24779ced2 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -115,7 +115,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( dynamic_state->max_dynamic_types = column_dynamic.getMaxDynamicTypes(); /// Write max_dynamic_types parameter, because it can differ from the max_dynamic_types /// that is specified in the Dynamic type (we could decrease it before merge). - writeBinaryLittleEndian(dynamic_state->max_dynamic_types, *stream); + writeVarUInt(dynamic_state->max_dynamic_types, *stream); dynamic_state->variant_type = variant_info.variant_type; dynamic_state->variant_names = variant_info.variant_names; @@ -123,7 +123,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( /// Write information about variants. size_t num_variants = dynamic_state->variant_names.size() - 1; /// Don't write shared variant, Dynamic column should always have it. - writeBinaryLittleEndian(num_variants, *stream); + writeVarUInt(num_variants, *stream); if (settings.data_types_binary_encoding) { const auto & variants = assert_cast(*dynamic_state->variant_type).getVariants(); @@ -252,11 +252,11 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD readBinaryLittleEndian(structure_version, *structure_stream); auto structure_state = std::make_shared(structure_version); /// Read max_dynamic_types parameter. - readBinaryLittleEndian(structure_state->max_dynamic_types, *structure_stream); + readVarUInt(structure_state->max_dynamic_types, *structure_stream); /// Read information about variants. DataTypes variants; size_t num_variants; - readBinaryLittleEndian(num_variants, *structure_stream); + readVarUInt(num_variants, *structure_stream); variants.reserve(num_variants + 1); /// +1 for shared variant. if (settings.data_types_binary_encoding) { From 5fe46af4221a36ecb4566ca7bfad314d732f1de2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 16 Aug 2024 18:12:51 +0200 Subject: [PATCH 1062/1722] Update 02497_schema_inference_nulls.sql --- tests/queries/0_stateless/02497_schema_inference_nulls.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_schema_inference_nulls.sql b/tests/queries/0_stateless/02497_schema_inference_nulls.sql index 5670b031e8b..d62fc76d9b9 100644 --- a/tests/queries/0_stateless/02497_schema_inference_nulls.sql +++ b/tests/queries/0_stateless/02497_schema_inference_nulls.sql @@ -18,7 +18,7 @@ desc format(JSONEachRow, '{"x" : [[], [null], [1, 2, 3]]}'); desc format(JSONEachRow, '{"x" : [{"a" : null}, {"b" : 1}]}'); desc format(JSONEachRow, '{"x" : [["2020-01-01", null, "1234"], ["abcd"]]}'); -set schema_inference_make_columns_nullable=0; +set schema_inference_make_columns_nullable='auto'; desc format(JSONEachRow, '{"x" : [1, 2]}'); desc format(JSONEachRow, '{"x" : [null, 1]}'); desc format(JSONEachRow, '{"x" : [1, 2]}, {"x" : [3]}'); From 4f84c82d6d53ded0adda46aac1db1d345b5ba2eb Mon Sep 17 00:00:00 2001 From: Linh Giang <165205637+linhgiang24@users.noreply.github.com> Date: Fri, 16 Aug 2024 11:02:44 -0600 Subject: [PATCH 1063/1722] Update grant.md to include POSTGRES privilege Added POSTGRES privilege under the SOURCES category as it seems to be missing. --- docs/en/sql-reference/statements/grant.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 43fa344a16d..6118f4c1d36 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -200,6 +200,7 @@ Hierarchy of privileges: - `JDBC` - `HDFS` - `S3` + - `POSTGRES` - [dictGet](#dictget) - [displaySecretsInShowAndSelect](#displaysecretsinshowandselect) - [NAMED COLLECTION ADMIN](#named-collection-admin) @@ -476,6 +477,7 @@ Allows using external data sources. Applies to [table engines](../../engines/tab - `JDBC`. Level: `GLOBAL` - `HDFS`. Level: `GLOBAL` - `S3`. Level: `GLOBAL` + - `POSTGRES`. Level: `GLOBAL` The `SOURCES` privilege enables use of all the sources. Also you can grant a privilege for each source individually. To use sources, you need additional privileges. From cb9039e91d073942ff07a68e10336f8db38f4a79 Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Tue, 6 Aug 2024 19:05:12 +0800 Subject: [PATCH 1064/1722] Add thread pool and cancellation if merge data with key This patch will add thread pool and cancellation if merge data with key. During the merge, if the data size is large, we may convert the singleLevelHash to twoLevelHash and merge in parallel. Test the patch with 2 x 80 vCPUs, Q8 and Q9 has got 10.3% and 7.6% performance improvement. Signed-off-by: Jiebin Sun --- .../AggregateFunctionUniq.h | 2 ++ src/AggregateFunctions/IAggregateFunction.h | 12 +++++++--- src/AggregateFunctions/UniqExactSet.h | 7 ++++++ src/Interpreters/Aggregator.cpp | 24 ++++++++++++------- src/Interpreters/Aggregator.h | 4 ++-- .../Transforms/AggregatingTransform.cpp | 2 +- 6 files changed, 36 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index cef23f766c7..35d6e599e38 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -459,6 +459,8 @@ public: bool isParallelizeMergePrepareNeeded() const override { return is_parallelize_merge_prepare_needed; } + constexpr static bool parallelizeMergeWithKey() { return true; } + void parallelizeMergePrepare(AggregateDataPtrs & places, ThreadPool & thread_pool, std::atomic & is_cancelled) const override { if constexpr (is_parallelize_merge_prepare_needed) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index ee227db6d9d..f8e7051d635 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -145,6 +145,8 @@ public: virtual bool isParallelizeMergePrepareNeeded() const { return false; } + constexpr static bool parallelizeMergeWithKey() { return false; } + virtual void parallelizeMergePrepare(AggregateDataPtrs & /*places*/, ThreadPool & /*thread_pool*/, std::atomic & /*is_cancelled*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() with thread pool parameter isn't implemented for {} ", getName()); @@ -169,7 +171,7 @@ public: /// Merges states (on which src places points to) with other states (on which dst places points to) of current aggregation function /// then destroy states (on which src places points to). - virtual void mergeAndDestroyBatch(AggregateDataPtr * dst_places, AggregateDataPtr * src_places, size_t size, size_t offset, Arena * arena) const = 0; + virtual void mergeAndDestroyBatch(AggregateDataPtr * dst_places, AggregateDataPtr * src_places, size_t size, size_t offset, ThreadPool & thread_pool, std::atomic & is_cancelled, Arena * arena) const = 0; /// Serializes state (to transmit it over the network, for example). virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version = std::nullopt) const = 0; /// NOLINT @@ -499,11 +501,15 @@ public: static_cast(this)->merge(places[i] + place_offset, rhs[i], arena); } - void mergeAndDestroyBatch(AggregateDataPtr * dst_places, AggregateDataPtr * rhs_places, size_t size, size_t offset, Arena * arena) const override + void mergeAndDestroyBatch(AggregateDataPtr * dst_places, AggregateDataPtr * rhs_places, size_t size, size_t offset, ThreadPool & thread_pool, std::atomic & is_cancelled, Arena * arena) const override { for (size_t i = 0; i < size; ++i) { - static_cast(this)->merge(dst_places[i] + offset, rhs_places[i] + offset, arena); + if constexpr (Derived::parallelizeMergeWithKey()) + static_cast(this)->merge(dst_places[i] + offset, rhs_places[i] + offset, thread_pool, is_cancelled, arena); + else + static_cast(this)->merge(dst_places[i] + offset, rhs_places[i] + offset, arena); + static_cast(this)->destroy(rhs_places[i] + offset); } } diff --git a/src/AggregateFunctions/UniqExactSet.h b/src/AggregateFunctions/UniqExactSet.h index 2ae8c3a8386..25c6f7ac55f 100644 --- a/src/AggregateFunctions/UniqExactSet.h +++ b/src/AggregateFunctions/UniqExactSet.h @@ -101,6 +101,13 @@ public: auto merge(const UniqExactSet & other, ThreadPool * thread_pool = nullptr, std::atomic * is_cancelled = nullptr) { + /// If the size is large, we may convert the singleLevelHash to twoLevelHash and merge in parallel. + if (other.size() > 40000) + { + if (isSingleLevel()) + convertToTwoLevel(); + } + if (isSingleLevel() && other.isTwoLevel()) convertToTwoLevel(); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e073b7a49b6..71be3c1c30c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2371,7 +2371,7 @@ void NO_INLINE Aggregator::mergeDataNullKey( template void NO_INLINE Aggregator::mergeDataImpl( - Table & table_dst, Table & table_src, Arena * arena, bool use_compiled_functions [[maybe_unused]], bool prefetch) const + Table & table_dst, Table & table_src, Arena * arena, bool use_compiled_functions [[maybe_unused]], bool prefetch, ThreadPool & thread_pool, std::atomic & is_cancelled) const { if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) mergeDataNullKey(table_dst, table_src, arena); @@ -2410,7 +2410,7 @@ void NO_INLINE Aggregator::mergeDataImpl( { if (!is_aggregate_function_compiled[i]) aggregate_functions[i]->mergeAndDestroyBatch( - dst_places.data(), src_places.data(), dst_places.size(), offsets_of_aggregate_states[i], arena); + dst_places.data(), src_places.data(), dst_places.size(), offsets_of_aggregate_states[i], thread_pool, is_cancelled, arena); } return; @@ -2420,7 +2420,7 @@ void NO_INLINE Aggregator::mergeDataImpl( for (size_t i = 0; i < params.aggregates_size; ++i) { aggregate_functions[i]->mergeAndDestroyBatch( - dst_places.data(), src_places.data(), dst_places.size(), offsets_of_aggregate_states[i], arena); + dst_places.data(), src_places.data(), dst_places.size(), offsets_of_aggregate_states[i], thread_pool, is_cancelled, arena); } } @@ -2535,8 +2535,10 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl( template void NO_INLINE Aggregator::mergeSingleLevelDataImpl( - ManyAggregatedDataVariants & non_empty_data) const + ManyAggregatedDataVariants & non_empty_data, std::atomic & is_cancelled) const { + ThreadPool thread_pool{CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, CurrentMetrics::AggregatorThreadsScheduled, params.max_threads}; + AggregatedDataVariantsPtr & res = non_empty_data[0]; bool no_more_keys = false; @@ -2557,13 +2559,13 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl( if (compiled_aggregate_functions_holder) { mergeDataImpl( - getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool, true, prefetch); + getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool, true, prefetch, thread_pool, is_cancelled); } else #endif { mergeDataImpl( - getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool, false, prefetch); + getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool, false, prefetch, thread_pool, is_cancelled); } } else if (res->without_key) @@ -2589,7 +2591,7 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl( #define M(NAME) \ template void NO_INLINE Aggregator::mergeSingleLevelDataImpl( \ - ManyAggregatedDataVariants & non_empty_data) const; + ManyAggregatedDataVariants & non_empty_data, std::atomic & is_cancelled) const; APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) #undef M @@ -2597,6 +2599,8 @@ template void NO_INLINE Aggregator::mergeBucketImpl( ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena, std::atomic & is_cancelled) const { + ThreadPool thread_pool{CurrentMetrics::AggregatorThreads, CurrentMetrics::AggregatorThreadsActive, CurrentMetrics::AggregatorThreadsScheduled, params.max_threads}; + /// We merge all aggregation results to the first. AggregatedDataVariantsPtr & res = data[0]; @@ -2613,7 +2617,7 @@ void NO_INLINE Aggregator::mergeBucketImpl( if (compiled_aggregate_functions_holder) { mergeDataImpl( - getDataVariant(*res).data.impls[bucket], getDataVariant(current).data.impls[bucket], arena, true, prefetch); + getDataVariant(*res).data.impls[bucket], getDataVariant(current).data.impls[bucket], arena, true, prefetch, thread_pool, is_cancelled); } else #endif @@ -2623,7 +2627,9 @@ void NO_INLINE Aggregator::mergeBucketImpl( getDataVariant(current).data.impls[bucket], arena, false, - prefetch); + prefetch, + thread_pool, + is_cancelled); } } } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index f4f1e9a1df3..c357da2d9de 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -455,7 +455,7 @@ private: /// Merge data from hash table `src` into `dst`. template - void mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena, bool use_compiled_functions, bool prefetch) const; + void mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena, bool use_compiled_functions, bool prefetch, ThreadPool & thread_pool, std::atomic & is_cancelled) const; /// Merge data from hash table `src` into `dst`, but only for keys that already exist in dst. In other cases, merge the data into `overflows`. template @@ -478,7 +478,7 @@ private: template void mergeSingleLevelDataImpl( - ManyAggregatedDataVariants & non_empty_data) const; + ManyAggregatedDataVariants & non_empty_data, std::atomic & is_cancelled) const; template using ConvertToBlockRes = std::conditional_t; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index cdbe194cfac..5e7fba6d80c 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -495,7 +495,7 @@ private: #define M(NAME) \ else if (first->type == AggregatedDataVariants::Type::NAME) \ - params->aggregator.mergeSingleLevelDataImplNAME)::element_type>(*data); + params->aggregator.mergeSingleLevelDataImplNAME)::element_type>(*data, shared_data->is_cancelled); if (false) {} // NOLINT APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) #undef M From 32f4b1f89143a293047beb83169d83c3e20c5480 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 17:49:38 +0000 Subject: [PATCH 1065/1722] Add aggregate functions distinctDynamicTypes/distinctJSONPaths/distinctJSONPathsAndTypes --- .../reference/distinctdynamictypes.md | 44 +++ .../reference/distinctjsonpaths.md | 84 +++++ docs/en/sql-reference/data-types/newjson.md | 125 ++++++- .../AggregateFunctionDistinctDynamicTypes.cpp | 155 ++++++++ .../AggregateFunctionDistinctJSONPaths.cpp | 331 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 4 + src/Columns/ColumnDynamic.cpp | 35 ++ src/Columns/ColumnDynamic.h | 3 + ...istinct_dynamic_types_json_paths.reference | 121 +++++++ ...3227_distinct_dynamic_types_json_paths.sql | 50 +++ 10 files changed, 951 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md create mode 100644 docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md create mode 100644 src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp create mode 100644 tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference create mode 100644 tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md b/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md new file mode 100644 index 00000000000..970209252fa --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/distinctdynamictypes +sidebar_position: 215 +--- + +# distinctDynamicTypes + +Calculates the list of distinct data types stored in [Dynamic](../../data-types/dynamic.md) column. + +**Syntax** + +```sql +distinctDynamicTypes(dynamic) +``` + +**Arguments** + +- `dynamic` — [Dynamic](../../data-types/dynamic.md) column. + +**Returned Value** + +- The sorted list of data type names [Array(String)](../../data-types/array.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_dynamic; +CREATE TABLE test_dynamic(d Dynamic) ENGINE = Memory; +INSERT INTO test_dynamic VALUES (42), (NULL), ('Hello'), ([1, 2, 3]), ('2020-01-01'), (map(1, 2)), (43), ([4, 5]), (NULL), ('World'), (map(3, 4)) +``` + +```sql +SELECT distinctDynamicTypes(d) FROM test_dynamic; +``` + +Result: + +```reference +┌─distinctDynamicTypes(d)──────────────────────────────────────┠+│ ['Array(Int64)','Date','Int64','Map(UInt8, UInt8)','String'] │ +└──────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md new file mode 100644 index 00000000000..f916734ca44 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md @@ -0,0 +1,84 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/distinctjsonpaths +sidebar_position: 216 +--- + +# distinctJSONPaths + +Calculates the list of distinct paths stored in [JSON](../../data-types/newjson.md) column. + +**Syntax** + +```sql +distinctJSONPaths(json) +``` + +**Arguments** + +- `json` — [JSON](../../data-types/newjson.md) column. + +**Returned Value** + +- The sorted list of paths [Array(String)](../../data-types/array.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"a" : 42, "b" : "Hello"}'), ('{"b" : [1, 2, 3], "c" : {"d" : {"e" : "2020-01-01"}}}'), ('{"a" : 43, "c" : {"d" : {"f" : [{"g" : 42}]}}}') +``` + +```sql +SELECT distinctJSONPaths(json) FROM test_json; +``` + +Result: + +```reference +┌─distinctJSONPaths(json)───┠+│ ['a','b','c.d.e','c.d.f'] │ +└───────────────────────────┘ +``` + +# distinctJSONPathsAndTypes + +Calculates the list of distinct paths and their types stored in [JSON](../../data-types/newjson.md) column. + +**Syntax** + +```sql +distinctJSONPathsAndTypes(json) +``` + +**Arguments** + +- `json` — [JSON](../../data-types/newjson.md) column. + +**Returned Value** + +- The sorted map of paths and types [Map(String, Array(String))](../../data-types/map.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"a" : 42, "b" : "Hello"}'), ('{"b" : [1, 2, 3], "c" : {"d" : {"e" : "2020-01-01"}}}'), ('{"a" : 43, "c" : {"d" : {"f" : [{"g" : 42}]}}}') +``` + +```sql +SELECT distinctJSONPathsAndTypes(json) FROM test_json; +``` + +Result: + +```reference +┌─distinctJSONPathsAndTypes(json)───────────────────────────────────────────────────────────────────────────────────────────────────────────────┠+│ {'a':['Int64'],'b':['Array(Nullable(Int64))','String'],'c.d.e':['Date'],'c.d.f':['Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))']} │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 9e43216df6c..16cc324a168 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -505,7 +505,130 @@ As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and move ## Introspection functions -There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes). +There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes), [distinctDynamicTypes](../aggregate-functions/reference/distinctdynamictypes.md), [distinctJSONPaths and distinctJSONPathsAndTypes](../aggregate-functions/reference/distinctjsonpaths.md) + +**Examples** + +Let's investigate the content of [GH Archive](https://www.gharchive.org/) dataset for `2020-01-01` date: + +```sql +SELECT arrayJoin(distinctJSONPaths(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +``` + +```text +┌─arrayJoin(distinctJSONPaths(json))─────────────────────────┠+│ actor.avatar_url │ +│ actor.display_login │ +│ actor.gravatar_id │ +│ actor.id │ +│ actor.login │ +│ actor.url │ +│ created_at │ +│ id │ +│ org.avatar_url │ +│ org.gravatar_id │ +│ org.id │ +│ org.login │ +│ org.url │ +│ payload.action │ +│ payload.before │ +│ payload.comment._links.html.href │ +│ payload.comment._links.pull_request.href │ +│ payload.comment._links.self.href │ +│ payload.comment.author_association │ +│ payload.comment.body │ +│ payload.comment.commit_id │ +│ payload.comment.created_at │ +│ payload.comment.diff_hunk │ +│ payload.comment.html_url │ +│ payload.comment.id │ +│ payload.comment.in_reply_to_id │ +│ payload.comment.issue_url │ +│ payload.comment.line │ +│ payload.comment.node_id │ +│ payload.comment.original_commit_id │ +│ payload.comment.original_position │ +│ payload.comment.path │ +│ payload.comment.position │ +│ payload.comment.pull_request_review_id │ +... +│ payload.release.node_id │ +│ payload.release.prerelease │ +│ payload.release.published_at │ +│ payload.release.tag_name │ +│ payload.release.tarball_url │ +│ payload.release.target_commitish │ +│ payload.release.upload_url │ +│ payload.release.url │ +│ payload.release.zipball_url │ +│ payload.size │ +│ public │ +│ repo.id │ +│ repo.name │ +│ repo.url │ +│ type │ +└─arrayJoin(distinctJSONPaths(json))─────────────────────────┘ +``` + +```sql +SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +``` + + +```text +┌─arrayJoin(distinctJSONPathsAndTypes(json))──────────────────┠+│ ('actor.avatar_url',['String']) │ +│ ('actor.display_login',['String']) │ +│ ('actor.gravatar_id',['String']) │ +│ ('actor.id',['Int64']) │ +│ ('actor.login',['String']) │ +│ ('actor.url',['String']) │ +│ ('created_at',['String']) │ +│ ('id',['String']) │ +│ ('org.avatar_url',['String']) │ +│ ('org.gravatar_id',['String']) │ +│ ('org.id',['Int64']) │ +│ ('org.login',['String']) │ +│ ('org.url',['String']) │ +│ ('payload.action',['String']) │ +│ ('payload.before',['String']) │ +│ ('payload.comment._links.html.href',['String']) │ +│ ('payload.comment._links.pull_request.href',['String']) │ +│ ('payload.comment._links.self.href',['String']) │ +│ ('payload.comment.author_association',['String']) │ +│ ('payload.comment.body',['String']) │ +│ ('payload.comment.commit_id',['String']) │ +│ ('payload.comment.created_at',['String']) │ +│ ('payload.comment.diff_hunk',['String']) │ +│ ('payload.comment.html_url',['String']) │ +│ ('payload.comment.id',['Int64']) │ +│ ('payload.comment.in_reply_to_id',['Int64']) │ +│ ('payload.comment.issue_url',['String']) │ +│ ('payload.comment.line',['Int64']) │ +│ ('payload.comment.node_id',['String']) │ +│ ('payload.comment.original_commit_id',['String']) │ +│ ('payload.comment.original_position',['Int64']) │ +│ ('payload.comment.path',['String']) │ +│ ('payload.comment.position',['Int64']) │ +│ ('payload.comment.pull_request_review_id',['Int64']) │ +... +│ ('payload.release.node_id',['String']) │ +│ ('payload.release.prerelease',['Bool']) │ +│ ('payload.release.published_at',['String']) │ +│ ('payload.release.tag_name',['String']) │ +│ ('payload.release.tarball_url',['String']) │ +│ ('payload.release.target_commitish',['String']) │ +│ ('payload.release.upload_url',['String']) │ +│ ('payload.release.url',['String']) │ +│ ('payload.release.zipball_url',['String']) │ +│ ('payload.size',['Int64']) │ +│ ('public',['Bool']) │ +│ ('repo.id',['Int64']) │ +│ ('repo.name',['String']) │ +│ ('repo.url',['String']) │ +│ ('type',['String']) │ +└─arrayJoin(distinctJSONPathsAndTypes(json))──────────────────┘ +``` ## Tips for better usage of the JSON type diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp new file mode 100644 index 00000000000..6c899d6cda8 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -0,0 +1,155 @@ +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct AggregateFunctionDistinctDynamicTypesData +{ + std::unordered_set data; + + void add(const String & type) + { + data.insert(type); + } + + void merge(const AggregateFunctionDistinctDynamicTypesData & other) + { + data.insert(other.data.begin(), other.data.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & type : data) + writeStringBinary(type, buf); + } + + void deserialize(ReadBuffer & buf) + { + size_t size; + readVarUInt(size, buf); + data.reserve(size); + String type; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(type, buf); + data.insert(type); + } + } + + void insertResultInto(IColumn & column) + { + /// Insert types in sorted order for better output. + auto & array_column = assert_cast(column); + auto & string_column = assert_cast(array_column.getData()); + std::vector sorted_data(data.begin(), data.end()); + std::sort(sorted_data.begin(), sorted_data.end()); + for (const auto & type : sorted_data) + string_column.insertData(type.data(), type.size()); + array_column.getOffsets().push_back(string_column.size()); + } +}; + +/// Calculates the list of distinct data types in Dynamic column. +class AggregateFunctionDistinctDynamicTypes final : public IAggregateFunctionDataHelper +{ +public: + explicit AggregateFunctionDistinctDynamicTypes(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}, std::make_shared(std::make_shared())) + { + } + + String getName() const override { return "distinctDynamicTypes"; } + + bool allocatesMemoryInArena() const override { return false; } + + void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & dynamic_column = assert_cast(*columns[0]); + if (dynamic_column.isNullAt(row_num)) + return; + + this->data(place).add(dynamic_column.getTypeNameAt(row_num)); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) + const override + { + if (if_argument_pos >= 0 || row_begin != 0 || row_end != columns[0]->size()) + IAggregateFunctionDataHelper::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + /// Optimization for case when we add all rows from the column into single place. + /// In this case we can avoid iterating over all rows because we can get all types + /// in Dynamic column in a more efficient way. + else + assert_cast(*columns[0]).getAllTypeNames(this->data(place).data); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + /// Default value for Dynamic is NULL, so nothing to add. + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + this->data(place).insertResultInto(to); + } +}; + +AggregateFunctionPtr createAggregateFunctionDistinctDynamicTypes( + const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + if (argument_types.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function {}. Expected single argument with type Dynamic, got {} arguments", name, argument_types.size()); + + if (!isDynamic(argument_types[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}. Expected type Dynamic", argument_types[0]->getName(), name); + + return std::make_shared(argument_types); +} + +void registerAggregateFunctionDistinctDynamicTypes(AggregateFunctionFactory & factory) +{ + factory.registerFunction("distinctDynamicTypes", createAggregateFunctionDistinctDynamicTypes); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp new file mode 100644 index 00000000000..a3ede0750f2 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -0,0 +1,331 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +struct AggregateFunctionDistinctJSONPathsData +{ + static constexpr auto name = "distinctJSONPaths"; + + std::unordered_set data; + + void add(const ColumnObject & column, size_t row_num, const std::unordered_map &) + { + for (const auto & [path, _] : column.getTypedPaths()) + data.insert(path); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add path from dynamic paths only if it's not NULL in this row. + if (!dynamic_column->isNullAt(row_num)) + data.insert(path); + } + + /// Iterate over paths in shared data in this row. + const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column.getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(row_num) - 1]; + size_t end = shared_data_offsets[static_cast(row_num)]; + for (size_t i = start; i != end; ++i) + data.insert(shared_data_paths->getDataAt(i).toString()); + } + + void addWholeColumn(const ColumnObject & column, const std::unordered_map &) + { + for (const auto & [path, _] : column.getTypedPaths()) + data.insert(path); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add dynamic path only if it has at least one non-null value. + /// getNumberOfDefaultRows for Dynamic column is O(1). + if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) + data.insert(path); + } + + /// Iterate over all paths in shared data. + const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + data.insert(shared_data_paths->getDataAt(i).toString()); + } + + void merge(const AggregateFunctionDistinctJSONPathsData & other) + { + data.insert(other.data.begin(), other.data.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & path : data) + writeStringBinary(path, buf); + } + + void deserialize(ReadBuffer & buf) + { + size_t size; + readVarUInt(size, buf); + String path; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(path, buf); + data.insert(path); + } + } + + void insertResultInto(IColumn & column) + { + /// Insert paths in sorted order for better output. + auto & array_column = assert_cast(column); + auto & string_column = assert_cast(array_column.getData()); + std::vector sorted_data(data.begin(), data.end()); + std::sort(sorted_data.begin(), sorted_data.end()); + for (const auto & path : sorted_data) + string_column.insertData(path.data(), path.size()); + array_column.getOffsets().push_back(string_column.size()); + } + + static DataTypePtr getResultType() + { + return std::make_shared(std::make_shared()); + } +}; + +struct AggregateFunctionDistinctJSONPathsAndTypesData +{ + static constexpr auto name = "distinctJSONPathsAndTypes"; + + std::unordered_map> data; + + void add(const ColumnObject & column, size_t row_num, const std::unordered_map & typed_paths_type_names) + { + for (const auto & [path, _] : column.getTypedPaths()) + data[path].insert(typed_paths_type_names.at(path)); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + if (!dynamic_column->isNullAt(row_num)) + data[path].insert(dynamic_column->getTypeNameAt(row_num)); + } + + /// Iterate over paths om shared data in this row and decode the data types. + const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column.getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(row_num) - 1]; + size_t end = shared_data_offsets[static_cast(row_num)]; + for (size_t i = start; i != end; ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + /// We should not have Nulls here but let's check just in case. + if (!isNothing(type)) + data[path].insert(type->getName()); + } + } + + void addWholeColumn(const ColumnObject & column, const std::unordered_map & typed_paths_type_names) + { + for (const auto & [path, _] : column.getTypedPaths()) + data[path].insert(typed_paths_type_names.at(path)); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add dynamic path only if it has at least one non-null value. + /// getNumberOfDefaultRows for Dynamic column is O(1). + if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) + dynamic_column->getAllTypeNames(data[path]); + } + + /// Iterate over all paths in shared data and decode the data types. + const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + /// We should not have Nulls here but let's check just in case. + if (!isNothing(type)) + data[path].insert(type->getName()); + } + } + + void merge(const AggregateFunctionDistinctJSONPathsAndTypesData & other) + { + for (const auto & [path, types] : other.data) + data[path].insert(types.begin(), types.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & [path, types] : data) + { + writeStringBinary(path, buf); + writeVarUInt(types.size(), buf); + for (const auto & type : types) + writeStringBinary(type, buf); + } + } + + void deserialize(ReadBuffer & buf) + { + size_t paths_size, types_size; + readVarUInt(paths_size, buf); + data.reserve(paths_size); + String path, type; + for (size_t i = 0; i != paths_size; ++i) + { + readStringBinary(path, buf); + readVarUInt(types_size, buf); + data[path].reserve(types_size); + for (size_t j = 0; j != types_size; ++j) + { + readStringBinary(type, buf); + data[path].insert(type); + } + } + } + + void insertResultInto(IColumn & column) + { + /// Insert sorted paths and types for better output. + auto & array_column = assert_cast(column).getNestedColumn(); + auto & tuple_column = assert_cast(array_column.getData()); + auto & key_column = assert_cast(tuple_column.getColumn(0)); + auto & value_column = assert_cast(tuple_column.getColumn(1)); + auto & value_column_data = assert_cast(value_column.getData()); + std::vector>> sorted_data; + sorted_data.reserve(data.size()); + for (const auto & [path, types] : data) + { + std::vector sorted_types(types.begin(), types.end()); + std::sort(sorted_types.begin(), sorted_types.end()); + sorted_data.emplace_back(path, std::move(sorted_types)); + } + std::sort(sorted_data.begin(), sorted_data.end()); + + for (const auto & [path, types] : sorted_data) + { + key_column.insertData(path.data(), path.size()); + for (const auto & type : types) + value_column_data.insertData(type.data(), type.size()); + value_column.getOffsets().push_back(value_column_data.size()); + } + + array_column.getOffsets().push_back(key_column.size()); + } + + static DataTypePtr getResultType() + { + return std::make_shared(std::make_shared(), std::make_shared(std::make_shared())); + } +}; + +/// Calculates the list of distinct data types in Dynamic column. +template +class AggregateFunctionDistinctJSONPathsAndTypes final : public IAggregateFunctionDataHelper> +{ +public: + explicit AggregateFunctionDistinctJSONPathsAndTypes(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper>( + argument_types_, {}, Data::getResultType()) + { + const auto & typed_paths_types = assert_cast(*argument_types_[0]).getTypedPaths(); + typed_paths_type_names.reserve(typed_paths_types.size()); + for (const auto & [path, type] : typed_paths_types) + typed_paths_type_names[path] = type->getName(); + } + + String getName() const override { return Data::name; } + + bool allocatesMemoryInArena() const override { return false; } + + void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & object_column = assert_cast(*columns[0]); + this->data(place).add(object_column, row_num, typed_paths_type_names); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) + const override + { + if (if_argument_pos >= 0 || row_begin != 0 || row_end != columns[0]->size()) + IAggregateFunctionDataHelper>::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + /// Optimization for case when we add all rows from the column into single place. + /// In this case we can avoid iterating over all rows because we can get all paths + /// and types in JSON column in a more efficient way. + else + this->data(place).addWholeColumn(assert_cast(*columns[0]), typed_paths_type_names); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + /// Default value for JSON is empty object, so nothing to add. + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + this->data(place).insertResultInto(to); + } + +private: + std::unordered_map typed_paths_type_names; +}; + +template +AggregateFunctionPtr createAggregateFunctionDistinctJSONPathsAndTypes( + const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + if (argument_types.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function {}. Expected single argument with type JSON, got {} arguments", name, argument_types.size()); + + if (!isObject(argument_types[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}. Expected type JSON", argument_types[0]->getName(), name); + + return std::make_shared>(argument_types); +} + +void registerAggregateFunctionDistinctJSONPathsAndTypes(AggregateFunctionFactory & factory) +{ + factory.registerFunction("distinctJSONPaths", createAggregateFunctionDistinctJSONPathsAndTypes); + factory.registerFunction("distinctJSONPathsAndTypes", createAggregateFunctionDistinctJSONPathsAndTypes); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 4ac25e14ee6..068f1aebd87 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -89,6 +89,8 @@ void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &); void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &); void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory); void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory); +void registerAggregateFunctionDistinctDynamicTypes(AggregateFunctionFactory & factory); +void registerAggregateFunctionDistinctJSONPathsAndTypes(AggregateFunctionFactory & factory); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -191,6 +193,8 @@ void registerAggregateFunctions() registerAggregateFunctionFlameGraph(factory); registerAggregateFunctionKolmogorovSmirnovTest(factory); registerAggregateFunctionLargestTriangleThreeBuckets(factory); + registerAggregateFunctionDistinctDynamicTypes(factory); + registerAggregateFunctionDistinctJSONPathsAndTypes(factory); registerWindowFunctions(factory); } diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..b5378e983c6 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -980,6 +980,41 @@ ColumnPtr ColumnDynamic::compress() const }); } +String ColumnDynamic::getTypeNameAt(size_t row_num) const +{ + const auto & variant_col = getVariantColumn(); + size_t discr = variant_col.globalDiscriminatorAt(row_num); + if (discr == ColumnVariant::NULL_DISCRIMINATOR) + return ""; + + if (discr == getSharedVariantDiscriminator()) + { + const auto value = getSharedVariant().getDataAt(variant_col.offsetAt(row_num)); + ReadBufferFromMemory buf(value.data, value.size); + return decodeDataType(buf)->getName(); + } + + return variant_info.variant_names[discr]; +} + +void ColumnDynamic::getAllTypeNames(std::unordered_set & names) const +{ + auto shared_variant_discr = getSharedVariantDiscriminator(); + for (size_t i = 0; i != variant_info.variant_names.size(); ++i) + { + if (i != shared_variant_discr && !variant_column_ptr->getVariantByGlobalDiscriminator(i).empty()) + names.insert(variant_info.variant_names[i]); + } + + const auto & shared_variant = getSharedVariant(); + for (size_t i = 0; i != shared_variant.size(); ++i) + { + const auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + names.insert(decodeDataType(buf)->getName()); + } +} + void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { if (source_columns.empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 2ae862de3af..c06c31bb8c9 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -430,6 +430,9 @@ public: const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } + String getTypeNameAt(size_t row_num) const; + void getAllTypeNames(std::unordered_set & names) const; + private: void createVariantInfo(const DataTypePtr & variant_type); diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference new file mode 100644 index 00000000000..d6538a1c4eb --- /dev/null +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference @@ -0,0 +1,121 @@ +a0 +a1 +a10 +a11 +a12 +a2 +a3 +a4 +a5 +a6 +a7 +a8 +a9 +('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a1',['String']) +('a10',['Array(Nullable(Int64))','Date','Int64','String']) +('a11',['Array(Nullable(Int64))','Date','Int64','String']) +('a12',['Array(Nullable(Int64))','Date','Int64','String']) +('a2',['Array(Nullable(Int64))','Date','Int64','String']) +('a3',['Array(Nullable(Int64))','Date','Int64','String']) +('a4',['Array(Nullable(Int64))','Date','Int64','String']) +('a5',['Array(Nullable(Int64))','Date','Int64','String']) +('a6',['Array(Nullable(Int64))','Date','Int64','String']) +('a7',['Array(Nullable(Int64))','Date','Int64','String']) +('a8',['Array(Nullable(Int64))','Date','Int64','String']) +('a9',['Array(Nullable(Int64))','Date','Int64','String']) +Array(Nullable(Int64)) +Date +Int64 +String +Array(Nullable(Int64)) +Date +Int64 +String +Filter +a1 +a2 +('a1',['String']) +('a2',['String']) +String +If +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Group by +Array(Nullable(Int64)) ['a1','a2'] +Date ['a1','a2'] +Int64 ['a1','a2'] +None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] +String ['a1','a2'] +Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} +Date {'a1':['String'],'a2':['Date']} +Int64 {'a1':['String'],'a2':['Int64']} +None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +String {'a1':['String'],'a2':['String']} +Array(Nullable(Int64)) ['Array(Nullable(Int64))'] +Date ['Date'] +Int64 ['Int64'] +None [] +String ['String'] +Remote +a0 +a1 +a10 +a11 +a12 +a2 +a3 +a4 +a5 +a6 +a7 +a8 +a9 +('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a1',['String']) +('a10',['Array(Nullable(Int64))','Date','Int64','String']) +('a11',['Array(Nullable(Int64))','Date','Int64','String']) +('a12',['Array(Nullable(Int64))','Date','Int64','String']) +('a2',['Array(Nullable(Int64))','Date','Int64','String']) +('a3',['Array(Nullable(Int64))','Date','Int64','String']) +('a4',['Array(Nullable(Int64))','Date','Int64','String']) +('a5',['Array(Nullable(Int64))','Date','Int64','String']) +('a6',['Array(Nullable(Int64))','Date','Int64','String']) +('a7',['Array(Nullable(Int64))','Date','Int64','String']) +('a8',['Array(Nullable(Int64))','Date','Int64','String']) +('a9',['Array(Nullable(Int64))','Date','Int64','String']) +Array(Nullable(Int64)) +Date +Int64 +String +Remote filter +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Remote if +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Remote group by +Array(Nullable(Int64)) ['a1','a2'] +Date ['a1','a2'] +Int64 ['a1','a2'] +None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] +String ['a1','a2'] +Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} +Date {'a1':['String'],'a2':['Date']} +Int64 {'a1':['String'],'a2':['Int64']} +None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +String {'a1':['String'],'a2':['String']} +Array(Nullable(Int64)) ['Array(Nullable(Int64))'] +Date ['Date'] +Int64 ['Int64'] +None [] +String ['String'] diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql new file mode 100644 index 00000000000..4f33cc10d46 --- /dev/null +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -0,0 +1,50 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_json_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test_json_dynamic_aggregate_functions; +create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toDate(number)))) from numbers(200000); +select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a3)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a42)) from test_json_dynamic_aggregate_functions; + +select 'Filter'; +select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; + +select 'If'; +select arrayJoin(distinctJSONPathsIf(json, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctJSONPathsAndTypesIf(json, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypesIf(json.a2, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; + +select 'Group by'; +select dynamicType(json.a2), distinctJSONPaths(json) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctDynamicTypes(json.a2) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); + +select 'Remote'; +select arrayJoin(distinctJSONPaths(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctJSONPathsAndTypes(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctDynamicTypes(json.a2)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); + +select 'Remote filter'; +select arrayJoin(distinctJSONPaths(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctJSONPathsAndTypes(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctDynamicTypes(json.a2)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; + +select 'Remote if'; +select arrayJoin(distinctJSONPathsIf(json, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctJSONPathsAndTypesIf(json, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctDynamicTypesIf(json.a2, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); + +select 'Remote group by'; +select dynamicType(json.a2), distinctJSONPaths(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctDynamicTypes(json.a2) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); + +drop table test_json_dynamic_aggregate_functions; From fd266284e1a2c5170183f7e1d2425b4b2b6ca103 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 19:49:47 +0200 Subject: [PATCH 1066/1722] Fix test `02122_join_group_by_timeout` --- tests/queries/0_stateless/02122_join_group_by_timeout.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.sh b/tests/queries/0_stateless/02122_join_group_by_timeout.sh index b4644878544..17d8200f20c 100755 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.sh +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.sh @@ -15,6 +15,7 @@ fi # TCP CLIENT: As of today (02/12/21) uses PullingAsyncPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) +### However, in the test, the server can be overloaded, so we assert query duration in the interval of 1 to 60 seconds. query_id=$(random_str 12) $CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q " SELECT * FROM @@ -33,7 +34,7 @@ $CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q " FORMAT Null " 2>&1 | grep -m1 -o "Code: 159" $CLICKHOUSE_CLIENT -q "system flush logs" -${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) BETWEEN 1 AND 60 from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" ### Should stop pulling data and return what has been generated already (return code 0) @@ -52,7 +53,7 @@ $CLICKHOUSE_CLIENT --query_id "$query_id" -q " " echo $? $CLICKHOUSE_CLIENT -q "system flush logs" -${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) BETWEEN 1 AND 60 from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" # HTTP CLIENT: As of today (02/12/21) uses PullingPipelineExecutor From 85a813bbedeec62fdf4704041a62092bbf6a9789 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 17:55:57 +0000 Subject: [PATCH 1067/1722] Better example --- docs/en/sql-reference/data-types/newjson.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 16cc324a168..99156b11a0b 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -571,7 +571,7 @@ SELECT arrayJoin(distinctJSONPaths(json)) FROM s3('s3://clickhouse-public-datase ``` ```sql -SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) SETTINGS date_time_input_format='best_effort' ``` @@ -583,7 +583,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi │ ('actor.id',['Int64']) │ │ ('actor.login',['String']) │ │ ('actor.url',['String']) │ -│ ('created_at',['String']) │ +│ ('created_at',['DateTime']) │ │ ('id',['String']) │ │ ('org.avatar_url',['String']) │ │ ('org.gravatar_id',['String']) │ @@ -598,7 +598,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi │ ('payload.comment.author_association',['String']) │ │ ('payload.comment.body',['String']) │ │ ('payload.comment.commit_id',['String']) │ -│ ('payload.comment.created_at',['String']) │ +│ ('payload.comment.created_at',['DateTime']) │ │ ('payload.comment.diff_hunk',['String']) │ │ ('payload.comment.html_url',['String']) │ │ ('payload.comment.id',['Int64']) │ @@ -614,7 +614,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi ... │ ('payload.release.node_id',['String']) │ │ ('payload.release.prerelease',['Bool']) │ -│ ('payload.release.published_at',['String']) │ +│ ('payload.release.published_at',['DateTime']) │ │ ('payload.release.tag_name',['String']) │ │ ('payload.release.tarball_url',['String']) │ │ ('payload.release.target_commitish',['String']) │ From f21e982f03488634cd09ec7ab570fec07d78a0e1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 18:01:35 +0000 Subject: [PATCH 1068/1722] Fix style, update tests --- .../AggregateFunctionDistinctJSONPaths.cpp | 18 ++++++++++++------ ...03227_distinct_dynamic_types_json_paths.sql | 10 ++++++++++ .../aspell-ignore/en/aspell-dict.txt | 5 +++++ 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index a3ede0750f2..58ce7e27ab0 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -19,6 +19,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + struct AggregateFunctionDistinctJSONPathsData { static constexpr auto name = "distinctJSONPaths"; @@ -44,7 +50,7 @@ struct AggregateFunctionDistinctJSONPathsData for (size_t i = start; i != end; ++i) data.insert(shared_data_paths->getDataAt(i).toString()); } - + void addWholeColumn(const ColumnObject & column, const std::unordered_map &) { for (const auto & [path, _] : column.getTypedPaths()) @@ -98,7 +104,7 @@ struct AggregateFunctionDistinctJSONPathsData string_column.insertData(path.data(), path.size()); array_column.getOffsets().push_back(string_column.size()); } - + static DataTypePtr getResultType() { return std::make_shared(std::make_shared()); @@ -110,7 +116,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData static constexpr auto name = "distinctJSONPathsAndTypes"; std::unordered_map> data; - + void add(const ColumnObject & column, size_t row_num, const std::unordered_map & typed_paths_type_names) { for (const auto & [path, _] : column.getTypedPaths()) @@ -137,7 +143,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData data[path].insert(type->getName()); } } - + void addWholeColumn(const ColumnObject & column, const std::unordered_map & typed_paths_type_names) { for (const auto & [path, _] : column.getTypedPaths()) @@ -229,7 +235,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData array_column.getOffsets().push_back(key_column.size()); } - + static DataTypePtr getResultType() { return std::make_shared(std::make_shared(), std::make_shared(std::make_shared())); @@ -302,7 +308,7 @@ public: { this->data(place).insertResultInto(to); } - + private: std::unordered_map typed_paths_type_names; }; diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index 4f33cc10d46..f7707b536e6 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -47,4 +47,14 @@ select dynamicType(json.a2), distinctJSONPaths(json) from remote('127.0.0.{1,2,3 select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); select dynamicType(json.a2), distinctDynamicTypes(json.a2) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select distinctJSONPaths() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPaths(json, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPaths(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select distinctJSONPathsAndTypes() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPathsAndTypes(json, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPathsAndTypes(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select distinctDynamicTypes() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctDynamicTypes(json.a2, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctDynamicTypes(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} + drop table test_json_dynamic_aggregate_functions; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index fd836d93143..a5dd1ac5f9d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1551,6 +1551,11 @@ disjunctions displayName displaySecretsInShowAndSelect distro +distinctdynamictypes +distinctDynamicTypes +distinctjsonpaths +distinctJSONPaths +distinctJSONPathsAndTypes divideDecimal dmesg doesnt From 76960eff8005818db7f744115798888e72e4b2e5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 18:38:03 +0000 Subject: [PATCH 1069/1722] This should work --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index 73752937366..b0fce6066fc 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 737529373665bc067971ba098a12d6928580a0ae +Subproject commit b0fce6066fc2678fa17ee7a98f794da9da8492ff From faad7f4ba27f665ecdb4fa6212a695f86413746a Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 16 Aug 2024 20:35:44 +0000 Subject: [PATCH 1070/1722] Fix Broken pipe error for 03149_numbers_max_block_size_zero.sh The error appeared while debugging https://github.com/ClickHouse/clickhouse-private/issues/14225 Logs: https://pastila.nl/?00628486/754eaf7d96fd03ceecdf1a45458867dc#B9vFn07WAielph/Z5lHbrQ== From the `man grep`: > -q, --quiet, --silent > <...> Exit immediately with zero status if any match is found, even if an error was detected. When `grep -q` finds a match, it immediately exits with status `0 and closes its side of the pipe. If the clickhouse-client is still trying to send data through the pipe, it leads to SIGPIPE signal. Use grep -c instead. It is less efficient, but the output in this test is small. We should also revisit how we handle SIGPIPE signal, e.g. the server should not try to send logs if it already encountered the Broken pipe error. --- tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh index 6f70a0d2536..2c4669325bb 100755 --- a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh @@ -4,4 +4,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | grep -q "Sanity check: 'max_block_size' cannot be 0. Set to default value" && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | [ $(grep -c "Sanity check: 'max_block_size' cannot be 0. Set to default value") -gt 0 ] && echo "OK" || echo "FAIL" From a6d5047bb09640bcf99bef84f655602d7dfb3361 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 16 Aug 2024 21:29:46 +0100 Subject: [PATCH 1071/1722] impl --- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 4 ++++ .../MergeTree/MergeTreeReadPoolBase.cpp | 6 +++++ .../MergeTreeReadPoolParallelReplicas.cpp | 5 +++++ ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 5 +++++ .../ParallelReplicasReadingCoordinator.cpp | 4 ++++ ...icas_read_task_size_overflow_bug.reference | 0 ...l_replicas_read_task_size_overflow_bug.sql | 22 +++++++++++++++++++ 7 files changed, 46 insertions(+) create mode 100644 tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.reference create mode 100644 tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.sql diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index a9b77fb6c03..7081eb716f5 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -395,6 +395,10 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_ part_stat.prefetch_step_marks = std::max(part_stat.prefetch_step_marks, per_part_infos[i]->min_marks_per_task); + if (part_stat.prefetch_step_marks == 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + LOG_DEBUG( log, "Part: {}, sum_marks: {}, approx mark size: {}, prefetch_step_bytes: {}, prefetch_step_marks: {}, (ranges: {})", diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 6d2560bc9c7..9d3c38822e1 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } MergeTreeReadPoolBase::MergeTreeReadPoolBase( @@ -85,6 +86,11 @@ static size_t calculateMinMarksPerTask( min_marks_per_task = heuristic_min_marks; } } + + if (min_marks_per_task == 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + LOG_TEST(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task); return min_marks_per_task; } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 33eaf5a49bd..d23072771f2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -8,6 +8,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( @@ -38,6 +39,10 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( for (const auto & info : per_part_infos) min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); + if (min_marks_per_task == 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + extension.all_callback( InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica)); } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 6b5cf978423..42ffc4304b2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -6,6 +6,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrder( @@ -37,6 +38,10 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd for (const auto & info : per_part_infos) min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); + if (min_marks_per_task == 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + for (const auto & part : parts_ranges) request.push_back({part.data_part->info, MarkRanges{}}); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f46b4de10b7..ee47fe3549a 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -1004,6 +1004,10 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request) { + if (request.min_number_of_marks == 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + ProfileEventTimeIncrement watch(ProfileEvents::ParallelReplicasHandleRequestMicroseconds); std::lock_guard lock(mutex); diff --git a/tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.reference b/tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.sql b/tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.sql new file mode 100644 index 00000000000..984c7fe0db7 --- /dev/null +++ b/tests/queries/0_stateless/03223_parallel_replicas_read_task_size_overflow_bug.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS test__fuzz_22 SYNC; + +CREATE TABLE test__fuzz_22 (k Float32, v String) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 1; + +SYSTEM STOP MERGES test__fuzz_22; + +INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1); +INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1); +INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1); +INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1); + +SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', + merge_tree_min_rows_for_concurrent_read = 9223372036854775806, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem = 9223372036854775806; + + SELECT v + FROM test__fuzz_22 +ORDER BY v + LIMIT 10, 10 +SETTINGS max_threads = 4 + FORMAT Null; -- { serverError BAD_ARGUMENTS } + +DROP TABLE test__fuzz_22 SYNC; From 4b11522990f0f3557355a1123f877a9394c438d3 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 16 Aug 2024 14:06:55 -0700 Subject: [PATCH 1072/1722] Update JSON link --- docs/en/sql-reference/data-types/json.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 12ab9f1dddc..e48b308a620 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -79,6 +79,5 @@ SELECT * FROM json FORMAT JSONEachRow ## Related Content -- [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) +- [Using JSON in ClickHouse](/en/integrations/data-formats/json/overview) - [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) -- \ No newline at end of file From cc7d22a7b83440cfbf7d37086ece7fac222f24de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 23:08:16 +0200 Subject: [PATCH 1073/1722] Proper parsing of the PostgreSQL-style CAST operator --- src/Parsers/ExpressionElementParsers.cpp | 26 +++++++++++-------- ..._proper_parsing_of_cast_operator.reference | 4 +++ .../03227_proper_parsing_of_cast_operator.sql | 6 +++++ 3 files changed, 25 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.reference create mode 100644 tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index dd22b80b1cb..ffa1bd93ded 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -853,9 +853,9 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected /// Parse numbers (including decimals), strings, arrays and tuples of them. + Pos begin = pos; const char * data_begin = pos->begin; const char * data_end = pos->end; - bool is_string_literal = pos->type == StringLiteral; if (pos->type == Minus) { @@ -866,7 +866,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected data_end = pos->end; ++pos; } - else if (pos->type == Number || is_string_literal) + else if (pos->type == Number || pos->type == StringLiteral) { ++pos; } @@ -939,18 +939,22 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected { String s; size_t data_size = data_end - data_begin; - if (is_string_literal) + if (begin->type == StringLiteral) { - ReadBufferFromMemory buf(data_begin, data_size); - readQuotedStringWithSQLStyle(s, buf); - assert(buf.count() == data_size); + ASTPtr literal; + if (ParserStringLiteral().parse(begin, literal, expected)) + { + node = createFunctionCast(literal, type_ast); + return true; + } + return false; } else - s = String(data_begin, data_size); - - auto literal = std::make_shared(std::move(s)); - node = createFunctionCast(literal, type_ast); - return true; + { + auto literal = std::make_shared(String(data_begin, data_size)); + node = createFunctionCast(literal, type_ast); + return true; + } } return false; diff --git a/tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.reference b/tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.reference new file mode 100644 index 00000000000..2127d396bb3 --- /dev/null +++ b/tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.reference @@ -0,0 +1,4 @@ +414243 +ABC +A +{"a": \'A\'} diff --git a/tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.sql b/tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.sql new file mode 100644 index 00000000000..0c2e7dc582a --- /dev/null +++ b/tests/queries/0_stateless/03227_proper_parsing_of_cast_operator.sql @@ -0,0 +1,6 @@ +SELECT '414243'::String; +SELECT x'414243'::String; +SELECT b'01000001'::String; +SELECT '{"a": \'\x41\'}'::String; +SELECT '{"a": \'\x4\'}'::String; -- { clientError SYNTAX_ERROR } +SELECT '{"a": \'a\x4\'}'::String; -- { clientError SYNTAX_ERROR } From aee031ad4468b870073dc46770d07cea07aa829f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 23:25:49 +0200 Subject: [PATCH 1074/1722] Slightly better --- src/Parsers/ExpressionElementParsers.cpp | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index ffa1bd93ded..726326bfc85 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -856,6 +856,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected Pos begin = pos; const char * data_begin = pos->begin; const char * data_end = pos->end; + ASTPtr string_literal; if (pos->type == Minus) { @@ -866,10 +867,15 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected data_end = pos->end; ++pos; } - else if (pos->type == Number || pos->type == StringLiteral) + else if (pos->type == Number) { ++pos; } + else if (pos->type == StringLiteral) + { + if (!ParserStringLiteral().parse(begin, string_literal, expected)) + return false; + } else if (isOneOf(pos->type)) { TokenType last_token = OpeningSquareBracket; @@ -939,15 +945,10 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected { String s; size_t data_size = data_end - data_begin; - if (begin->type == StringLiteral) + if (string_literal) { - ASTPtr literal; - if (ParserStringLiteral().parse(begin, literal, expected)) - { - node = createFunctionCast(literal, type_ast); - return true; - } - return false; + node = createFunctionCast(string_literal, type_ast); + return true; } else { From 2a5de86fe4f21d77671a317d52b3f378062c7555 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 16 Aug 2024 21:46:21 +0000 Subject: [PATCH 1075/1722] shellcheck --- .../queries/0_stateless/03149_numbers_max_block_size_zero.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh index 2c4669325bb..7f606d889a7 100755 --- a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh @@ -1,7 +1,9 @@ #!/usr/bin/env bash +# shellcheck disable=SC2266 CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | [ $(grep -c "Sanity check: 'max_block_size' cannot be 0. Set to default value") -gt 0 ] && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | + [ "$(grep -c "Sanity check: 'max_block_size' cannot be 0. Set to default value")" -gt 0 ] && echo "OK" || echo "FAIL" From d952f7cff579d28a51eea428aee7460121862ce5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 23:50:26 +0200 Subject: [PATCH 1076/1722] Update test --- tests/queries/0_stateless/01825_new_type_json_ghdata.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh index acb4925ce6e..ee702300094 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 -cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata FORMAT JSONAsObject" +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} --max_block_size 8192 --max_insert_block_size 8192 --max_insert_threads 1 --min_insert_block_size_bytes 0 --min_insert_block_size_rows 0 -q "INSERT INTO ghdata FORMAT JSONAsObject" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)" From b98249ea7fda526a7a561862fcc4a721e5a4587f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 00:06:47 +0200 Subject: [PATCH 1077/1722] Use temporary tables for input and output in clickhouse-local --- programs/local/LocalServer.cpp | 2 +- tests/queries/0_stateless/01191_rename_dictionary.sql | 1 + .../02141_clickhouse_local_interactive_table.reference | 4 ++-- .../0_stateless/02141_clickhouse_local_interactive_table.sh | 4 ++-- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 200beea7b63..a8b774562f9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -367,7 +367,7 @@ std::string LocalServer::getInitialCreateTableQuery() else table_structure = "(" + table_structure + ")"; - return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});", + return fmt::format("CREATE TEMPORARY TABLE {} {} ENGINE = File({}, {});", table_name, table_structure, data_format, table_file); } diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index c5012dabc81..be95e5a7d4b 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -27,6 +27,7 @@ RENAME DICTIONARY test_01191.t TO test_01191.dict1; -- {serverError INCORRECT_QU DROP DICTIONARY test_01191.t; -- {serverError INCORRECT_QUERY} DROP TABLE test_01191.t; +DROP DATABASE IF EXISTS dummy_db; CREATE DATABASE dummy_db ENGINE=Atomic; RENAME DICTIONARY test_01191.dict TO dummy_db.dict1; RENAME DICTIONARY dummy_db.dict1 TO test_01191.dict; diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference index 0bb8966cbe4..0e74c0a083e 100644 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference @@ -1,2 +1,2 @@ -CREATE TABLE default.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') -CREATE TABLE foo.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') +CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\') +CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\') diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh index 934d87616ac..3a95e59416a 100755 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh @@ -4,5 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' -$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' +$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table' +$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table' From 8ba142559ca05295670b0a899610ab613c2d5658 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 00:09:39 +0200 Subject: [PATCH 1078/1722] Pass-through RENAME and UUID-related operations in Overlay database to underlying databases --- src/Databases/DatabasesOverlay.cpp | 47 ++++++++++++++++++++++++++++++ src/Databases/DatabasesOverlay.h | 9 ++++++ src/Interpreters/StorageID.h | 1 - 3 files changed, 56 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 801356b3dd7..495733e15fd 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -14,6 +14,8 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int CANNOT_GET_CREATE_TABLE_QUERY; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TABLE; } DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_) @@ -124,6 +126,39 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab getEngineName()); } +void DatabasesOverlay::renameTable( + ContextPtr current_context, + const String & name, + IDatabase & to_database, + const String & to_name, + bool exchange, + bool dictionary) +{ + for (auto & db : databases) + { + if (db->isTableExist(name, current_context)) + { + if (DatabasesOverlay * to_overlay_database = typeid_cast(&to_database)) + { + /// Renaming from Overlay database inside itself or into another Overlay database. + /// Just use the first database in the overlay as a destination. + if (to_overlay_database->databases.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The destination Overlay database {} does not have any members", to_database.getDatabaseName()); + + db->renameTable(current_context, name, *to_overlay_database->databases[0], to_name, exchange, dictionary); + } + else + { + /// Renaming into a different type of database. E.g. from Overlay on top of Atomic database into just Atomic database. + db->renameTable(current_context, name, to_database, to_name, exchange, dictionary); + } + + return; + } + } + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(getDatabaseName()), backQuote(name)); +} + ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const { ASTPtr result = nullptr; @@ -178,6 +213,18 @@ String DatabasesOverlay::getTableDataPath(const ASTCreateQuery & query) const return result; } +UUID DatabasesOverlay::getUUID() const +{ + UUID result = UUIDHelpers::Nil; + for (const auto & db : databases) + { + result = db->getUUID(); + if (result != UUIDHelpers::Nil) + break; + } + return result; +} + UUID DatabasesOverlay::tryGetTableUUID(const String & table_name) const { UUID result = UUIDHelpers::Nil; diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index b0c7e7e4032..40c653e5cb5 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -35,12 +35,21 @@ public: StoragePtr detachTable(ContextPtr context, const String & table_name) override; + void renameTable( + ContextPtr current_context, + const String & name, + IDatabase & to_database, + const String & to_name, + bool exchange, + bool dictionary) override; + ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; + UUID getUUID() const override; UUID tryGetTableUUID(const String & table_name) const override; void drop(ContextPtr context) override; diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index f9afbc7b98d..ad55d16e284 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -27,7 +27,6 @@ class ASTQueryWithTableAndOutput; class ASTTableIdentifier; class Context; -// TODO(ilezhankin): refactor and merge |ASTTableIdentifier| struct StorageID { String database_name; From b29d5242be83294af8429d30fd560a83de3c08b4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 22:10:11 +0000 Subject: [PATCH 1079/1722] Fix tidy build --- .../AggregateFunctionDistinctDynamicTypes.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 6c899d6cda8..649d64a3904 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -87,7 +87,7 @@ public: if (dynamic_column.isNullAt(row_num)) return; - this->data(place).add(dynamic_column.getTypeNameAt(row_num)); + data(place).add(dynamic_column.getTypeNameAt(row_num)); } void ALWAYS_INLINE addBatchSinglePlace( @@ -100,7 +100,7 @@ public: /// In this case we can avoid iterating over all rows because we can get all types /// in Dynamic column in a more efficient way. else - assert_cast(*columns[0]).getAllTypeNames(this->data(place).data); + assert_cast(*columns[0]).getAllTypeNames(data(place).data); } void addManyDefaults( @@ -114,22 +114,22 @@ public: void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs)); + data(place).merge(data(rhs)); } void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - this->data(place).serialize(buf); + data(place).serialize(buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - this->data(place).deserialize(buf); + data(place).deserialize(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { - this->data(place).insertResultInto(to); + data(place).insertResultInto(to); } }; From a66db7abc2cced463e8ff9a04530c4647ddf8a22 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Aug 2024 00:05:18 +0200 Subject: [PATCH 1080/1722] Fix output of clickhouse-test in case of tests timeouts After https://github.com/ClickHouse/ClickHouse/pull/67737 the output will be broken, since in case of timeout it will print to stdout. Let's just capture it and add it to stderr. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3d7e0e922d..24149d46d14 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -30,6 +30,7 @@ import subprocess import sys import traceback import urllib.parse +import io # for crc32 import zlib @@ -39,6 +40,8 @@ from errno import ESRCH from subprocess import PIPE, Popen from time import sleep, time from typing import Dict, List, Optional, Set, Tuple, Union +from contextlib import redirect_stdout + try: import termcolor # type: ignore @@ -1324,9 +1327,13 @@ class TestCase: return None def process_result_impl(self, proc, total_time: float): + kill_output = "" if proc: if proc.returncode is None: - kill_process_group(os.getpgid(proc.pid)) + f = io.StringIO() + with redirect_stdout(f): + kill_process_group(os.getpgid(proc.pid)) + kill_output = f.getvalue() description = "" @@ -1342,7 +1349,7 @@ class TestCase: with open(self.stdout_file, "rb") as stdfd: stdout = str(stdfd.read(), errors="replace", encoding="utf-8") - stderr = "" + stderr = kill_output if os.path.exists(self.stderr_file): with open(self.stderr_file, "rb") as stdfd: stderr += str(stdfd.read(), errors="replace", encoding="utf-8") From da0a8051d8c8e8c2c72145e15cdf5a96e99641d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 00:22:57 +0200 Subject: [PATCH 1081/1722] Miscellaneous changes in database engines --- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 10 +++++----- src/Databases/DatabaseOnDisk.h | 4 ++-- src/Databases/DatabaseOrdinary.cpp | 4 ++-- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 3fb6d30fcb8..2ccdd8510a8 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -52,7 +52,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/) { - iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) + iterateMetadataFiles([this, &local_context](const String & file_name) { const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 41cfb751141..aeac130594f 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -12,7 +12,7 @@ class DatabaseLazyIterator; class Context; /** Lazy engine of databases. - * Works like DatabaseOrdinary, but stores in memory only the cache. + * Works like DatabaseOrdinary, but stores only recently accessed tables in memory. * Can be used only with *Log engines. */ class DatabaseLazy final : public DatabaseOnDisk diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 734f354d9a5..c80e4def94e 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -568,14 +568,14 @@ void DatabaseOnDisk::drop(ContextPtr local_context) assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty()); if (local_context->getSettingsRef().force_remove_data_recursively_on_drop) { - (void)fs::remove_all(local_context->getPath() + getDataPath()); + (void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path); (void)fs::remove_all(getMetadataPath()); } else { try { - (void)fs::remove(local_context->getPath() + getDataPath()); + (void)fs::remove(std::filesystem::path(getContext()->getPath()) / data_path); (void)fs::remove(getMetadataPath()); } catch (const fs::filesystem_error & e) @@ -613,7 +613,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n } } -void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const +void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const { auto process_tmp_drop_metadata_file = [&](const String & file_name) { @@ -621,7 +621,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); - if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name)) + if (fs::exists(std::filesystem::path(getContext()->getPath()) / data_path / object_name)) { fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name)); @@ -638,7 +638,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat std::vector> metadata_files; fs::directory_iterator dir_end; - for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it(metadata_path); dir_it != dir_end; ++dir_it) { String file_name = dir_it->path().filename(); /// For '.svn', '.gitignore' directory and similar. diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 12656068643..ffc95a7c128 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -64,7 +64,7 @@ public: time_t getObjectMetadataModificationTime(const String & object_name) const override; String getDataPath() const override { return data_path; } - String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; } + String getTableDataPath(const String & table_name) const override { return std::filesystem::path(data_path) / escapeForFileName(table_name) / ""; } String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); } String getMetadataPath() const override { return metadata_path; } @@ -83,7 +83,7 @@ protected: using IteratingFunction = std::function; - void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const; + void iterateMetadataFiles(const IteratingFunction & process_metadata_file) const; ASTPtr getCreateTableQueryImpl( const String & table_name, diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 8808261654f..dd8a3f42ea8 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -55,7 +55,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; static constexpr const char * const CONVERT_TO_REPLICATED_FLAG_NAME = "convert_to_replicated"; DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_) - : DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) + : DatabaseOrdinary(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseOrdinary (" + name_ + ")", context_) { } @@ -265,7 +265,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables } }; - iterateMetadataFiles(local_context, process_metadata); + iterateMetadataFiles(process_metadata); size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count; size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries; From 07fa798ffa53216fd37dac51bceb327665fc8dda Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 16 Aug 2024 23:31:41 +0000 Subject: [PATCH 1082/1722] add total in system.one, test --- src/Storages/System/StorageSystemOne.cpp | 5 ++++- .../System/StorageSystemViewRefreshes.cpp | 2 +- ...3221_refreshable_matview_progress.reference | 2 ++ .../03221_refreshable_matview_progress.sql | 18 ++++++++++++++++++ 4 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03221_refreshable_matview_progress.reference create mode 100644 tests/queries/0_stateless/03221_refreshable_matview_progress.sql diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 936d55e61a0..70377715dc3 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -41,7 +41,10 @@ Pipe StorageSystemOne::read( auto column = DataTypeUInt8().createColumnConst(1, 0u)->convertToFullColumnIfConst(); Chunk chunk({ std::move(column) }, 1); - return Pipe(std::make_shared(std::move(header), std::move(chunk))); + auto source = std::make_shared(std::move(header), std::move(chunk)); + source->addTotalRowsApprox(1); + + return Pipe(source); } diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 3941c4c39c2..30539ed6b6a 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -86,7 +86,7 @@ void StorageSystemViewRefreshes::fillData( res_columns[i++]->insert(refresh.exception_message); res_columns[i++]->insert(refresh.refresh_count); - res_columns[i++]->insert(std::min(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read, 1.0)); + res_columns[i++]->insert(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read); res_columns[i++]->insert(refresh.progress.elapsed_ns / 1e9); res_columns[i++]->insert(refresh.progress.read_rows); res_columns[i++]->insert(refresh.progress.read_bytes); diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.reference b/tests/queries/0_stateless/03221_refreshable_matview_progress.reference new file mode 100644 index 00000000000..5ed392e61c7 --- /dev/null +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.reference @@ -0,0 +1,2 @@ +0 +4 4 1 diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql new file mode 100644 index 00000000000..4794359fd2b --- /dev/null +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql @@ -0,0 +1,18 @@ +set allow_experimental_refreshable_materialized_view=1; + +CREATE MATERIALIZED VIEW 03221_rmv +REFRESH AFTER 1 SECOND +( +x UInt64 +) +ENGINE = Memory +AS SELECT number AS x +FROM numbers(3) +UNION ALL +SELECT rand64() AS x; + +SELECT sleep(2); + +SELECT read_rows, total_rows, progress FROM system.view_refreshes WHERE view = '03221_rmv'; + +DROP TABLE 03221_rmv; From 142d7b15828c9eeef145fd56d56b33d1004fb68c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 01:35:49 +0200 Subject: [PATCH 1083/1722] Miscellaneous changes in BaseDaemon --- src/Daemon/BaseDaemon.cpp | 20 ++------------------ 1 file changed, 2 insertions(+), 18 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index e7ae8ea5a1d..f75699881bd 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -16,39 +16,29 @@ #include #if defined(OS_LINUX) - #include +#include #endif #include #include #include - #include #include #include -#include #include -#include #include #include #include #include #include - #include #include #include -#include #include -#include #include -#include #include -#include #include -#include -#include #include #include #include @@ -459,13 +449,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() signal_listener_thread.start(*signal_listener); #if defined(__ELF__) && !defined(OS_FREEBSD) - String build_id_hex = SymbolIndex::instance().getBuildIDHex(); - if (build_id_hex.empty()) - build_id = ""; - else - build_id = build_id_hex; -#else - build_id = ""; + build_id = SymbolIndex::instance().getBuildIDHex(); #endif git_hash = GIT_HASH; From 83608cb7bfcbf0801994cc916ed36b91c7113307 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 01:40:07 +0200 Subject: [PATCH 1084/1722] Miscellaneous changes from #66999 (2) --- base/base/CMakeLists.txt | 3 +++ {src/Daemon => base/base}/GitHash.cpp.in | 0 src/Daemon/CMakeLists.txt | 3 --- 3 files changed, 3 insertions(+), 3 deletions(-) rename {src/Daemon => base/base}/GitHash.cpp.in (100%) diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 247028b96e0..3d236f52c36 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -8,6 +8,8 @@ endif () # when instantiated from JSON.cpp. Try again when libcxx(abi) and Clang are upgraded to 16. set (CMAKE_CXX_STANDARD 20) +configure_file(GitHash.cpp.in GitHash.generated.cpp) + set (SRCS argsToConfig.cpp cgroupsv2.cpp @@ -33,6 +35,7 @@ set (SRCS safeExit.cpp throwError.cpp Numa.cpp + GitHash.generated.cpp ) add_library (common ${SRCS}) diff --git a/src/Daemon/GitHash.cpp.in b/base/base/GitHash.cpp.in similarity index 100% rename from src/Daemon/GitHash.cpp.in rename to base/base/GitHash.cpp.in diff --git a/src/Daemon/CMakeLists.txt b/src/Daemon/CMakeLists.txt index 35ea2122dbb..2068af2200d 100644 --- a/src/Daemon/CMakeLists.txt +++ b/src/Daemon/CMakeLists.txt @@ -1,10 +1,7 @@ -configure_file(GitHash.cpp.in GitHash.generated.cpp) - add_library (daemon BaseDaemon.cpp GraphiteWriter.cpp SentryWriter.cpp - GitHash.generated.cpp ) target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_parsers clickhouse_common_io clickhouse_common_config) From 4b68ba23c0372331401cd327ca5849ba9d1bce8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 03:43:53 +0200 Subject: [PATCH 1085/1722] Pretty print tuples in CREATE TABLE statements --- programs/format/Format.cpp | 6 +++++- src/Client/ClientBase.cpp | 6 +++++- src/Core/ExternalTable.cpp | 4 ++-- src/Core/Settings.h | 2 +- src/DataTypes/IDataType.cpp | 1 - src/Databases/DatabaseOnDisk.cpp | 2 +- src/Functions/formatQuery.cpp | 8 +++++++- .../InterpreterShowCreateQuery.cpp | 5 ++++- .../formatWithPossiblyHidingSecrets.h | 3 ++- src/Parsers/ASTColumnDeclaration.cpp | 8 ++------ src/Parsers/ASTDataType.cpp | 19 ++++++++++++++----- src/Parsers/ASTExpressionList.cpp | 5 ++--- src/Parsers/ASTNameTypePair.cpp | 6 +----- src/Parsers/IAST.cpp | 3 ++- src/Parsers/IAST.h | 14 +++++++++----- 15 files changed, 57 insertions(+), 35 deletions(-) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index a434c9171e9..4af77533c53 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -264,7 +264,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv) if (!backslash) { WriteBufferFromOwnString str_buf; - formatAST(*res, str_buf, hilite, oneline || approx_query_length < max_line_length); + oneline = oneline || approx_query_length < max_line_length; + IAST::FormatSettings settings(str_buf, oneline, hilite); + settings.show_secrets = true; + settings.print_pretty_type_names = !oneline; + res->format(settings); if (insert_query_payload) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 187ef079eda..74357d33f1c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -331,7 +331,11 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting { output_stream << std::endl; WriteBufferFromOStream res_buf(output_stream, 4096); - formatAST(*res, res_buf); + IAST::FormatSettings format_settings(res_buf, /* one_line */ false); + format_settings.hilite = true; + format_settings.show_secrets = true; + format_settings.print_pretty_type_names = true; + res->format(format_settings); res_buf.finalize(); output_stream << std::endl << std::endl; } diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index c2bcf6ec651..4ff0d7092d8 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -85,7 +85,7 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg /// We use `formatWithPossiblyHidingSensitiveData` instead of `getColumnNameWithoutAlias` because `column->type` is an ASTFunction. /// `getColumnNameWithoutAlias` will return name of the function with `(arguments)` even if arguments is empty. if (column) - structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true)); + structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -102,7 +102,7 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); for (size_t i = 0; i < type_list_raw->children.size(); ++i) - structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true)); + structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true, false)); } void BaseExternalTable::initSampleBlock() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d84ad9022a..1a71494c8cd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -882,7 +882,7 @@ class IColumn; M(Bool, use_json_alias_for_old_object_type, false, "When enabled, JSON type alias will create old experimental Object type instead of a new JSON type", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ - M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ + M(Bool, print_pretty_type_names, true, "Print pretty type names in the DESCRIBE query and `toTypeName` function", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0) \ M(SQLSecurityType, default_normal_view_sql_security, SQLSecurityType::INVOKER, "Allows to set a default value for SQL SECURITY option when creating a normal view.", 0) \ diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 7fd8a85aeca..49e5b2d022e 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -8,7 +8,6 @@ #include #include -#include #include #include diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 734f354d9a5..4e1ddd8cc77 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -504,7 +504,7 @@ void DatabaseOnDisk::renameTable( } -/// It returns create table statement (even if table is detached) +/// It returns the create table statement (even if table is detached) ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const { ASTPtr ast; diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index 9591ea95254..be633bdfe37 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -43,6 +43,7 @@ public: max_query_size = settings.max_query_size; max_parser_depth = settings.max_parser_depth; max_parser_backtracks = settings.max_parser_backtracks; + print_pretty_type_names = settings.print_pretty_type_names; } String getName() const override { return name; } @@ -138,7 +139,11 @@ private: } } - formatAST(*ast, buf, /*hilite*/ false, /*single_line*/ output_formatting == OutputFormatting::SingleLine); + IAST::FormatSettings settings(buf, output_formatting == OutputFormatting::SingleLine, /*hilite*/ false); + settings.show_secrets = true; + settings.print_pretty_type_names = print_pretty_type_names; + ast->format(settings); + auto formatted = buf.stringView(); const size_t res_data_new_size = res_data_size + formatted.size() + 1; @@ -165,6 +170,7 @@ private: size_t max_query_size; size_t max_parser_depth; size_t max_parser_backtracks; + bool print_pretty_type_names; }; } diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index e5549b2e539..7af6b4948f8 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -97,7 +97,10 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() } MutableColumnPtr column = ColumnString::create(); - column->insert(format({.ctx = getContext(), .query = *create_query, .one_line = false})); + column->insert(format({ + .ctx = getContext(), + .query = *create_query, + .one_line = false})); return QueryPipeline(std::make_shared(Block{{ std::move(column), diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index ea8c295b169..14e84f1d1a4 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -25,7 +25,8 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getSettingsRef().format_display_secrets_in_show_and_select && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); - return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets); + return settings.query.formatWithPossiblyHidingSensitiveData( + settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names); } } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index c96499095d5..23d653012f8 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -66,17 +66,13 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo { frame.need_parens = false; - /// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query. + /// We have to always backquote column names to avoid ambiguity with INDEX and other declarations in CREATE query. format_settings.ostr << backQuote(name); if (type) { format_settings.ostr << ' '; - - FormatStateStacked type_frame = frame; - type_frame.indent = 0; - - type->formatImpl(format_settings, state, type_frame); + type->formatImpl(format_settings, state, frame); } if (null_modifier) diff --git a/src/Parsers/ASTDataType.cpp b/src/Parsers/ASTDataType.cpp index 3c17ae8c380..21f56e5f7a2 100644 --- a/src/Parsers/ASTDataType.cpp +++ b/src/Parsers/ASTDataType.cpp @@ -40,12 +40,21 @@ void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & stat { settings.ostr << '(' << (settings.hilite ? hilite_none : ""); - for (size_t i = 0, size = arguments->children.size(); i < size; ++i) + if (!settings.one_line && settings.print_pretty_type_names && name == "Tuple") { - if (i != 0) - settings.ostr << ", "; - - arguments->children[i]->formatImpl(settings, state, frame); + ++frame.indent; + std::string indent_str = settings.one_line ? "" : "\n" + std::string(4 * frame.indent, ' '); + for (size_t i = 0, size = arguments->children.size(); i < size; ++i) + { + if (i != 0) + settings.ostr << ','; + settings.ostr << indent_str; + arguments->children[i]->formatImpl(settings, state, frame); + } + } + else + { + arguments->formatImpl(settings, state, frame); } settings.ostr << (settings.hilite ? hilite_function : "") << ')'; diff --git a/src/Parsers/ASTExpressionList.cpp b/src/Parsers/ASTExpressionList.cpp index 61ac482af82..f345b0c6a6f 100644 --- a/src/Parsers/ASTExpressionList.cpp +++ b/src/Parsers/ASTExpressionList.cpp @@ -42,7 +42,8 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - std::string indent_str = "\n" + std::string(4 * (frame.indent + 1), ' '); + ++frame.indent; + std::string indent_str = "\n" + std::string(4 * frame.indent, ' '); if (frame.expression_list_prepend_whitespace) { @@ -50,8 +51,6 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For settings.ostr << ' '; } - ++frame.indent; - for (size_t i = 0, size = children.size(); i < size; ++i) { if (i && separator) diff --git a/src/Parsers/ASTNameTypePair.cpp b/src/Parsers/ASTNameTypePair.cpp index e4066081a9b..1515700365f 100644 --- a/src/Parsers/ASTNameTypePair.cpp +++ b/src/Parsers/ASTNameTypePair.cpp @@ -23,12 +23,8 @@ ASTPtr ASTNameTypePair::clone() const void ASTNameTypePair::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - - settings.ostr << indent_str << backQuoteIfNeed(name) << ' '; + settings.ostr << backQuoteIfNeed(name) << ' '; type->formatImpl(settings, state, frame); } } - - diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 37d7f458d61..5bd2c92c60a 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,11 +165,12 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const +String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const { WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; + settings.print_pretty_type_names = print_pretty_type_names; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index e2cf7579667..2293d50b0ec 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -201,6 +201,7 @@ public: bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. LiteralEscapingStyle literal_escaping_style; + bool print_pretty_type_names; explicit FormatSettings( WriteBuffer & ostr_, @@ -209,7 +210,8 @@ public: bool always_quote_identifiers_ = false, IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, bool show_secrets_ = true, - LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular) + LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular, + bool print_pretty_type_names_ = false) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -218,6 +220,7 @@ public: , show_secrets(show_secrets_) , nl_or_ws(one_line ? ' ' : '\n') , literal_escaping_style(literal_escaping_style_) + , print_pretty_type_names(print_pretty_type_names_) { } @@ -230,6 +233,7 @@ public: , show_secrets(other.show_secrets) , nl_or_ws(other.nl_or_ws) , literal_escaping_style(other.literal_escaping_style) + , print_pretty_type_names(other.print_pretty_type_names) { } @@ -251,7 +255,7 @@ public: /// The state that is copied when each node is formatted. For example, nesting level. struct FormatStateStacked { - UInt8 indent = 0; + UInt16 indent = 0; bool need_parens = false; bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element. bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required) @@ -274,7 +278,7 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const; + String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about @@ -283,12 +287,12 @@ public: */ String formatForLogging(size_t max_length = 0) const { - return formatWithPossiblyHidingSensitiveData(max_length, true, false); + return formatWithPossiblyHidingSensitiveData(max_length, true, false, false); } String formatForErrorMessage() const { - return formatWithPossiblyHidingSensitiveData(0, true, false); + return formatWithPossiblyHidingSensitiveData(0, true, false, false); } virtual bool hasSecretParts() const { return childrenHaveSecretParts(); } From 566e043c2c84a979ca1c05996f6e3a4303708bff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 03:44:36 +0200 Subject: [PATCH 1086/1722] Add a test --- ...print_pretty_tuples_create_query.reference | 56 +++++++++++++++++++ .../03227_print_pretty_tuples_create_query.sh | 35 ++++++++++++ 2 files changed, 91 insertions(+) create mode 100644 tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference create mode 100755 tests/queries/0_stateless/03227_print_pretty_tuples_create_query.sh diff --git a/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference b/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference new file mode 100644 index 00000000000..c65dc32a224 --- /dev/null +++ b/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference @@ -0,0 +1,56 @@ + +SHOW CREATE TABLE: +CREATE TABLE test.test +( + `x` Tuple( + a String, + b Array(Tuple( + c Tuple( + e String), + d String))), + `y` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 8192 +CREATE TABLE test.test +( + `x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), + `y` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 8192 + +clickhouse-format: +CREATE TABLE test +( + `x` Tuple( + a String, + b Array(Tuple( + c Tuple( + e String), + d String))), + `y` String +) +ORDER BY tuple() +CREATE TABLE test (`x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), `y` String) ORDER BY tuple() + +formatQuery: +CREATE TABLE test +( + `x` Tuple( + a String, + b Array(Tuple( + c Tuple( + e String), + d String))), + `y` String +) +ORDER BY tuple() +CREATE TABLE test +( + `x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), + `y` String +) +ORDER BY tuple() diff --git a/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.sh b/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.sh new file mode 100755 index 00000000000..e5614f9f228 --- /dev/null +++ b/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-asan, no-msan, no-tsan +# ^ requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo +echo "SHOW CREATE TABLE:" +${CLICKHOUSE_CLIENT} --output-format Raw --query " + DROP TABLE IF EXISTS test; + CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY (); + SET print_pretty_type_names = 1; + SHOW CREATE TABLE test; + SET print_pretty_type_names = 0; + SHOW CREATE TABLE test; + DROP TABLE test; +" + +echo +echo "clickhouse-format:" +${CLICKHOUSE_FORMAT} --query " + CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY () +" +${CLICKHOUSE_FORMAT} --oneline --query " + CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY () +" + +echo +echo "formatQuery:" +${CLICKHOUSE_CLIENT} --output-format Raw --query " + SELECT formatQuery('CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ()') SETTINGS print_pretty_type_names = 1; + SELECT formatQuery('CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ()') SETTINGS print_pretty_type_names = 0; +" From 3065b386a4e033a802bbc4855d5113e814648848 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 03:49:45 +0200 Subject: [PATCH 1087/1722] Update documentation --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1a71494c8cd..d8837d26e54 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -882,7 +882,7 @@ class IColumn; M(Bool, use_json_alias_for_old_object_type, false, "When enabled, JSON type alias will create old experimental Object type instead of a new JSON type", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ - M(Bool, print_pretty_type_names, true, "Print pretty type names in the DESCRIBE query and `toTypeName` function", 0) \ + M(Bool, print_pretty_type_names, true, "Print pretty type names in the DESCRIBE query and `toTypeName` function, as well as in the `SHOW CREATE TABLE` query and the `formatQuery` function.", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0) \ M(SQLSecurityType, default_normal_view_sql_security, SQLSecurityType::INVOKER, "Allows to set a default value for SQL SECURITY option when creating a normal view.", 0) \ From 02ec4e2f92ac769f92aebdb714d0d8da1a924984 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 04:00:31 +0200 Subject: [PATCH 1088/1722] Fix build --- src/Parsers/ExpressionElementParsers.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 726326bfc85..61b5723072e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -943,7 +943,6 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (ParserToken(DoubleColon).ignore(pos, expected) && ParserDataType().parse(pos, type_ast, expected)) { - String s; size_t data_size = data_end - data_begin; if (string_literal) { From 7a5df67b3b3b5bd7a8481562e0293150427fef90 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Aug 2024 05:08:58 +0200 Subject: [PATCH 1089/1722] Fix style --- src/Interpreters/InterpreterShowCreateQuery.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 7af6b4948f8..3de6b755609 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -97,10 +97,12 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() } MutableColumnPtr column = ColumnString::create(); - column->insert(format({ + column->insert(format( + { .ctx = getContext(), .query = *create_query, - .one_line = false})); + .one_line = false + })); return QueryPipeline(std::make_shared(Block{{ std::move(column), From 330086c621c860524b68ce7598d12c8db958101d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 17 Aug 2024 03:46:33 +0000 Subject: [PATCH 1090/1722] update 02136_scalar_progress results according to fixed bug --- tests/queries/0_stateless/02136_scalar_progress.reference | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02136_scalar_progress.reference b/tests/queries/0_stateless/02136_scalar_progress.reference index 5378c52de89..b8957f78e6d 100644 --- a/tests/queries/0_stateless/02136_scalar_progress.reference +++ b/tests/queries/0_stateless/02136_scalar_progress.reference @@ -1,6 +1,7 @@ < X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"100000","read_bytes":"800000","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} -< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} -< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"} -< X-ClickHouse-Summary: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"} +< X-ClickHouse-Progress: {"read_rows":"100000","read_bytes":"800000","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"0","result_bytes":"0"} +< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"0","result_bytes":"0"} +< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"1","result_bytes":"272"} +< X-ClickHouse-Summary: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"1","result_bytes":"272"} From 3adfea86539389fd690407b40c2708ab4e19d71a Mon Sep 17 00:00:00 2001 From: Denis Hananein Date: Sat, 17 Aug 2024 06:06:04 +0200 Subject: [PATCH 1091/1722] Fix #68239 SAMPLE n --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 59f3a299c99..3b148a4971d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -92,11 +92,11 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( /// We will find out how many rows we would have read without sampling. LOG_DEBUG(log, "Preliminary index scan with condition: {}", key_condition.toString()); + MarkRanges exact_ranges; for (const auto & part : parts) { - MarkRanges exact_ranges; - markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log); - for (const auto & range : exact_ranges) + MarkRanges part_ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log); + for (const auto & range : part_ranges) rows_count += part->index_granularity.getRowsCountInRange(range); } From 41f6e06cbe07f6bd1ce3bdb48154f1f2745b3cc2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 17 Aug 2024 13:08:30 +0200 Subject: [PATCH 1092/1722] Update 03227_distinct_dynamic_types_json_paths.sql --- .../0_stateless/03227_distinct_dynamic_types_json_paths.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index f7707b536e6..c9c86277bce 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -1,3 +1,5 @@ +-- Tags: long + set allow_experimental_dynamic_type = 1; set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; From 00891a2dd8f69dde0b1fe364b6891fd3629d5dbe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 17 Aug 2024 13:57:24 +0000 Subject: [PATCH 1093/1722] fix test --- .../queries/0_stateless/03221_refreshable_matview_progress.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql index 4794359fd2b..30228277bb5 100644 --- a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql @@ -1,3 +1,5 @@ +-- Tags: no-ordinary-database + set allow_experimental_refreshable_materialized_view=1; CREATE MATERIALIZED VIEW 03221_rmv From 9dee9ecfb4adee4cff099f13afe61bdc2c38170e Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sat, 17 Aug 2024 10:33:35 -0600 Subject: [PATCH 1094/1722] Fix incorrect default value for postgresql_connection_pool_auto_close_connection in docs --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index de601fe02dc..5bf1fe197ae 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1381,7 +1381,7 @@ Default value: `2`. Close connection before returning connection to the pool. -Default value: true. +Default value: false. ## odbc_bridge_connection_pool_size {#odbc-bridge-connection-pool-size} From a42b12725ab37df74a96e85f7c644c90bd6e30f6 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 16 Aug 2024 17:39:09 +0200 Subject: [PATCH 1095/1722] CI: Native build for package_aarch64 --- tests/ci/ci_config.py | 3 ++- tests/ci/ci_definitions.py | 1 + tests/ci/test_ci_config.py | 14 ++++++++++---- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7a19eb6f827..173c6c9c931 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -94,7 +94,8 @@ class CI: package_type="deb", static_binary_name="aarch64", additional_pkgs=True, - ) + ), + runner_type=Runners.BUILDER_ARM, ), BuildNames.PACKAGE_ASAN: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 48847b0d7a6..1bed9db06f2 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -57,6 +57,7 @@ class Runners(metaclass=WithIter): """ BUILDER = "builder" + BUILDER_ARM = "builder-aarch64" STYLE_CHECKER = "style-checker" STYLE_CHECKER_ARM = "style-checker-aarch64" FUNC_TESTER = "func-tester" diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 525b3bf367b..c3e55aeac06 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -35,10 +35,16 @@ class TestCIConfig(unittest.TestCase): f"Job [{job}] must have style-checker(-aarch64) runner", ) elif "binary_" in job.lower() or "package_" in job.lower(): - self.assertTrue( - CI.JOB_CONFIGS[job].runner_type == CI.Runners.BUILDER, - f"Job [{job}] must have [{CI.Runners.BUILDER}] runner", - ) + if job.lower() == CI.BuildNames.PACKAGE_AARCH64: + self.assertTrue( + CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_ARM,), + f"Job [{job}] must have [{CI.Runners.BUILDER_ARM}] runner", + ) + else: + self.assertTrue( + CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER,), + f"Job [{job}] must have [{CI.Runners.BUILDER}] runner", + ) elif "aarch64" in job.lower(): self.assertTrue( "aarch" in CI.JOB_CONFIGS[job].runner_type, From 7432400fd0c07b7c967f47b1536706b8f791fcb1 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 16 Aug 2024 21:06:58 +0200 Subject: [PATCH 1096/1722] revert hacks made to prevent OOM in aarch64 --- cmake/limit_jobs.cmake | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 17d8dd42a2c..8e48fc9b9d8 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -42,19 +42,9 @@ endif () # But use 2 parallel jobs, since: # - this is what llvm does # - and I've verfied that lld-11 does not use all available CPU time (in peak) while linking one binary -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO) - if (ARCH_AARCH64) - # aarch64 builds start to often fail with OOMs (reason not yet clear), for now let's limit the concurrency - message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 1.") - set (PARALLEL_LINK_JOBS 1) - if (LINKER_NAME MATCHES "lld") - math(EXPR LTO_JOBS ${NUMBER_OF_LOGICAL_CORES}/4) - set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -Wl,--thinlto-jobs=${LTO_JOBS}") - endif() - elseif (PARALLEL_LINK_JOBS GREATER 2) - message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.") - set (PARALLEL_LINK_JOBS 2) - endif () +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLEL_LINK_JOBS GREATER 2) + message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.") + set (PARALLEL_LINK_JOBS 2) endif() message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB RAM, 'OFF' means the native core count).") From cb8d9a05643d3aac5f410c4eac53124224c63bc8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 17 Aug 2024 20:13:35 +0200 Subject: [PATCH 1097/1722] fix typo --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index cd6fd9ab839..a03394be226 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -4952,7 +4952,7 @@ toIntervalMonth(n) **Arguments** -- `n` — Number of m. Positive integer number. [Int*](../data-types/int-uint.md). +- `n` — Number of months. Positive integer number. [Int*](../data-types/int-uint.md). **Returned values** From ae389d14ee65ff5fea3543868b6b161f9fcb806e Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 17 Aug 2024 20:42:00 +0200 Subject: [PATCH 1098/1722] Fix stylecheck --- src/Formats/SchemaInferenceUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 54352b88578..e8eab3b4453 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -333,7 +333,7 @@ namespace type = variant_type; type_indexes = {TypeIndex::Variant}; } - + /// If we have only date/datetimes types (Date/DateTime/DateTime64), convert all of them to the common type, /// otherwise, convert all Date, DateTime and DateTime64 to String. void transformDatesAndDateTimes(DataTypes & data_types, TypeIndexesSet & type_indexes) From a8a31ed137df53ccad60a16e09f63edf82a7b9cd Mon Sep 17 00:00:00 2001 From: Denis Hananein Date: Sat, 17 Aug 2024 21:14:25 +0200 Subject: [PATCH 1099/1722] Add unused --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3b148a4971d..bf12b3966d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -99,6 +99,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( for (const auto & range : part_ranges) rows_count += part->index_granularity.getRowsCountInRange(range); } + UNUSED(exact_ranges); return rows_count; } From c68b597eee752d5921d4469af01104a27681296a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 17 Aug 2024 20:03:18 +0000 Subject: [PATCH 1100/1722] fix test --- .../queries/0_stateless/03221_refreshable_matview_progress.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql index 30228277bb5..1be276c485c 100644 --- a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql @@ -15,6 +15,6 @@ SELECT rand64() AS x; SELECT sleep(2); -SELECT read_rows, total_rows, progress FROM system.view_refreshes WHERE view = '03221_rmv'; +SELECT read_rows, total_rows, progress FROM system.view_refreshes WHERE database = currentDatabase() and view = '03221_rmv'; DROP TABLE 03221_rmv; From a36688b07387822cae18eca1fbc798a517119e02 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 03:02:20 +0200 Subject: [PATCH 1101/1722] Fix error --- src/Parsers/ASTDataType.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ASTDataType.cpp b/src/Parsers/ASTDataType.cpp index 21f56e5f7a2..4211347fb74 100644 --- a/src/Parsers/ASTDataType.cpp +++ b/src/Parsers/ASTDataType.cpp @@ -54,6 +54,7 @@ void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & stat } else { + frame.expression_list_prepend_whitespace = false; arguments->formatImpl(settings, state, frame); } From 3c021e02b69f9237d9765d6295fd843ad2503398 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 03:02:26 +0200 Subject: [PATCH 1102/1722] Fix tests --- .../0_stateless/01458_named_tuple_millin.reference | 8 ++++++-- .../01504_compression_multiple_streams.reference | 8 ++++---- ...01548_create_table_compound_column_format.reference | 3 ++- .../01881_aggregate_functions_versioning.reference | 2 +- .../02117_show_create_table_system.reference | 10 ++++++++-- .../02286_tuple_numeric_identifier.reference | 2 +- .../02907_backup_restore_flatten_nested.reference | 4 ++-- 7 files changed, 24 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.reference b/tests/queries/0_stateless/01458_named_tuple_millin.reference index 954dfe36563..86561570985 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.reference +++ b/tests/queries/0_stateless/01458_named_tuple_millin.reference @@ -1,12 +1,16 @@ CREATE TABLE default.tuple ( - `j` Tuple(a Int8, b String) + `j` Tuple( + a Int8, + b String) ) ENGINE = Memory j Tuple(\n a Int8,\n b String) CREATE TABLE default.tuple ( - `j` Tuple(a Int8, b String) + `j` Tuple( + a Int8, + b String) ) ENGINE = Memory j Tuple(\n a Int8,\n b String) diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference index 4d3aba66526..14cdce72044 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.reference +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -1,20 +1,20 @@ 1 1 [[1]] (1,[1]) 1 1 [[1]] (1,[1]) -CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) -CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 3 3 [[3]] (3,[3]) 1 1 [[1]] (1,[1]) 1 1 [[1]] (1,[1]) -CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) -CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 3 3 [[3]] (3,[3]) diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.reference b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference index 21e31e8f034..c23cc57548b 100644 --- a/tests/queries/0_stateless/01548_create_table_compound_column_format.reference +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference @@ -7,6 +7,7 @@ ENGINE = TinyLog CREATE TABLE test ( `a` Int64, - `b` Tuple(a Int64) + `b` Tuple( + a Int64) ) ENGINE = TinyLog diff --git a/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference b/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference index c30c4ca7e74..e15f312c2c8 100644 --- a/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference +++ b/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference @@ -1 +1 @@ -CREATE TABLE default.test_table\n(\n `col1` DateTime,\n `col2` Int64,\n `col3` AggregateFunction(1, sumMap, Tuple(Array(UInt8), Array(UInt8)))\n)\nENGINE = AggregatingMergeTree\nORDER BY (col1, col2)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test_table\n(\n `col1` DateTime,\n `col2` Int64,\n `col3` AggregateFunction(1, sumMap, Tuple(\n Array(UInt8),\n Array(UInt8)))\n)\nENGINE = AggregatingMergeTree\nORDER BY (col1, col2)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 32e8b2f4312..638a46a142f 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -510,9 +510,15 @@ CREATE TABLE system.parts `rows_where_ttl_info.max` Array(DateTime), `projections` Array(String), `visible` UInt8, - `creation_tid` Tuple(UInt64, UInt64, UUID), + `creation_tid` Tuple( + UInt64, + UInt64, + UUID), `removal_tid_lock` UInt64, - `removal_tid` Tuple(UInt64, UInt64, UUID), + `removal_tid` Tuple( + UInt64, + UInt64, + UUID), `creation_csn` UInt64, `removal_csn` UInt64, `has_lightweight_delete` UInt8, diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference index 21348493d1d..916cdaf83cd 100644 --- a/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(`1` Tuple(`2` Int32, `3` Int32), `4` Int32)\n)\nENGINE = Memory +CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(\n `1` Tuple(\n `2` Int32,\n `3` Int32),\n `4` Int32)\n)\nENGINE = Memory {"t":{"1":{"2":2,"3":3},"4":4}} 2 3 4 2 3 4 diff --git a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference index aa8f22f590a..0db19f0591a 100644 --- a/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference +++ b/tests/queries/0_stateless/02907_backup_restore_flatten_nested.reference @@ -1,8 +1,8 @@ BACKUP_CREATED -CREATE TABLE default.test\n(\n `test` Array(Tuple(foo String, bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test\n(\n `test` Array(Tuple(\n foo String,\n bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 BACKUP_CREATED CREATE TABLE default.test2\n(\n `test` Nested(foo String, bar Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 RESTORED -CREATE TABLE default.test\n(\n `test` Array(Tuple(foo String, bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.test\n(\n `test` Array(Tuple(\n foo String,\n bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 RESTORED CREATE TABLE default.test2\n(\n `test` Nested(foo String, bar Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 From 60dd7e962a19e92cef4e3ab40d2607b3f5a59e90 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 05:56:27 +0200 Subject: [PATCH 1103/1722] Fix tests --- programs/format/Format.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 4af77533c53..f07387bd395 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -264,10 +264,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv) if (!backslash) { WriteBufferFromOwnString str_buf; - oneline = oneline || approx_query_length < max_line_length; - IAST::FormatSettings settings(str_buf, oneline, hilite); + bool oneline_current_query = oneline || approx_query_length < max_line_length; + IAST::FormatSettings settings(str_buf, oneline_current_query, hilite); settings.show_secrets = true; - settings.print_pretty_type_names = !oneline; + settings.print_pretty_type_names = !oneline_current_query; res->format(settings); if (insert_query_payload) @@ -311,7 +311,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv) else { WriteBufferFromOwnString str_buf; - formatAST(*res, str_buf, hilite, oneline); + bool oneline_current_query = oneline || approx_query_length < max_line_length; + IAST::FormatSettings settings(str_buf, oneline_current_query, hilite); + settings.show_secrets = true; + settings.print_pretty_type_names = !oneline_current_query; + res->format(settings); auto res_string = str_buf.str(); WriteBufferFromOStream res_cout(std::cout, 4096); From 1691e4c4977dbaf44bedf61cdf248e7a1997d407 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 06:14:21 +0200 Subject: [PATCH 1104/1722] Fix test --- tests/queries/0_stateless/01825_new_type_json_ghdata.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh index ee702300094..fbd7d897fb8 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh @@ -8,7 +8,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 -cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} --max_block_size 8192 --max_insert_block_size 8192 --max_insert_threads 1 --min_insert_block_size_bytes 0 --min_insert_block_size_rows 0 -q "INSERT INTO ghdata FORMAT JSONAsObject" +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} \ + --max_memory_usage 10G --query "INSERT INTO ghdata FORMAT JSONAsObject" ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)" @@ -16,7 +17,7 @@ ${CLICKHOUSE_CLIENT} -q \ "SELECT data.repo.name, count() AS stars FROM ghdata \ WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer=1 -q \ +${CLICKHOUSE_CLIENT} --enable_analyzer=1 -q \ "SELECT data.payload.commits[].author.name AS name, count() AS c FROM ghdata \ ARRAY JOIN data.payload.commits[].author.name \ GROUP BY name ORDER BY c DESC, name LIMIT 5" From 7f2c61799dc984add3056f9c77c4e64476ad917b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 06:14:59 +0200 Subject: [PATCH 1105/1722] Fix test --- .../0_stateless/01825_new_type_json_ghdata_insert_select.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh index ef87034ff89..2afec5ba7fe 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh @@ -13,10 +13,10 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree OR ${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_json_type 1 -cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2 FORMAT JSONAsObject" +cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} --max_memory_usage 10G -q "INSERT INTO ghdata_2 FORMAT JSONAsObject" cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_string FORMAT JSONAsString" -${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_from_string SELECT data FROM ghdata_2_string" +${CLICKHOUSE_CLIENT} --max_memory_usage 10G -q "INSERT INTO ghdata_2_from_string SELECT data FROM ghdata_2_string" ${CLICKHOUSE_CLIENT} -q "SELECT \ (SELECT mapSort(groupUniqArrayMap(JSONAllPathsWithTypes(data))), sum(cityHash64(toString(data))) FROM ghdata_2_from_string) = \ From d34d41d1e240f0df6256a827661174a30011f204 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 18 Aug 2024 04:34:38 +0000 Subject: [PATCH 1106/1722] fix test --- .../queries/0_stateless/03221_refreshable_matview_progress.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql index 1be276c485c..de8de41fd04 100644 --- a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql @@ -16,5 +16,3 @@ SELECT rand64() AS x; SELECT sleep(2); SELECT read_rows, total_rows, progress FROM system.view_refreshes WHERE database = currentDatabase() and view = '03221_rmv'; - -DROP TABLE 03221_rmv; From 03ab872f5c628613e0e187d4e1ad3ca9b9148bf6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 07:34:44 +0200 Subject: [PATCH 1107/1722] Fix error --- src/IO/S3/PocoHTTPClient.h | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 3b7ec4d1d9c..eb65460ce13 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -20,6 +20,7 @@ #include #include + namespace Aws::Http::Standard { class StandardHttpResponse; @@ -27,15 +28,17 @@ class StandardHttpResponse; namespace DB { - class Context; } + namespace DB::S3 { + class ClientFactory; class PocoHTTPClient; + struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration { std::function per_request_configuration; @@ -76,6 +79,7 @@ private: friend ClientFactory; }; + class PocoHTTPResponse : public Aws::Http::Standard::StandardHttpResponse { public: @@ -115,6 +119,7 @@ private: Aws::Utils::Stream::ResponseStream body_stream; }; + class PocoHTTPClient : public Aws::Http::HttpClient { public: @@ -170,10 +175,10 @@ protected: std::function error_report; ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; - unsigned int s3_max_redirects; + unsigned int s3_max_redirects = 0; bool s3_use_adaptive_timeouts = true; bool enable_s3_requests_logging = false; - bool for_disk_s3; + bool for_disk_s3 = false; /// Limits get request per second rate for GET, SELECT and all other requests, excluding throttled by put throttler /// (i.e. throttles GetObject, HeadObject) From 4e91c663a62529d27ddcf6d364338f001e1706eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 08:28:16 +0200 Subject: [PATCH 1108/1722] Fix error --- src/IO/S3/PocoHTTPClient.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index dc7dcdc6793..3e060e21c51 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -397,8 +397,11 @@ void PocoHTTPClient::makeRequestInternalImpl( try { - const auto proxy_configuration = per_request_configuration(); - for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt) + ProxyConfiguration proxy_configuration; + if (per_request_configuration) + proxy_configuration = per_request_configuration(); + + for (size_t attempt = 0; attempt <= s3_max_redirects; ++attempt) { Poco::URI target_uri(uri); @@ -516,7 +519,6 @@ void PocoHTTPClient::makeRequestInternalImpl( LOG_TEST(log, "Redirecting request to new location: {}", location); addMetric(request, S3MetricType::Redirects); - continue; } @@ -564,9 +566,9 @@ void PocoHTTPClient::makeRequestInternalImpl( } else { - if (status_code == 429 || status_code == 503) - { // API throttling + { + /// API throttling addMetric(request, S3MetricType::Throttling); } else if (status_code >= 300) From 7071942858e44053b92b8386e68251be7718e3b5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 09:05:45 +0200 Subject: [PATCH 1109/1722] Miscellanous changes from #66999 --- programs/keeper/Keeper.cpp | 4 +++- src/Daemon/BaseDaemon.cpp | 4 +--- src/Daemon/BaseDaemon.h | 1 - 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index a447a9e50f6..ced661d9772 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -66,6 +66,8 @@ /// A minimal file used when the keeper is run without installation INCBIN(keeper_resource_embedded_xml, SOURCE_DIR "/programs/keeper/keeper_embedded.xml"); +extern const char * GIT_HASH; + int mainEntryClickHouseKeeper(int argc, char ** argv) { DB::Keeper app; @@ -675,7 +677,7 @@ void Keeper::logRevision() const "Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}", VERSION_STRING, ClickHouseRevision::getVersionRevision(), - git_hash.empty() ? "" : git_hash, + GIT_HASH, build_id.empty() ? "" : build_id, getpid()); } diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index f75699881bd..c42bf7641d2 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -452,8 +452,6 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() build_id = SymbolIndex::instance().getBuildIDHex(); #endif - git_hash = GIT_HASH; - #if defined(OS_LINUX) std::string executable_path = getExecutablePath(); @@ -466,7 +464,7 @@ void BaseDaemon::logRevision() const { logger().information("Starting " + std::string{VERSION_FULL} + " (revision: " + std::to_string(ClickHouseRevision::getVersionRevision()) - + ", git hash: " + (git_hash.empty() ? "" : git_hash) + + ", git hash: " + std::string(GIT_HASH) + ", build id: " + (build_id.empty() ? "" : build_id) + ")" + ", PID " + std::to_string(getpid())); } diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index b15aa74fcf3..a6efa94a567 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -165,7 +165,6 @@ protected: Poco::Util::AbstractConfiguration * last_configuration = nullptr; String build_id; - String git_hash; String stored_binary_hash; bool should_setup_watchdog = false; From 2a48aaad561f52539edbede94015c35e264bd344 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 09:12:49 +0200 Subject: [PATCH 1110/1722] Fix build --- src/Interpreters/executeQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index fe87eed5570..decc16a3704 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -786,7 +786,7 @@ static std::tuple executeQueryImpl( /// Verify that AST formatting is consistent: /// If you format AST, parse it back, and format it again, you get the same string. - String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true); + String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false); /// The query can become more verbose after formatting, so: size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0; @@ -811,7 +811,7 @@ static std::tuple executeQueryImpl( chassert(ast2); - String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true); + String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false); if (formatted1 != formatted2) throw Exception(ErrorCodes::LOGICAL_ERROR, From 4f7e3e8374acd98496092e8f8a219af6755a2f70 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 09:38:00 +0200 Subject: [PATCH 1111/1722] Fix test 01017_uniqCombined_memory_usage --- .../0_stateless/01017_uniqCombined_memory_usage.sql | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index c13a0859183..eca370d94af 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -7,7 +7,8 @@ -- sizeof(HLL) is (2^K * 6 / 8) -- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400 -SET use_uncompressed_cache = 0; +SET use_uncompressed_cache = 0; +SET memory_profiler_step = 1; -- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements SELECT 'UInt32'; @@ -31,14 +32,14 @@ SELECT 'K=16'; SELECT 'UInt32'; SET max_memory_usage = 2000000; SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(16)(number % 4096) u FROM numbers(4096 * 100) GROUP BY k); -- { serverError MEMORY_LIMIT_EXCEEDED } -SET max_memory_usage = 4915200; +SET max_memory_usage = 5230000; SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(16)(number % 4096) u FROM numbers(4096 * 100) GROUP BY k); -- HashTable for UInt64 (used until (1<<11) elements), hence 2048 elements SELECT 'UInt64'; SET max_memory_usage = 2000000; SELECT sum(u) FROM (SELECT intDiv(number, 2048) AS k, uniqCombined(16)(reinterpretAsString(number % 2048)) u FROM numbers(2048 * 100) GROUP BY k); -- { serverError MEMORY_LIMIT_EXCEEDED } -SET max_memory_usage = 4915200; +SET max_memory_usage = 5900000; SELECT sum(u) FROM (SELECT intDiv(number, 2048) AS k, uniqCombined(16)(reinterpretAsString(number % 2048)) u FROM numbers(2048 * 100) GROUP BY k); SELECT 'K=18'; From ec8554c85aeae5dcc8367ce09d093c5526ef1d47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 09:41:29 +0200 Subject: [PATCH 1112/1722] Fix build --- src/Common/SignalHandlers.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index c4358da2453..6ac6cbcae29 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -18,13 +18,17 @@ namespace DB { + namespace ErrorCodes { extern const int CANNOT_SET_SIGNAL_HANDLER; extern const int CANNOT_SEND_SIGNAL; } + } +extern const char * GIT_HASH; + using namespace DB; @@ -334,7 +338,7 @@ void SignalListener::onTerminate(std::string_view message, UInt32 thread_num) co size_t pos = message.find('\n'); LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", thread_num, message.substr(0, pos)); + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, thread_num, message.substr(0, pos)); /// Print trace from std::terminate exception line-by-line to make it easy for grep. while (pos != std::string_view::npos) @@ -368,7 +372,7 @@ try LOG_FATAL(log, "########## Short fault info ############"); LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, thread_num, sig); std::string signal_description = "Unknown signal"; @@ -434,13 +438,13 @@ try if (query_id.empty()) { LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, thread_num, signal_description, sig); } else { LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, thread_num, query_id, query, signal_description, sig); } From d6e170f77704833fa6655820d55090ba18b0b9fe Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Sat, 17 Aug 2024 18:31:11 +0800 Subject: [PATCH 1113/1722] repeat field is also compound types, ignore it. --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c6167e572df..7b5c29e321f 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -286,6 +286,9 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa if (!s) continue; + if(s->descr()->schema_node()->is_repeated()) + continue; + auto path = c->path_in_schema()->ToDotVector(); if (path.size() != 1) continue; // compound types not supported From 858f8b502002661584e6153d39a23edc87b49dda Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Sun, 18 Aug 2024 19:11:30 +0800 Subject: [PATCH 1114/1722] add test and update codes per commit --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- .../02841_parquet_filter_pushdown_bug.reference | 1 + .../02841_parquet_filter_pushdown_bug.sh.sh | 8 ++++++++ .../0_stateless/data_parquet/68131.parquet | Bin 0 -> 289 bytes 4 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference create mode 100755 tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh create mode 100644 tests/queries/0_stateless/data_parquet/68131.parquet diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7b5c29e321f..1f213fef731 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -286,7 +286,7 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa if (!s) continue; - if(s->descr()->schema_node()->is_repeated()) + if (s->descr()->schema_node()->is_repeated()) continue; auto path = c->path_in_schema()->ToDotVector(); diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference new file mode 100644 index 00000000000..6ed63af507a --- /dev/null +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.reference @@ -0,0 +1 @@ +[1,2] diff --git a/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh new file mode 100755 index 00000000000..58eb207b6e6 --- /dev/null +++ b/tests/queries/0_stateless/02841_parquet_filter_pushdown_bug.sh.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select f from file('$CURDIR/data_parquet/68131.parquet', Parquet, 'f Array(Int32)')" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_parquet/68131.parquet b/tests/queries/0_stateless/data_parquet/68131.parquet new file mode 100644 index 0000000000000000000000000000000000000000..169f6152003db164c78e33cd69205caa33f906b5 GIT binary patch literal 289 zcmXAl!D_=W42Bgqgq#M0O4Q&(E)5xMp|QKBg Date: Sun, 18 Aug 2024 12:18:24 +0000 Subject: [PATCH 1115/1722] fix test --- .../0_stateless/03221_refreshable_matview_progress.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql index de8de41fd04..ecb385c9bfa 100644 --- a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql @@ -3,7 +3,7 @@ set allow_experimental_refreshable_materialized_view=1; CREATE MATERIALIZED VIEW 03221_rmv -REFRESH AFTER 1 SECOND +REFRESH AFTER 10 SECOND ( x UInt64 ) @@ -16,3 +16,5 @@ SELECT rand64() AS x; SELECT sleep(2); SELECT read_rows, total_rows, progress FROM system.view_refreshes WHERE database = currentDatabase() and view = '03221_rmv'; + +DROP TABLE 03221_rmv; From f5308635d193859cdb19a71040006278a21bdc51 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 18 Aug 2024 15:25:07 +0200 Subject: [PATCH 1116/1722] Revert "Improve compatibility of `upper/lowerUTF8` with Spark" --- .gitmodules | 7 +- contrib/icu | 2 +- src/Common/examples/CMakeLists.txt | 5 - src/Common/examples/utf8_upper_lower.cpp | 27 -- src/Functions/LowerUpperImpl.h | 1 + src/Functions/LowerUpperUTF8Impl.h | 283 +++++++++++++++--- src/Functions/initcapUTF8.cpp | 3 +- src/Functions/lowerUTF8.cpp | 25 +- src/Functions/upperUTF8.cpp | 24 +- .../00170_lower_upper_utf8.reference | 4 - .../0_stateless/00170_lower_upper_utf8.sql | 11 - .../00233_position_function_family.sql | 3 - .../0_stateless/00761_lower_utf8_bug.sql | 3 - .../0_stateless/01278_random_string_utf8.sql | 3 - .../0_stateless/01431_utf8_ubsan.reference | 4 +- .../queries/0_stateless/01431_utf8_ubsan.sql | 3 - .../0_stateless/01590_countSubstrings.sql | 3 - ...71_lower_upper_utf8_row_overlaps.reference | 4 +- .../02071_lower_upper_utf8_row_overlaps.sql | 3 - ...new_functions_must_be_documented.reference | 2 + .../02514_if_with_lazy_low_cardinality.sql | 3 - .../0_stateless/02807_lower_utf8_msan.sql | 3 - tests/queries/0_stateless/03015_peder1001.sql | 3 - 23 files changed, 265 insertions(+), 164 deletions(-) delete mode 100644 src/Common/examples/utf8_upper_lower.cpp diff --git a/.gitmodules b/.gitmodules index f18844e5eb4..cdee6a43ad8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -106,6 +106,9 @@ [submodule "contrib/icudata"] path = contrib/icudata url = https://github.com/ClickHouse/icudata +[submodule "contrib/icu"] + path = contrib/icu + url = https://github.com/unicode-org/icu [submodule "contrib/flatbuffers"] path = contrib/flatbuffers url = https://github.com/ClickHouse/flatbuffers @@ -366,7 +369,3 @@ [submodule "contrib/numactl"] path = contrib/numactl url = https://github.com/ClickHouse/numactl.git -[submodule "contrib/icu"] - path = contrib/icu - url = https://github.com/ClickHouse/icu - branch = ClickHouse/release-75-1 diff --git a/contrib/icu b/contrib/icu index 4216173eeeb..7750081bda4 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit 4216173eeeb39c1d4caaa54a68860e800412d273 +Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 8383e80d09d..69580d4ad0e 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -92,8 +92,3 @@ endif() clickhouse_add_executable (check_pointer_valid check_pointer_valid.cpp) target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io clickhouse_common_config) - -if (TARGET ch_contrib::icu) - clickhouse_add_executable (utf8_upper_lower utf8_upper_lower.cpp) - target_link_libraries (utf8_upper_lower PRIVATE ch_contrib::icu) -endif () diff --git a/src/Common/examples/utf8_upper_lower.cpp b/src/Common/examples/utf8_upper_lower.cpp deleted file mode 100644 index 826e1763105..00000000000 --- a/src/Common/examples/utf8_upper_lower.cpp +++ /dev/null @@ -1,27 +0,0 @@ -#include -#include - -std::string utf8_to_lower(const std::string & input) -{ - icu::UnicodeString unicodeInput(input.c_str(), "UTF-8"); - unicodeInput.toLower(); - std::string output; - unicodeInput.toUTF8String(output); - return output; -} - -std::string utf8_to_upper(const std::string & input) -{ - icu::UnicodeString unicodeInput(input.c_str(), "UTF-8"); - unicodeInput.toUpper(); - std::string output; - unicodeInput.toUTF8String(output); - return output; -} - -int main() -{ - std::string input = "ır"; - std::cout << "upper:" << utf8_to_upper(input) << std::endl; - return 0; -} diff --git a/src/Functions/LowerUpperImpl.h b/src/Functions/LowerUpperImpl.h index a52703d10c8..d463ef96e16 100644 --- a/src/Functions/LowerUpperImpl.h +++ b/src/Functions/LowerUpperImpl.h @@ -1,6 +1,7 @@ #pragma once #include + namespace DB { diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 5da085f48e5..eedabca5b22 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -1,14 +1,15 @@ #pragma once - -#include "config.h" - -#if USE_ICU - #include #include -#include -#include +#include +#include #include +#include + +#ifdef __SSE2__ +#include +#endif + namespace DB { @@ -18,7 +19,71 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -template +/// xor or do nothing +template +UInt8 xor_or_identity(const UInt8 c, const int mask) +{ + return c ^ mask; +} + +template <> +inline UInt8 xor_or_identity(const UInt8 c, const int) +{ + return c; +} + +/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array +template +inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) +{ + if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) + { + /// ЀÐЂЃЄЅІЇЈЉЊЋЌÐÐŽÐ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x10); + } + else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) + { + /// ÑёђѓєѕіїјљњћќÑўџ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x10); + } + else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) + { + /// Ð-П + *dst++ = *src++; + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) + { + /// а-п + *dst++ = *src++; + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) + { + /// Р-Я + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) + { + /// Ñ€-Ñ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x20); + } +} + + +/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case. + * Note: It is assumed that after the character is converted to another case, + * the length of its multibyte sequence in UTF-8 does not change. + * Otherwise, the behavior is undefined. + */ +template struct LowerUpperUTF8Impl { static void vector( @@ -38,46 +103,180 @@ struct LowerUpperUTF8Impl return; } - res_data.resize(data.size()); - res_offsets.resize_exact(offsets.size()); - - String output; - size_t curr_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) - { - const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); - size_t size = offsets[i] - offsets[i - 1]; - - icu::UnicodeString input(data_start, static_cast(size), "UTF-8"); - if constexpr (upper) - input.toUpper(); - else - input.toLower(); - - output.clear(); - input.toUTF8String(output); - - /// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first - /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this - /// case, the behavior is also reasonable. - const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size()); - size_t valid_size = res_end ? res_end - output.data() + 1 : 0; - - res_data.resize(curr_offset + valid_size + 1); - memcpy(&res_data[curr_offset], output.data(), valid_size); - res_data[curr_offset + valid_size] = 0; - - curr_offset += valid_size + 1; - res_offsets[i] = curr_offset; - } + res_data.resize_exact(data.size()); + res_offsets.assign(offsets); + array(data.data(), data.data() + data.size(), offsets, res_data.data()); } static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument"); } + + /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. + * `src` and `dst` are incremented by corresponding sequence lengths. */ + static bool toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst, bool partial) + { + if (src[0] <= ascii_upper_bound) + { + if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) + *dst++ = *src++ ^ flip_case_mask; + else + *dst++ = *src++; + } + else if (src + 1 < src_end + && ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu)))) + { + cyrillic_to_case(src, dst); + } + else if (src + 1 < src_end && src[0] == 0xC2u) + { + /// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF + *dst++ = *src++; + *dst++ = *src++; + } + else if (src + 2 < src_end && src[0] == 0xE2u) + { + /// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF + *dst++ = *src++; + *dst++ = *src++; + *dst++ = *src++; + } + else + { + size_t src_sequence_length = UTF8::seqLength(*src); + /// In case partial buffer was passed (due to SSE optimization) + /// we cannot convert it with current src_end, but we may have more + /// bytes to convert and eventually got correct symbol. + if (partial && src_sequence_length > static_cast(src_end - src)) + return false; + + auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src); + if (src_code_point) + { + int dst_code_point = to_case(*src_code_point); + if (dst_code_point > 0) + { + size_t dst_sequence_length = UTF8::convertCodePointToUTF8(dst_code_point, dst, src_end - src); + assert(dst_sequence_length <= 4); + + /// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8. + /// As an example, this happens for ß and ẞ. + if (dst_sequence_length == src_sequence_length) + { + src += dst_sequence_length; + dst += dst_sequence_length; + return true; + } + } + } + + *dst = *src; + ++dst; + ++src; + } + + return true; + } + +private: + static constexpr auto ascii_upper_bound = '\x7f'; + static constexpr auto flip_case_mask = 'A' ^ 'a'; + + static void array(const UInt8 * src, const UInt8 * src_end, const ColumnString::Offsets & offsets, UInt8 * dst) + { + const auto * offset_it = offsets.begin(); + const UInt8 * begin = src; + +#ifdef __SSE2__ + static constexpr auto bytes_sse = sizeof(__m128i); + + /// If we are before this position, we can still read at least bytes_sse. + const auto * src_end_sse = src_end - bytes_sse + 1; + + /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) + const auto v_zero = _mm_setzero_si128(); + const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1); + const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1); + const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask); + + while (src < src_end_sse) + { + const auto chars = _mm_loadu_si128(reinterpret_cast(src)); + + /// check for ASCII + const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero); + const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii); + + /// ASCII + if (mask_is_not_ascii == 0) + { + const auto is_not_case + = _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound)); + const auto mask_is_not_case = _mm_movemask_epi8(is_not_case); + + /// everything in correct case ASCII + if (mask_is_not_case == 0) + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars); + else + { + /// ASCII in mixed case + /// keep `flip_case_mask` only where necessary, zero out elsewhere + const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case); + + /// flip case by applying calculated mask + const auto cased_chars = _mm_xor_si128(chars, xor_mask); + + /// store result back to destination + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars); + } + + src += bytes_sse; + dst += bytes_sse; + } + else + { + /// UTF-8 + + /// Find the offset of the next string after src + size_t offset_from_begin = src - begin; + while (offset_from_begin >= *offset_it) + ++offset_it; + + /// Do not allow one row influence another (since row may have invalid sequence, and break the next) + const UInt8 * row_end = begin + *offset_it; + chassert(row_end >= src); + const UInt8 * expected_end = std::min(src + bytes_sse, row_end); + + while (src < expected_end) + { + if (!toCase(src, expected_end, dst, /* partial= */ true)) + { + /// Fallback to handling byte by byte. + src_end_sse = src; + break; + } + } + } + } + + /// Find the offset of the next string after src + size_t offset_from_begin = src - begin; + while (offset_it != offsets.end() && offset_from_begin >= *offset_it) + ++offset_it; +#endif + + /// handle remaining symbols, row by row (to avoid influence of bad UTF8 symbols from one row, to another) + while (src < src_end) + { + const UInt8 * row_end = begin + *offset_it; + chassert(row_end >= src); + + while (src < row_end) + toCase(src, row_end, dst, /* partial= */ false); + ++offset_it; + } + } }; } - -#endif diff --git a/src/Functions/initcapUTF8.cpp b/src/Functions/initcapUTF8.cpp index 004586dce26..282d846094e 100644 --- a/src/Functions/initcapUTF8.cpp +++ b/src/Functions/initcapUTF8.cpp @@ -1,8 +1,9 @@ #include #include +#include #include #include -#include + namespace DB { diff --git a/src/Functions/lowerUTF8.cpp b/src/Functions/lowerUTF8.cpp index e2f7cb84730..7adb0069121 100644 --- a/src/Functions/lowerUTF8.cpp +++ b/src/Functions/lowerUTF8.cpp @@ -1,10 +1,9 @@ -#include "config.h" - -#if USE_ICU - -#include +#include #include #include +#include +#include + namespace DB { @@ -16,25 +15,13 @@ struct NameLowerUTF8 static constexpr auto name = "lowerUTF8"; }; -using FunctionLowerUTF8 = FunctionStringToString, NameLowerUTF8>; +using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; } REGISTER_FUNCTION(LowerUTF8) { - FunctionDocumentation::Description description - = R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)"; - FunctionDocumentation::Syntax syntax = "lowerUTF8(input)"; - FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}}; - FunctionDocumentation::ReturnedValue returned_value = "A String data type value"; - FunctionDocumentation::Examples examples = { - {"first", "SELECT lowerUTF8('München') as Lowerutf8;", "münchen"}, - }; - FunctionDocumentation::Categories categories = {"String"}; - - factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); + factory.registerFunction(); } } - -#endif diff --git a/src/Functions/upperUTF8.cpp b/src/Functions/upperUTF8.cpp index ef26430331f..659e67f0ef3 100644 --- a/src/Functions/upperUTF8.cpp +++ b/src/Functions/upperUTF8.cpp @@ -1,10 +1,8 @@ -#include "config.h" - -#if USE_ICU - -#include #include #include +#include +#include + namespace DB { @@ -16,25 +14,13 @@ struct NameUpperUTF8 static constexpr auto name = "upperUTF8"; }; -using FunctionUpperUTF8 = FunctionStringToString, NameUpperUTF8>; +using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; } REGISTER_FUNCTION(UpperUTF8) { - FunctionDocumentation::Description description - = R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)"; - FunctionDocumentation::Syntax syntax = "upperUTF8(input)"; - FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}}; - FunctionDocumentation::ReturnedValue returned_value = "A String data type value"; - FunctionDocumentation::Examples examples = { - {"first", "SELECT upperUTF8('München') as Upperutf8;", "MÃœNCHEN"}, - }; - FunctionDocumentation::Categories categories = {"String"}; - - factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); + factory.registerFunction(); } } - -#endif diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.reference b/tests/queries/0_stateless/00170_lower_upper_utf8.reference index 3c644f22b9b..f202cb75513 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.reference +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.reference @@ -22,7 +22,3 @@ 1 1 1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/00170_lower_upper_utf8.sql b/tests/queries/0_stateless/00170_lower_upper_utf8.sql index 85b6c5c6095..4caba2033ff 100644 --- a/tests/queries/0_stateless/00170_lower_upper_utf8.sql +++ b/tests/queries/0_stateless/00170_lower_upper_utf8.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - select lower('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str; select lowerUTF8('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str; select lower('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'aaaaaaaaaaaaaaa012345789,.!aaaa'; @@ -30,11 +27,3 @@ select sum(lower(materialize('aaaaÐБВГAAAAaaAA')) = materialize('aaaaÐБВ select sum(upper(materialize('aaaaÐБВГAAAAaaAA')) = materialize('AAAAÐБВГAAAAAAAA')) = count() from system.one array join range(16384) as n; select sum(lowerUTF8(materialize('aaaaÐБВГAAAAaaAA')) = materialize('aaaaабвгaaaaaaaa')) = count() from system.one array join range(16384) as n; select sum(upperUTF8(materialize('aaaaÐБВГAAAAaaAA')) = materialize('AAAAÐБВГAAAAAAAA')) = count() from system.one array join range(16384) as n; - --- Turkish language -select upperUTF8('ır') = 'IR'; -select lowerUTF8('ır') = 'ır'; - --- German language -select upper('öäüß') = 'öäüß'; -select lower('ÖÄÜẞ') = 'ÖÄÜẞ'; diff --git a/tests/queries/0_stateless/00233_position_function_family.sql b/tests/queries/0_stateless/00233_position_function_family.sql index d6668cb7ba4..dd7394bc39a 100644 --- a/tests/queries/0_stateless/00233_position_function_family.sql +++ b/tests/queries/0_stateless/00233_position_function_family.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - SET send_logs_level = 'fatal'; select 1 = position('', ''); diff --git a/tests/queries/0_stateless/00761_lower_utf8_bug.sql b/tests/queries/0_stateless/00761_lower_utf8_bug.sql index a0ab55edc15..de20b894331 100644 --- a/tests/queries/0_stateless/00761_lower_utf8_bug.sql +++ b/tests/queries/0_stateless/00761_lower_utf8_bug.sql @@ -1,4 +1 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - SELECT lowerUTF8('\xF0') = lowerUTF8('\xF0'); diff --git a/tests/queries/0_stateless/01278_random_string_utf8.sql b/tests/queries/0_stateless/01278_random_string_utf8.sql index 290d6a0c759..da2dc48c3e1 100644 --- a/tests/queries/0_stateless/01278_random_string_utf8.sql +++ b/tests/queries/0_stateless/01278_random_string_utf8.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - SELECT randomStringUTF8('string'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT lengthUTF8(randomStringUTF8(100)); SELECT toTypeName(randomStringUTF8(10)); diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.reference b/tests/queries/0_stateless/01431_utf8_ubsan.reference index dc785e57851..c98c950d535 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.reference +++ b/tests/queries/0_stateless/01431_utf8_ubsan.reference @@ -1,2 +1,2 @@ -EFBFBD -EFBFBD +FF +FF diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.sql b/tests/queries/0_stateless/01431_utf8_ubsan.sql index 3a28e023805..d6a299225b1 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.sql +++ b/tests/queries/0_stateless/01431_utf8_ubsan.sql @@ -1,5 +1,2 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - SELECT hex(lowerUTF8('\xFF')); SELECT hex(upperUTF8('\xFF')); diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index 5ec4f412d7f..b38cbb7d188 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - -- -- countSubstrings -- diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index deabef61a88..a3bac432482 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo -- MONOGRAM FOR YANG with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV; -1,"�","�" +1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -1,"�","�" +1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -- NOTE: regression test for introduced bug -- https://github.com/ClickHouse/ClickHouse/issues/42756 diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql index d175e0659d0..8ca0a3f5f75 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - drop table if exists utf8_overlap; create table utf8_overlap (str String) engine=Memory(); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 0980e25b70f..c39f1fb1ce9 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -416,6 +416,7 @@ logTrace lowCardinalityIndices lowCardinalityKeys lower +lowerUTF8 makeDate makeDate32 makeDateTime @@ -896,6 +897,7 @@ tupleToNameValuePairs unbin unhex upper +upperUTF8 uptime validateNestedArraySizes version diff --git a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql index b169cfd0ab9..80e3c0a9ece 100644 --- a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql +++ b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - create table if not exists t (`arr.key` Array(LowCardinality(String)), `arr.value` Array(LowCardinality(String))) engine = Memory; insert into t (`arr.key`, `arr.value`) values (['a'], ['b']); select if(true, if(lowerUTF8(arr.key) = 'a', 1, 2), 3) as x from t left array join arr; diff --git a/tests/queries/0_stateless/02807_lower_utf8_msan.sql b/tests/queries/0_stateless/02807_lower_utf8_msan.sql index 95f224577f7..e9eb18bf615 100644 --- a/tests/queries/0_stateless/02807_lower_utf8_msan.sql +++ b/tests/queries/0_stateless/02807_lower_utf8_msan.sql @@ -1,5 +1,2 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - SELECT lowerUTF8(arrayJoin(['©--------------------------------------', '©--------------------'])) ORDER BY 1; SELECT upperUTF8(materialize('aaaaÐБВГaaaaaaaaaaaaÐБВГAAAAaaAA')) FROM numbers(2); diff --git a/tests/queries/0_stateless/03015_peder1001.sql b/tests/queries/0_stateless/03015_peder1001.sql index df8e4db1536..810503207f2 100644 --- a/tests/queries/0_stateless/03015_peder1001.sql +++ b/tests/queries/0_stateless/03015_peder1001.sql @@ -1,6 +1,3 @@ --- Tags: no-fasttest --- no-fasttest: upper/lowerUTF8 use ICU - DROP TABLE IF EXISTS test_data; CREATE TABLE test_data From 427016a450cad536e8cbaf4de04d07313456aa4b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 6 Aug 2024 21:39:41 +0200 Subject: [PATCH 1117/1722] CI: Functional tests to store artifacts on timeout --- docker/test/fasttest/run.sh | 18 +--------- docker/test/sqllogic/Dockerfile | 1 - docker/test/sqllogic/run.sh | 2 +- docker/test/sqltest/Dockerfile | 1 - docker/test/stateful/run.sh | 25 +------------- docker/test/stateless/Dockerfile | 1 - docker/test/stateless/run.sh | 21 ++---------- docker/test/stateless/utils.lib | 16 --------- tests/ci/ci.py | 30 ++++++++-------- tests/ci/ci_definitions.py | 3 +- tests/ci/functional_test_check.py | 37 +++++++++++++++----- tests/ci/report.py | 11 +++--- tests/ci/tee_popen.py | 57 +++++++++++++++++++++++++++---- 13 files changed, 107 insertions(+), 116 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 394d31addb1..9920326b11c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -256,22 +256,6 @@ function configure rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" } -function timeout_with_logging() { - local exit_code=0 - - timeout -s TERM --preserve-status "${@}" || exit_code="${?}" - - echo "Checking if it is a timeout. The code 124 will indicate a timeout." - if [[ "${exit_code}" -eq "124" ]] - then - echo "The command 'timeout ${*}' has been killed by timeout." - else - echo "No, it isn't a timeout." - fi - - return $exit_code -} - function run_tests { clickhouse-server --version @@ -340,7 +324,7 @@ case "$stage" in configure 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/install_log.txt" ;& "run_tests") - timeout_with_logging 35m bash -c run_tests ||: + run_tests ||: /process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \ --out-results-file "$FASTTEST_OUTPUT/test_results.tsv" \ --out-status-file "$FASTTEST_OUTPUT/check_status.tsv" || echo -e "failure\tCannot parse results" > "$FASTTEST_OUTPUT/check_status.tsv" diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index 1425e12cd84..6397526388e 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -35,7 +35,6 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ ENV TZ=Europe/Amsterdam -ENV MAX_RUN_TIME=9000 RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ARG sqllogic_test_repo="https://github.com/gregrahn/sqllogictest.git" diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index ccba344035e..32368980f9b 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -94,7 +94,7 @@ function run_tests() export -f run_tests -timeout "${MAX_RUN_TIME:-9000}" bash -c run_tests || echo "timeout reached" >&2 +run_tests #/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile index 71d915b0c7a..b805bb03c2b 100644 --- a/docker/test/sqltest/Dockerfile +++ b/docker/test/sqltest/Dockerfile @@ -22,7 +22,6 @@ ARG sqltest_repo="https://github.com/elliotchance/sqltest/" RUN git clone ${sqltest_repo} ENV TZ=UTC -ENV MAX_RUN_TIME=900 RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone COPY run.sh / diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 3a4f0d97993..c072eeb0fa8 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -4,9 +4,6 @@ source /setup_export_logs.sh set -e -x -MAX_RUN_TIME=${MAX_RUN_TIME:-3600} -MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 3600 : MAX_RUN_TIME)) - # Choose random timezone for this test run TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)" echo "Choosen random timezone $TZ" @@ -123,9 +120,6 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] clickhouse-client --query "DROP TABLE datasets.hits_v1" clickhouse-client --query "DROP TABLE datasets.visits_v1" - - MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) - MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) else clickhouse-client --query "CREATE DATABASE test" clickhouse-client --query "SHOW TABLES FROM test" @@ -257,24 +251,7 @@ function run_tests() export -f run_tests -function timeout_with_logging() { - local exit_code=0 - - timeout -s TERM --preserve-status "${@}" || exit_code="${?}" - - echo "Checking if it is a timeout. The code 124 will indicate a timeout." - if [[ "${exit_code}" -eq "124" ]] - then - echo "The command 'timeout ${*}' has been killed by timeout." - else - echo "No, it isn't a timeout." - fi - - return $exit_code -} - -TIMEOUT=$((MAX_RUN_TIME - 700)) -timeout_with_logging "$TIMEOUT" bash -c run_tests ||: +run_tests ||: echo "Files in current directory" ls -la ./ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index d8eb072328f..b0c4914a4e8 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -65,7 +65,6 @@ ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV NUM_TRIES=1 -ENV MAX_RUN_TIME=0 # Unrelated to vars in setup_minio.sh, but should be the same there # to have the same binaries for local running scenario diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c70cbe1fe45..ad0cd321cc5 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -12,9 +12,6 @@ dmesg --clear # fail on errors, verbose and export all env variables set -e -x -a -MAX_RUN_TIME=${MAX_RUN_TIME:-9000} -MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 9000 : MAX_RUN_TIME)) - USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} @@ -308,8 +305,6 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" - TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800)) - START_TIME=${SECONDS} set +e TEST_ARGS=( @@ -324,32 +319,22 @@ function run_tests() --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" ) - timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s clickhouse-test "${TEST_ARGS[@]}" 2>&1 \ + clickhouse-test "${TEST_ARGS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e - DURATION=$((SECONDS - START_TIME)) - - echo "Elapsed ${DURATION} seconds." - if [[ $DURATION -ge $TIMEOUT ]] - then - echo "It looks like the command is terminated by the timeout, which is ${TIMEOUT} seconds." - fi } export -f run_tests - -# This should be enough to setup job and collect artifacts -TIMEOUT=$((MAX_RUN_TIME - 700)) if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. - timeout_with_logging "$TIMEOUT" bash -c 'NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests' \ + NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests \ | sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||: fi -timeout_with_logging "$TIMEOUT" bash -c run_tests ||: +run_tests ||: echo "Files in current directory" ls -la ./ diff --git a/docker/test/stateless/utils.lib b/docker/test/stateless/utils.lib index cb257536c36..31cd67254b4 100644 --- a/docker/test/stateless/utils.lib +++ b/docker/test/stateless/utils.lib @@ -40,22 +40,6 @@ function fn_exists() { declare -F "$1" > /dev/null; } -function timeout_with_logging() { - local exit_code=0 - - timeout -s TERM --preserve-status "${@}" || exit_code="${?}" - - echo "Checking if it is a timeout. The code 124 will indicate a timeout." - if [[ "${exit_code}" -eq "124" ]] - then - echo "The command 'timeout ${*}' has been killed by timeout." - else - echo "No, it isn't a timeout." - fi - - return $exit_code -} - function collect_core_dumps() { find . -type f -maxdepth 1 -name 'core.*' | while read -r core; do diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 49b597333dc..1208d8642ad 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -50,7 +50,6 @@ from github_helper import GitHub from pr_info import PRInfo from report import ( ERROR, - FAILURE, PENDING, SUCCESS, BuildResult, @@ -62,11 +61,11 @@ from report import ( FAIL, ) from s3_helper import S3Helper -from stopwatch import Stopwatch from tee_popen import TeePopen from ci_cache import CiCache from ci_settings import CiSettings from ci_buddy import CIBuddy +from stopwatch import Stopwatch from version_helper import get_version_from_repo # pylint: disable=too-many-lines @@ -370,8 +369,8 @@ def _pre_action(s3, job_name, batch, indata, pr_info): # skip_status = SUCCESS already there GH.print_in_group("Commit Status Data", job_status) - # create pre report - jr = JobReport.create_pre_report(status=skip_status, job_skipped=to_be_skipped) + # create dummy report + jr = JobReport.create_dummy(status=skip_status, job_skipped=to_be_skipped) jr.dump() if not to_be_skipped: @@ -990,19 +989,20 @@ def _run_test(job_name: str, run_command: str) -> int: stopwatch = Stopwatch() job_log = Path(TEMP_PATH) / "job_log.txt" with TeePopen(run_command, job_log, env, timeout) as process: + print(f"Job process started, pid [{process.process.pid}]") retcode = process.wait() if retcode != 0: print(f"Run action failed for: [{job_name}] with exit code [{retcode}]") - if timeout and process.timeout_exceeded: - print(f"Timeout {timeout} exceeded, dumping the job report") - JobReport( - status=FAILURE, - description=f"Timeout {timeout} exceeded", - test_results=[TestResult.create_check_timeout_expired(timeout)], - start_time=stopwatch.start_time_str, - duration=stopwatch.duration_seconds, - additional_files=[job_log], - ).dump() + if process.timeout_exceeded: + print(f"Job timed out: [{job_name}] exit code [{retcode}]") + assert JobReport.exist(), "JobReport real or dummy must be present" + jr = JobReport.load() + if jr.dummy: + print( + f"ERROR: Run action failed with timeout and did not generate JobReport - update dummy report with execution time" + ) + jr.test_results = [TestResult.create_check_timeout_expired()] + jr.duration = stopwatch.duration_seconds print(f"Run action done for: [{job_name}]") return retcode @@ -1205,7 +1205,7 @@ def main() -> int: job_report ), "BUG. There must be job report either real report, or pre-report if job was killed" error_description = "" - if not job_report.pre_report: + if not job_report.dummy: # it's a real job report ch_helper = ClickHouseHelper() check_url = "" diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 1bed9db06f2..1d1c39f913d 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -332,7 +332,7 @@ class JobConfig: # will be triggered for the job if omitted in CI workflow yml run_command: str = "" # job timeout, seconds - timeout: Optional[int] = None + timeout: int = 7200 # sets number of batches for a multi-batch job num_batches: int = 1 # label that enables job in CI, if set digest isn't used @@ -421,7 +421,6 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=9000, ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index b7391eff01b..d08f98fa05f 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -5,10 +5,11 @@ import csv import logging import os import re +import signal import subprocess import sys from pathlib import Path -from typing import List, Tuple +from typing import List, Tuple, Optional from build_download_helper import download_all_deb_packages from clickhouse_helper import CiLogsCredentials @@ -25,11 +26,12 @@ from report import ( TestResults, read_test_results, FAILURE, + TestResult, ) from stopwatch import Stopwatch from tee_popen import TeePopen from ci_config import CI -from ci_utils import Utils +from ci_utils import Utils, Shell NO_CHANGES_MSG = "Nothing to run" @@ -113,10 +115,6 @@ def get_run_command( if flaky_check: envs.append("-e NUM_TRIES=50") - envs.append("-e MAX_RUN_TIME=2800") - else: - max_run_time = os.getenv("MAX_RUN_TIME", "0") - envs.append(f"-e MAX_RUN_TIME={max_run_time}") envs += [f"-e {e}" for e in additional_envs] @@ -128,7 +126,7 @@ def get_run_command( ) return ( - f"docker run --volume={builds_path}:/package_folder " + f"docker run --rm --name func-tester --volume={builds_path}:/package_folder " # For dmesg and sysctl "--privileged " f"{ci_logs_args}" @@ -198,7 +196,7 @@ def process_results( state, description = status[0][0], status[0][1] if ret_code != 0: state = ERROR - description += " (but script exited with an error)" + description = f"Job failed, exit code: {ret_code}. " + description try: results_path = result_directory / "test_results.tsv" @@ -240,7 +238,19 @@ def parse_args(): return parser.parse_args() +test_process = None # type: Optional[TeePopen] +timeout_expired = False + + +def handle_sigterm(signum, _frame): + print(f"WARNING: Received signal {signum}") + global timeout_expired + timeout_expired = True + Shell.check(f"docker exec func-tester pkill -f clickhouse-test", verbose=True) + + def main(): + signal.signal(signal.SIGTERM, handle_sigterm) logging.basicConfig(level=logging.INFO) for handler in logging.root.handlers: # pylint: disable=protected-access @@ -328,11 +338,13 @@ def main(): logging.info("Going to run func tests: %s", run_command) with TeePopen(run_command, run_log_path) as process: + global test_process + test_process = process retcode = process.wait() if retcode == 0: logging.info("Run successfully") else: - logging.info("Run failed") + logging.info("Run failed, exit code %s", retcode) try: subprocess.check_call( @@ -348,6 +360,13 @@ def main(): state, description, test_results, additional_logs = process_results( retcode, result_path, server_log_path ) + if timeout_expired: + description = "Timeout expired" + state = FAILURE + test_results.insert( + 0, TestResult.create_check_timeout_expired(stopwatch.duration_seconds) + ) + else: print( "This is validate bugfix or flaky check run, but no changes test to run - skip with success" diff --git a/tests/ci/report.py b/tests/ci/report.py index 6779a6dae96..c2632719aef 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -249,6 +249,7 @@ JOB_REPORT_FILE = Path(GITHUB_WORKSPACE) / "job_report.json" JOB_STARTED_TEST_NAME = "STARTED" JOB_FINISHED_TEST_NAME = "COMPLETED" +JOB_TIMEOUT_TEST_NAME = "Job Timeout Expired" @dataclass @@ -277,8 +278,8 @@ class TestResult: self.log_files.append(log_path) @staticmethod - def create_check_timeout_expired(timeout: float) -> "TestResult": - return TestResult("Check timeout expired", "FAIL", timeout) + def create_check_timeout_expired(duration: Optional[float] = None) -> "TestResult": + return TestResult(JOB_TIMEOUT_TEST_NAME, "FAIL", time=duration) TestResults = List[TestResult] @@ -303,7 +304,7 @@ class JobReport: # indicates that this is not real job report but report for the job that was skipped by rerun check job_skipped: bool = False # indicates that report generated by CI script in order to check later if job was killed before real report is generated - pre_report: bool = False + dummy: bool = False exit_code: int = -1 @staticmethod @@ -311,7 +312,7 @@ class JobReport: return datetime.datetime.utcnow().strftime("%Y-%m-%d %H:%M:%S") @classmethod - def create_pre_report(cls, status: str, job_skipped: bool) -> "JobReport": + def create_dummy(cls, status: str, job_skipped: bool) -> "JobReport": return JobReport( status=status, description="", @@ -320,7 +321,7 @@ class JobReport: duration=0.0, additional_files=[], job_skipped=job_skipped, - pre_report=True, + dummy=True, ) def update_duration(self): diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index 13db50df53f..ad3e62dab9c 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -2,6 +2,8 @@ import logging import os +import signal +import subprocess import sys from io import TextIOWrapper from pathlib import Path @@ -30,20 +32,35 @@ class TeePopen: self._process = None # type: Optional[Popen] self.timeout = timeout self.timeout_exceeded = False + self.terminated_by_sigterm = False + self.terminated_by_sigkill = False + self.pid = 0 def _check_timeout(self) -> None: if self.timeout is None: return sleep(self.timeout) + logging.warning( + "Timeout exceeded. Send SIGTERM to process %s, timeout %s", + self.process.pid, + self.timeout, + ) + self.send_signal(signal.SIGTERM) + time_wait = 0 + self.terminated_by_sigterm = True self.timeout_exceeded = True + while self.process.poll() is None and time_wait < 100: + print("wait...") + wait = 5 + sleep(wait) + time_wait += wait while self.process.poll() is None: - logging.warning( - "Killing process %s, timeout %s exceeded", - self.process.pid, - self.timeout, + logging.error( + "Process is still running. Send SIGKILL", ) - os.killpg(self.process.pid, 9) - sleep(10) + self.send_signal(signal.SIGKILL) + self.terminated_by_sigkill = True + sleep(5) def __enter__(self) -> "TeePopen": self.process = Popen( @@ -57,6 +74,9 @@ class TeePopen: bufsize=1, errors="backslashreplace", ) + sleep(1) + self.pid = self._get_child_pid() + print(f"Subprocess started, pid [{self.process.pid}], child pid [{self.pid}]") if self.timeout is not None and self.timeout > 0: t = Thread(target=self._check_timeout) t.daemon = True # does not block the program from exit @@ -77,6 +97,22 @@ class TeePopen: self.log_file.close() + def _get_child_pid(self): + # linux only + ps_command = f"ps --ppid {self.process.pid} -o pid=" + res = "NA" + try: + result = subprocess.run( + ps_command, shell=True, capture_output=True, text=True + ) + res = result.stdout.strip() + pid = int(res) + return pid + except Exception as e: + print(f"Failed to get child's pid, command [{ps_command}], result [{res}]") + print(f"ERROR: getting Python subprocess PID: {e}") + return self.process.pid + def wait(self) -> int: if self.process.stdout is not None: for line in self.process.stdout: @@ -85,6 +121,15 @@ class TeePopen: return self.process.wait() + def poll(self): + return self.process.poll() + + def send_signal(self, signal_num): + if self.pid: + os.kill(self.pid, signal_num) + else: + print("ERROR: no process to send signal") + @property def process(self) -> Popen: if self._process is not None: From 8e35b082b2e3315655110bdce4238217dfe85914 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Sat, 10 Aug 2024 10:01:16 +0200 Subject: [PATCH 1118/1722] teepopen fix --- tests/ci/tee_popen.py | 26 ++------------------------ 1 file changed, 2 insertions(+), 24 deletions(-) diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index ad3e62dab9c..53b0a0f6c2c 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -3,7 +3,6 @@ import logging import os import signal -import subprocess import sys from io import TextIOWrapper from pathlib import Path @@ -34,7 +33,6 @@ class TeePopen: self.timeout_exceeded = False self.terminated_by_sigterm = False self.terminated_by_sigkill = False - self.pid = 0 def _check_timeout(self) -> None: if self.timeout is None: @@ -75,8 +73,7 @@ class TeePopen: errors="backslashreplace", ) sleep(1) - self.pid = self._get_child_pid() - print(f"Subprocess started, pid [{self.process.pid}], child pid [{self.pid}]") + print(f"Subprocess started, pid [{self.process.pid}]") if self.timeout is not None and self.timeout > 0: t = Thread(target=self._check_timeout) t.daemon = True # does not block the program from exit @@ -97,22 +94,6 @@ class TeePopen: self.log_file.close() - def _get_child_pid(self): - # linux only - ps_command = f"ps --ppid {self.process.pid} -o pid=" - res = "NA" - try: - result = subprocess.run( - ps_command, shell=True, capture_output=True, text=True - ) - res = result.stdout.strip() - pid = int(res) - return pid - except Exception as e: - print(f"Failed to get child's pid, command [{ps_command}], result [{res}]") - print(f"ERROR: getting Python subprocess PID: {e}") - return self.process.pid - def wait(self) -> int: if self.process.stdout is not None: for line in self.process.stdout: @@ -125,10 +106,7 @@ class TeePopen: return self.process.poll() def send_signal(self, signal_num): - if self.pid: - os.kill(self.pid, signal_num) - else: - print("ERROR: no process to send signal") + os.killpg(self.process.pid, signal_num) @property def process(self) -> Popen: From 66fa5a154a8895f481a598616df93f7cb83e42cd Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 12 Aug 2024 02:34:22 +0200 Subject: [PATCH 1119/1722] tune timeouts, batches --- tests/ci/ci_config.py | 7 ++++--- tests/ci/ci_definitions.py | 1 + 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 173c6c9c931..99f4ed38475 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -316,6 +316,7 @@ class CI: JobNames.STATEFUL_TEST_PARALLEL_REPL_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], random_bucket="parrepl_with_sanitizer", + timeout=3600, ), JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 @@ -346,7 +347,7 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 ), JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=1 ), JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], num_batches=3, release_only=True @@ -401,14 +402,14 @@ class CI: required_builds=[BuildNames.PACKAGE_ASAN], release_only=True, num_batches=4 ), JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: CommonJobConfigs.INTEGRATION_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=6 + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 ), JobNames.INTEGRATION_TEST_TSAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 ), JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], - num_batches=6, + num_batches=3, runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 1d1c39f913d..13c222b10b9 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -465,6 +465,7 @@ class CommonJobConfigs: ), run_command="upgrade_check.py", runner_type=Runners.STRESS_TESTER, + timeout=3600, ) INTEGRATION_TEST = JobConfig( job_name_keyword="integration", From 1deeca40dbbbc14373e51d830b851b54b82e5efa Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 15 Aug 2024 13:11:10 +0200 Subject: [PATCH 1120/1722] Handling timeout in integration tests --- tests/ci/ci.py | 13 ++++++++- tests/ci/ci_config.py | 3 ++- tests/ci/integration_test_check.py | 2 +- tests/ci/integration_tests_runner.py | 40 +++++++++++++++++++++++++++- 4 files changed, 54 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 1208d8642ad..a9ae078b449 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1003,6 +1003,7 @@ def _run_test(job_name: str, run_command: str) -> int: ) jr.test_results = [TestResult.create_check_timeout_expired()] jr.duration = stopwatch.duration_seconds + jr.additional_files += [job_log] print(f"Run action done for: [{job_name}]") return retcode @@ -1329,10 +1330,20 @@ def main() -> int: if CI.is_test_job(args.job_name): gh = GitHub(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) + check_url = "" + if job_report.test_results or job_report.additional_files: + check_url = upload_result_helper.upload_results( + s3, + pr_info.number, + pr_info.sha, + job_report.test_results, + job_report.additional_files, + job_report.check_name or _get_ext_check_name(args.job_name), + ) post_commit_status( commit, ERROR, - "", + check_url, "Error: " + error_description, _get_ext_check_name(args.job_name), pr_info, diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 99f4ed38475..b5e424c2b3f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -402,7 +402,8 @@ class CI: required_builds=[BuildNames.PACKAGE_ASAN], release_only=True, num_batches=4 ), JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: CommonJobConfigs.INTEGRATION_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 + required_builds=[BuildNames.PACKAGE_ASAN], + num_batches=3, ), JobNames.INTEGRATION_TEST_TSAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 6245f0490fc..7232ca375a1 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -29,7 +29,7 @@ from stopwatch import Stopwatch import integration_tests_runner as runner from ci_config import CI -from ci_utils import Utils +from ci_utils import Utils, Shell def get_json_params_dict( diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index f5dbef4f6db..d3cd3d16de1 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -9,6 +9,7 @@ import random import re import shlex import shutil +import signal import string import subprocess import sys @@ -16,11 +17,13 @@ import time import zlib # for crc32 from collections import defaultdict from itertools import chain -from typing import Any, Dict +from typing import Any, Dict, Optional from env_helper import IS_CI from integration_test_images import IMAGES from tee_popen import TeePopen +from report import JOB_TIMEOUT_TEST_NAME +from stopwatch import Stopwatch MAX_RETRY = 1 NUM_WORKERS = 5 @@ -621,6 +624,9 @@ class ClickhouseIntegrationTestsRunner: test_data_dirs = {} for i in range(num_tries): + if timeout_expired: + print("Timeout expired - break test group execution") + break logging.info("Running test group %s for the %s retry", test_group, i) clear_ip_tables_and_restart_daemons() @@ -657,6 +663,8 @@ class ClickhouseIntegrationTestsRunner: logging.info("Executing cmd: %s", cmd) # ignore retcode, since it meaningful due to pipe to tee with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as proc: + global runner_subprocess + runner_subprocess = proc proc.wait() extra_logs_names = [log_basename] @@ -780,6 +788,9 @@ class ClickhouseIntegrationTestsRunner: logs = [] tries_num = 1 if should_fail else FLAKY_TRIES_COUNT for i in range(tries_num): + if timeout_expired: + print("Timeout expired - break flaky check execution") + break final_retry += 1 logging.info("Running tests for the %s time", i) counters, tests_times, log_paths = self.try_run_test_group( @@ -839,6 +850,7 @@ class ClickhouseIntegrationTestsRunner: return result_state, status_text, test_result, logs def run_impl(self, repo_path, build_path): + stopwatch = Stopwatch() if self.flaky_check or self.bugfix_validate_check: return self.run_flaky_check( repo_path, build_path, should_fail=self.bugfix_validate_check @@ -921,6 +933,9 @@ class ClickhouseIntegrationTestsRunner: random.shuffle(items_to_run) for group, tests in items_to_run: + if timeout_expired: + print("Timeout expired - break tests execution") + break logging.info("Running test group %s containing %s tests", group, len(tests)) group_counters, group_test_times, log_paths = self.try_run_test_group( repo_path, group, tests, MAX_RETRY, NUM_WORKERS, 0 @@ -981,6 +996,17 @@ class ClickhouseIntegrationTestsRunner: status_text = "Timeout, " + status_text result_state = "failure" + if timeout_expired: + logging.error( + "Job killed by external timeout signal - setting status to failure!" + ) + status_text = "Job timeout expired, " + status_text + result_state = "failure" + # add mock test case to make timeout visible in job report and in ci db + test_result.insert( + 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") + ) + if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" @@ -1001,6 +1027,7 @@ def write_results(results_file, status_file, results, status): def run(): + signal.signal(signal.SIGTERM, handle_sigterm) logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") repo_path = os.environ.get("CLICKHOUSE_TESTS_REPO_PATH") @@ -1035,5 +1062,16 @@ def run(): logging.info("Result written") +timeout_expired = False +runner_subprocess = None # type:Optional[subprocess.Popen] + + +def handle_sigterm(signum, _frame): + print(f"WARNING: Received signal {signum}") + global timeout_expired + timeout_expired = True + runner_subprocess.send_signal(signal.SIGTERM) + + if __name__ == "__main__": run() From dde7ee29fc594f87bb35880bede845c4d4f29423 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 16 Aug 2024 10:22:12 +0200 Subject: [PATCH 1121/1722] sort tests in report by status --- tests/ci/ci_config.py | 8 ++++---- tests/ci/integration_test_check.py | 2 +- tests/ci/integration_tests_runner.py | 3 ++- tests/ci/report.py | 19 +++++++++++++++---- 4 files changed, 22 insertions(+), 10 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b5e424c2b3f..8ce0b9fde5a 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -344,7 +344,7 @@ class CI: runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=2 ), JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=1 @@ -354,7 +354,7 @@ class CI: ), JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], - num_batches=4, + num_batches=3, ), JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], @@ -403,14 +403,14 @@ class CI: ), JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], - num_batches=3, + num_batches=6, ), JobNames.INTEGRATION_TEST_TSAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 ), JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], - num_batches=3, + num_batches=6, runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 7232ca375a1..6245f0490fc 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -29,7 +29,7 @@ from stopwatch import Stopwatch import integration_tests_runner as runner from ci_config import CI -from ci_utils import Utils, Shell +from ci_utils import Utils def get_json_params_dict( diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index d3cd3d16de1..c3b71b85022 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -1070,7 +1070,8 @@ def handle_sigterm(signum, _frame): print(f"WARNING: Received signal {signum}") global timeout_expired timeout_expired = True - runner_subprocess.send_signal(signal.SIGTERM) + if runner_subprocess: + runner_subprocess.send_signal(signal.SIGTERM) if __name__ == "__main__": diff --git a/tests/ci/report.py b/tests/ci/report.py index c2632719aef..a1b25b994c7 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -742,10 +742,21 @@ def create_test_html_report( has_test_time = any(tr.time is not None for tr in test_results) has_log_urls = False - # Display entires with logs at the top (they correspond to failed tests) - test_results.sort( - key=lambda result: result.raw_logs is None and result.log_files is None - ) + def sort_key(status): + if "fail" in status.lower(): + return 0 + elif "error" in status.lower(): + return 1 + elif "not" in status.lower(): + return 2 + elif "ok" in status.lower(): + return 10 + elif "success" in status.lower(): + return 9 + else: + return 5 + + test_results.sort(key=lambda result: sort_key(result.status)) for test_result in test_results: colspan = 0 From 29fd5a6c90caeafee8bc930918f7f5544a1658a2 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Sun, 18 Aug 2024 15:10:35 +0000 Subject: [PATCH 1122/1722] Add explicit session_timezone to UTC --- tests/queries/0_stateless/03222_datetime64_small_value_const.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql index af06a622f8d..39266ba7992 100644 --- a/tests/queries/0_stateless/03222_datetime64_small_value_const.sql +++ b/tests/queries/0_stateless/03222_datetime64_small_value_const.sql @@ -1,4 +1,5 @@ -- Tags: shard +set session_timezone = 'UTC'; -- don't randomize the session timezone select *, (select toDateTime64(0, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; select *, (select toDateTime64(5, 3)) from remote('127.0.0.1', system.one) settings prefer_localhost_replica=0; From 90330077e5595c000cec82c8a0819db339296d33 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 18 Aug 2024 17:56:44 +0000 Subject: [PATCH 1123/1722] fix test --- .../queries/0_stateless/03221_refreshable_matview_progress.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql index ecb385c9bfa..98e1c48478d 100644 --- a/tests/queries/0_stateless/03221_refreshable_matview_progress.sql +++ b/tests/queries/0_stateless/03221_refreshable_matview_progress.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database +-- Tags: no-replicated-database, no-ordinary-database set allow_experimental_refreshable_materialized_view=1; From 683b84e6b66edbfbba12a3c56aad9aefd717bde2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 18 Aug 2024 19:37:52 +0100 Subject: [PATCH 1124/1722] fix --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 4 ++++ src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 4 ---- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index cc321cd5a4d..9927d369104 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -235,6 +235,10 @@ void MergeTreeReadPool::fillPerThreadInfo(size_t threads, size_t sum_marks) const auto part_idx = current_parts.back().part_idx; const auto min_marks_per_task = per_part_infos[part_idx]->min_marks_per_task; + if (min_marks_per_task == 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + /// Do not get too few rows from part. if (marks_in_part >= min_marks_per_task && need_marks < min_marks_per_task) need_marks = min_marks_per_task; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 9d3c38822e1..1cc13102794 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -87,10 +87,6 @@ static size_t calculateMinMarksPerTask( } } - if (min_marks_per_task == 0) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); - LOG_TEST(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task); return min_marks_per_task; } From 3883627aad0e23105ec9e1f985039cd27dcf227d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 18 Aug 2024 20:51:40 +0100 Subject: [PATCH 1125/1722] fix --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 1 + src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 9927d369104..23c314e48f5 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes { extern const int CANNOT_SCHEDULE_TASK; extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; } MergeTreeReadPool::MergeTreeReadPool( diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 1cc13102794..95a10454f9e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } MergeTreeReadPoolBase::MergeTreeReadPoolBase( From a258b4fb3dcdb8fa2484132aa72e9cece618d488 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 22:18:31 +0200 Subject: [PATCH 1126/1722] Fix race condition in MergeTree restarting thread --- .../ReplicatedMergeTreeRestartingThread.cpp | 15 ++++++++++++++- .../ReplicatedMergeTreeRestartingThread.h | 11 ++--------- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++------ 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 05fd6f6915b..d3ccda904b6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -49,6 +48,20 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); }); } +void ReplicatedMergeTreeRestartingThread::start(bool schedule) +{ + LOG_TRACE(log, "Starting the restating thread, schedule: {}", schedule); + if (schedule) + task->activateAndSchedule(); + else + task->activate(); +} + +void ReplicatedMergeTreeRestartingThread::wakeup() +{ + task->schedule(); +} + void ReplicatedMergeTreeRestartingThread::run() { if (need_stop) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 01071d80e8b..d719505ae5e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -24,16 +24,9 @@ class ReplicatedMergeTreeRestartingThread public: explicit ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_); - void start(bool schedule = true) - { - LOG_TRACE(log, "Starting restating thread, schedule: {}", schedule); - if (schedule) - task->activateAndSchedule(); - else - task->activate(); - } + void start(bool schedule); - void wakeup() { task->schedule(); } + void wakeup(); void shutdown(bool part_of_full_shutdown); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 068ff1387b3..ff8e362aa36 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5194,17 +5194,16 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) startBeingLeader(); - /// Activate replica in a separate thread if we are not calling from attach thread - restarting_thread.start(/*schedule=*/!from_attach_thread); - if (from_attach_thread) { LOG_TRACE(log, "Trying to startup table from right now"); - /// Try activating replica in current thread. + /// Try activating replica in the current thread. restarting_thread.run(); + restarting_thread.start(false); } else { + restarting_thread.start(true); /// Wait while restarting_thread finishing initialization. /// NOTE It does not mean that replication is actually started after receiving this event. /// It only means that an attempt to startup replication was made. @@ -5225,7 +5224,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() { LOG_TEST(log, "Received event for expired session. Waking up restarting thread"); - restarting_thread.start(); + restarting_thread.start(true); }); startBackgroundMovesIfNeeded(); @@ -5294,7 +5293,6 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() LOG_TRACE(log, "The attach thread is shutdown"); } - restarting_thread.shutdown(/* part_of_full_shutdown */true); /// Explicitly set the event, because the restarting thread will not set it again startup_event.set(); From 9d183b890529d686a673d5f6ecd2d057cf534478 Mon Sep 17 00:00:00 2001 From: avogar Date: Sun, 18 Aug 2024 20:35:30 +0000 Subject: [PATCH 1127/1722] Add check for loo large array size --- .../AggregateFunctionDistinctDynamicTypes.cpp | 6 ++++++ .../AggregateFunctionDistinctJSONPaths.cpp | 13 +++++++++++++ src/Core/Settings.h | 1 - 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 649d64a3904..17e32b20a99 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -19,10 +19,13 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; } struct AggregateFunctionDistinctDynamicTypesData { + constexpr static size_t MAX_ARRAY_SIZE = 0xFFFFFF; + std::unordered_set data; void add(const String & type) @@ -46,6 +49,9 @@ struct AggregateFunctionDistinctDynamicTypesData { size_t size; readVarUInt(size, buf); + if (size > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", MAX_ARRAY_SIZE, size); + data.reserve(size); String type; for (size_t i = 0; i != size; ++i) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 58ce7e27ab0..6100bd57515 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -23,8 +23,12 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; } +constexpr static size_t DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE = 0xFFFFFF; + + struct AggregateFunctionDistinctJSONPathsData { static constexpr auto name = "distinctJSONPaths"; @@ -85,6 +89,9 @@ struct AggregateFunctionDistinctJSONPathsData { size_t size; readVarUInt(size, buf); + if (size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, size); + String path; for (size_t i = 0; i != size; ++i) { @@ -192,12 +199,18 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData { size_t paths_size, types_size; readVarUInt(paths_size, buf); + if (paths_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); + data.reserve(paths_size); String path, type; for (size_t i = 0; i != paths_size; ++i) { readStringBinary(path, buf); readVarUInt(types_size, buf); + if (types_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); + data[path].reserve(types_size); for (size_t j = 0; j != types_size; ++j) { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d84ad9022a..dfcff052740 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,6 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From 3e5d070b8fb47600979a1a7cc672edc3a7327004 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 22:40:42 +0200 Subject: [PATCH 1128/1722] Fix tests --- .../00804_test_deflate_qpl_codec_compression.reference | 2 +- .../00804_test_zstd_qat_codec_compression.reference | 2 +- .../03227_print_pretty_tuples_create_query.reference | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference index a2178f5eda7..a6e03404f2b 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(DEFLATE_QPL),\n `data` String CODEC(DEFLATE_QPL),\n `ddd` Date CODEC(DEFLATE_QPL),\n `ddd32` Date32 CODEC(DEFLATE_QPL),\n `somenum` Float64 CODEC(DEFLATE_QPL),\n `somestr` FixedString(3) CODEC(DEFLATE_QPL),\n `othernum` Int64 CODEC(DEFLATE_QPL),\n `somearray` Array(UInt8) CODEC(DEFLATE_QPL),\n `somemap` Map(String, UInt32) CODEC(DEFLATE_QPL),\n `sometuple` Tuple(UInt16, UInt64) CODEC(DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(DEFLATE_QPL),\n `data` String CODEC(DEFLATE_QPL),\n `ddd` Date CODEC(DEFLATE_QPL),\n `ddd32` Date32 CODEC(DEFLATE_QPL),\n `somenum` Float64 CODEC(DEFLATE_QPL),\n `somestr` FixedString(3) CODEC(DEFLATE_QPL),\n `othernum` Int64 CODEC(DEFLATE_QPL),\n `somearray` Array(UInt8) CODEC(DEFLATE_QPL),\n `somemap` Map(String, UInt32) CODEC(DEFLATE_QPL),\n `sometuple` Tuple(\n UInt16,\n UInt64) CODEC(DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2) 2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4) 3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6) diff --git a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference index 31a4360469f..ff70403ce7a 100644 --- a/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_zstd_qat_codec_compression.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(ZSTD_QAT(1)),\n `data` String CODEC(ZSTD_QAT(1)),\n `ddd` Date CODEC(ZSTD_QAT(1)),\n `ddd32` Date32 CODEC(ZSTD_QAT(1)),\n `somenum` Float64 CODEC(ZSTD_QAT(1)),\n `somestr` FixedString(3) CODEC(ZSTD_QAT(1)),\n `othernum` Int64 CODEC(ZSTD_QAT(1)),\n `somearray` Array(UInt8) CODEC(ZSTD_QAT(1)),\n `somemap` Map(String, UInt32) CODEC(ZSTD_QAT(1)),\n `sometuple` Tuple(UInt16, UInt64) CODEC(ZSTD_QAT(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(ZSTD_QAT(1)),\n `data` String CODEC(ZSTD_QAT(1)),\n `ddd` Date CODEC(ZSTD_QAT(1)),\n `ddd32` Date32 CODEC(ZSTD_QAT(1)),\n `somenum` Float64 CODEC(ZSTD_QAT(1)),\n `somestr` FixedString(3) CODEC(ZSTD_QAT(1)),\n `othernum` Int64 CODEC(ZSTD_QAT(1)),\n `somearray` Array(UInt8) CODEC(ZSTD_QAT(1)),\n `somemap` Map(String, UInt32) CODEC(ZSTD_QAT(1)),\n `sometuple` Tuple(\n UInt16,\n UInt64) CODEC(ZSTD_QAT(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2) 2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4) 3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6) diff --git a/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference b/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference index c65dc32a224..afaaaaa6119 100644 --- a/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference +++ b/tests/queries/0_stateless/03227_print_pretty_tuples_create_query.reference @@ -1,6 +1,6 @@ SHOW CREATE TABLE: -CREATE TABLE test.test +CREATE TABLE default.test ( `x` Tuple( a String, @@ -13,7 +13,7 @@ CREATE TABLE test.test ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192 -CREATE TABLE test.test +CREATE TABLE default.test ( `x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), `y` String From 8f2c20806a7b757beecb99e52a8d5a1dcab8df07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 22:45:13 +0200 Subject: [PATCH 1129/1722] Fix test `01079_bad_alters_zookeeper_long` --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- tests/clickhouse-test | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 068ff1387b3..66dac0dfe31 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6342,7 +6342,7 @@ void StorageReplicatedMergeTree::alter( "Metadata on replica is not up to date with common metadata in Zookeeper. " "It means that this replica still not applied some of previous alters." " Probably too many alters executing concurrently (highly not recommended). " - "You can retry this error"); + "You can retry the query"); /// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level. if (query_context->getZooKeeperMetadataTransaction()) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3d7e0e922d..1203ad3730a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -59,6 +59,7 @@ MESSAGES_TO_RETRY = [ "is already started to be removing by another replica right now", # This is from LSan, and it indicates its own internal problem: "Unable to get registers from thread", + "You can retry", ] MAX_RETRIES = 3 From 207ef87782eca80410ed5747d971be7ddce65e6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 22:55:38 +0200 Subject: [PATCH 1130/1722] Fix tests --- tests/queries/0_stateless/01825_new_type_json_ghdata.sh | 3 ++- .../0_stateless/01825_type_json_ghdata_insert_select.sh | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh index fbd7d897fb8..b2f20d825dd 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-s3-storage +# ^ no-s3-storage: too memory hungry CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh index 711194e71a1..17398d9a0c1 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-s3-storage +# ^ no-s3-storage: too memory hungry CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 1049e366534635a510e9f0c769b5635a073a0c1c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 18 Aug 2024 23:35:20 +0200 Subject: [PATCH 1131/1722] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 19c8b2f084e..2465222ae6c 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -30,8 +30,10 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (result_part_name != result_part_info.getPartNameV1()) format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; + size_t normal_parts_count = 0; for (const auto & source_part : future_part->parts) { + normal_parts_count += !source_part->getParentPart(); if (!source_part->getParentPart() && !result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) throw Exception(ErrorCodes::LOGICAL_ERROR, "Source part {} is not covered by result part {}", source_part->name, result_part_info.getPartNameV1()); @@ -54,7 +56,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta part->partition.serializeText(part->storage, out, {}); } - if (is_mutation && future_part->parts.size() != 1) + if (is_mutation && normal_parts_count != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}", future_part->parts.size(), result_part_info.getPartNameV1()); thread_group = ThreadGroup::createForBackgroundProcess(context); From 5ed3c29d4a1e1992671af7168c0a1b01757d97bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2024 01:18:12 +0200 Subject: [PATCH 1132/1722] Update tests --- tests/queries/0_stateless/01825_new_type_json_ghdata.sh | 2 +- .../queries/0_stateless/01825_type_json_ghdata_insert_select.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh index b2f20d825dd..33940caec29 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-s3-storage +# Tags: no-fasttest, no-s3-storage, long # ^ no-s3-storage: too memory hungry CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh index 17398d9a0c1..fc503b345d9 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-s3-storage +# Tags: no-fasttest, no-s3-storage, long # ^ no-s3-storage: too memory hungry CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 8eb922036e5b7caa36c1b904b43fdaee8e45acaa Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 19 Aug 2024 12:45:15 +0800 Subject: [PATCH 1133/1722] change as request --- .../functions/string-replace-functions.md | 16 +- src/Functions/overlay.cpp | 140 +++++------ .../0_stateless/03205_overlay.reference | 230 +++++------------- tests/queries/0_stateless/03205_overlay.sql | 132 +++++----- .../0_stateless/03206_overlay_utf8.reference | 168 ------------- .../0_stateless/03206_overlay_utf8.sql | 60 ----- 6 files changed, 203 insertions(+), 543 deletions(-) delete mode 100644 tests/queries/0_stateless/03206_overlay_utf8.reference delete mode 100644 tests/queries/0_stateless/03206_overlay_utf8.sql diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index d086c9ee64b..408a896e607 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -251,24 +251,24 @@ select printf('%%%s %s %d', 'Hello', 'World', 2024); ## overlay -Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. +Replace the string `s` with the string `replace` starting from the 1-based `offset` for `length` bytes. If `length` is omitted or negative, then it defaults to the length of `replace`. **Syntax** ```sql -overlay(s, replace, position[, length]) +overlay(s, replace, offset[, length]) ``` **Parameters** - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int-uint.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). - `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** -- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. +- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. **Example** @@ -286,26 +286,26 @@ Result: ## overlayUTF8 -Replace the string `s` with the string `replace` starting from the 1-based `position` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. +Replace the string `s` with the string `replace` starting from the 1-based `offset` for `length` UTF-8 characters. If `length` is omitted or negative, then it defaults to the length of `replace`. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. **Syntax** ```sql -overlayUTF8(s, replace, position[, length]) +overlayUTF8(s, replace, offset[, length]) ``` **Parameters** - `s`: A string type [String](../data-types/string.md). - `replace`: A string type [String](../data-types/string.md). -- `position`: An integer type [Int](../data-types/int-uint.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). - `length`: Optional. An integer type [Int](../data-types/int-uint.md). **Returned value** -- A [String](../data-types/string.md) data type value. If `position` is negative the position is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. +- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. **Example** diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 094da27a71d..73ca0acbb8e 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -17,13 +17,13 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -using namespace GatherUtils; - namespace { /// If 'is_utf8' - measure offset and length in code points instead of bytes. -/// Syntax: overlay(input, replace, offset[, length]) +/// Syntax: +/// - overlay(input, replace, offset[, length]) +/// - overlayUTF8(input, replace, offset[, length]) - measure offset and length in code points instead of bytes template class FunctionOverlay : public IFunction { @@ -37,63 +37,39 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - const size_t number_of_arguments = arguments.size(); - if (number_of_arguments < 3 || number_of_arguments > 4) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: " - "passed {}, should be 3 or 4", - getName(), - number_of_arguments); + FunctionArgumentDescriptors mandatory_args{ + {"input", static_cast(&isString), nullptr, "String"}, + {"replace", static_cast(&isString), nullptr, "String"}, + {"offset", static_cast(&isNativeInteger), nullptr, "(U)Int8/16/32/64"}, + }; - /// first argument is string - if (!isString(arguments[0])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {}, expected String", - arguments[0]->getName(), - getName()); + FunctionArgumentDescriptors optional_args{ + {"length", static_cast(&isNativeInteger), nullptr, "(U)Int8/16/32/64"}, + }; - /// second argument is string - if (!isString(arguments[1])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}, expected String", - arguments[1]->getName(), - getName()); - - if (!isNativeNumber(arguments[2])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {}, expected (U)Int8|16|32|64", - arguments[2]->getName(), - getName()); - - if (number_of_arguments == 4 && !isNativeNumber(arguments[3])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}, expected (U)Int8|16|32|64", - arguments[3]->getName(), - getName()); + validateFunctionArguments(*this, arguments, mandatory_args, optional_args); return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (input_rows_count == 0) + return ColumnString::create(); + const size_t number_of_arguments = arguments.size(); - bool three_args = number_of_arguments == 3; + bool has_three_args = number_of_arguments == 3; ColumnPtr column_offset = arguments[2].column; ColumnPtr column_length; - if (!three_args) + if (!has_three_args) column_length = arguments[3].column; const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); const ColumnConst * column_length_const = nullptr; - if (!three_args) + if (!has_three_args) column_length_const = checkAndGetColumn(column_length.get()); bool offset_is_const = false; @@ -126,7 +102,7 @@ public: if (column_input_const) { StringRef input = column_input_const->getDataAt(0); - res_data.reserve(input.size * input_rows_count); + res_data.reserve((input.size + 1) * input_rows_count); } else { @@ -135,8 +111,8 @@ public: const auto * column_replace_const = checkAndGetColumn(column_replace.get()); const auto * column_replace_string = checkAndGetColumn(column_replace.get()); - bool input_is_const = column_input_const != nullptr; - bool replace_is_const = column_replace_const != nullptr; + bool input_is_const = (column_input_const != nullptr); + bool replace_is_const = (column_replace_const != nullptr); #define OVERLAY_EXECUTE_CASE(THREE_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ if (input_is_const && replace_is_const) \ @@ -150,8 +126,9 @@ public: length, \ res_data, \ res_offsets); \ - else if (input_is_const) \ + else if (input_is_const && !replace_is_const) \ constantVector( \ + input_rows_count, \ column_input_const->getDataAt(0), \ column_replace_string->getChars(), \ column_replace_string->getOffsets(), \ @@ -161,8 +138,9 @@ public: length, \ res_data, \ res_offsets); \ - else if (replace_is_const) \ + else if (!input_is_const && replace_is_const) \ vectorConstant( \ + input_rows_count, \ column_input_string->getChars(), \ column_input_string->getOffsets(), \ column_replace_const->getDataAt(0), \ @@ -174,6 +152,7 @@ public: res_offsets); \ else \ vectorVector( \ + input_rows_count, \ column_input_string->getChars(), \ column_input_string->getOffsets(), \ column_replace_string->getChars(), \ @@ -185,7 +164,7 @@ public: res_data, \ res_offsets); - if (three_args) + if (has_three_args) { if (offset_is_const) { @@ -251,7 +230,7 @@ private: return bytes; } - template + template void constantConstant( size_t rows, const StringRef & input, @@ -263,7 +242,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { constantConstant( rows, input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -277,12 +256,12 @@ private: size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (three_args) + else if constexpr (has_three_args) { valid_length = replace_size; } @@ -300,7 +279,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!three_args && !length_is_const) + if constexpr (!has_three_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; @@ -331,10 +310,10 @@ private: } else { - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator @@ -363,8 +342,9 @@ private: } } - template + template void vectorConstant( + size_t rows, const ColumnString::Chars & input_data, const ColumnString::Offsets & input_offsets, const StringRef & replace, @@ -375,27 +355,26 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { vectorConstant( - input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); + rows, input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (three_args) + else if constexpr (has_three_args) { valid_length = replace_size; } - size_t rows = input_offsets.size(); Int64 offset = 0; // start from 1, maybe negative size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; @@ -415,7 +394,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!three_args && !length_is_const) + if constexpr (!has_three_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; @@ -449,9 +428,9 @@ private: { const auto * input_begin = &input_data[input_offset]; const auto * input_end = &input_data[input_offset + input_bytes]; - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace.size + suffix_bytes + 1; /// +1 for zero terminator @@ -480,8 +459,9 @@ private: } } - template + template void constantVector( + size_t rows, const StringRef & input, const ColumnString::Chars & replace_data, const ColumnString::Offsets & replace_offsets, @@ -492,10 +472,10 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { constantVector( - input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); + rows, input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); return; } @@ -506,13 +486,12 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - size_t rows = replace_offsets.size(); const auto * input_begin = reinterpret_cast(input.data); const auto * input_end = reinterpret_cast(input.data + input.size); Int64 offset = 0; // start from 1, maybe negative @@ -529,7 +508,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (three_args) + if constexpr (has_three_args) { valid_length = replace_size; } @@ -564,9 +543,9 @@ private: } else { - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input.size : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator res_data.resize(new_res_size); @@ -594,8 +573,9 @@ private: } } - template + template void vectorVector( + size_t rows, const ColumnString::Chars & input_data, const ColumnString::Offsets & input_offsets, const ColumnString::Chars & replace_data, @@ -607,9 +587,10 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!three_args && length_is_const && const_length < 0) + if (!has_three_args && length_is_const && const_length < 0) { vectorVector( + rows, input_data, input_offsets, replace_data, @@ -625,13 +606,12 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!three_args && length_is_const) + if constexpr (!has_three_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - size_t rows = input_offsets.size(); Int64 offset = 0; // start from 1, maybe negative size_t valid_offset = 0; // start from 0, not negative size_t res_offset = 0; @@ -655,7 +635,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (three_args) + if constexpr (has_three_args) { valid_length = replace_size; } @@ -693,9 +673,9 @@ private: { const auto * input_begin = &input_data[input_offset]; const auto * input_end = &input_data[input_offset + input_bytes]; - const auto * prefix_end = UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); + const auto * prefix_end = GatherUtils::UTF8StringSource::skipCodePointsForward(input_begin, prefix_size, input_end); size_t prefix_bytes = prefix_end > input_end ? input_bytes : prefix_end - input_begin; - const auto * suffix_begin = UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); + const auto * suffix_begin = GatherUtils::UTF8StringSource::skipCodePointsBackward(input_end, suffix_size, input_begin); size_t suffix_bytes = input_end - suffix_begin; size_t new_res_size = res_data.size() + prefix_bytes + replace_bytes + suffix_bytes + 1; /// +1 for zero terminator res_data.resize(new_res_size); diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference index 9e79db2e131..383a26986d6 100644 --- a/tests/queries/0_stateless/03205_overlay.reference +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -1,168 +1,62 @@ -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark_SQL -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark CORE -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Spark ANSI SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL -Structured SQL +Negative test of overlay +Positive test 1 with various combinations of const/non-const columns +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Positive test 2 with various combinations of const/non-const columns +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Spark_SQL Spark_SQLå’ŒCH +Positive test 3 with various combinations of const/non-const columns +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Spark CORE Spark CORECH +Positive test 4 with various combinations of const/non-const columns +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Spark ANSI SQL Spark ANSI SQLå’ŒCH +Positive test 5 with various combinations of const/non-const columns +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH +Structured SQL Structured SQLå’ŒCH diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql index b131312c934..4fd0791521d 100644 --- a/tests/queries/0_stateless/03205_overlay.sql +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -1,60 +1,74 @@ -SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); +SELECT 'Negative test of overlay'; +SELECT overlay('hello', 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT overlay('hello', 'world'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT overlay('hello', 'world', 2, 3, 'extra'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT overlay(123, 'world', 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT overlay('hello', 456, 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT overlay('hello', 'world', 'two', 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT overlay('hello', 'world', 2, 'three'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT overlay('Spark SQL', '_', 6) from numbers(3); -SELECT overlay(materialize('Spark SQL'), '_', 6) from numbers(3); -SELECT overlay('Spark SQL', materialize('_'), 6) from numbers(3); -SELECT overlay('Spark SQL', '_', materialize(6)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('_'), 6) from numbers(3); -SELECT overlay(materialize('Spark SQL'), '_', materialize(6)) from numbers(3); -SELECT overlay('Spark SQL', materialize('_'), materialize(6)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)) from numbers(3); - -SELECT overlay('Spark SQL', 'CORE', 7) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'CORE', 7) from numbers(3); -SELECT overlay('Spark SQL', materialize('CORE'), 7) from numbers(3); -SELECT overlay('Spark SQL', 'CORE', materialize(7)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)) from numbers(3); -SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)) from numbers(3); - -SELECT overlay('Spark SQL', 'ANSI ', 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); - -SELECT overlay('Spark SQL', 'tructured', 2, 4) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4) from numbers(3); -SELECT overlay('Spark SQL', materialize('tructured'), 2, 4) from numbers(3); -SELECT overlay('Spark SQL', 'tructured', materialize(2), 4) from numbers(3); -SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4) from numbers(3); -SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4) from numbers(3); -SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)) from numbers(3); -SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)) from numbers(3); -SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); +SELECT 'Positive test 1 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'ANSI ', 7, 0), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, 0); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), 0); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, materialize(0)); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, materialize(0)); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), 0); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, materialize(0)); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), materialize(0)); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)); + +SELECT 'Positive test 2 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', '_', 6), overlayUTF8('Spark SQLå’ŒCH', '_', 6); +SELECT overlay(materialize('Spark SQL'), '_', 6), overlayUTF8(materialize('Spark SQLå’ŒCH'), '_', 6); +SELECT overlay('Spark SQL', materialize('_'), 6), overlayUTF8('Spark SQLå’ŒCH', materialize('_'), 6); +SELECT overlay('Spark SQL', '_', materialize(6)), overlayUTF8('Spark SQLå’ŒCH', '_', materialize(6)); +SELECT overlay(materialize('Spark SQL'), materialize('_'), 6), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('_'), 6); +SELECT overlay(materialize('Spark SQL'), '_', materialize(6)), overlayUTF8(materialize('Spark SQLå’ŒCH'), '_', materialize(6)); +SELECT overlay('Spark SQL', materialize('_'), materialize(6)), overlayUTF8('Spark SQLå’ŒCH', materialize('_'), materialize(6)); +SELECT overlay(materialize('Spark SQL'), materialize('_'), materialize(6)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('_'), materialize(6)); + +SELECT 'Positive test 3 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'CORE', 7), overlayUTF8('Spark SQLå’ŒCH', 'CORE', 7); +SELECT overlay(materialize('Spark SQL'), 'CORE', 7), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'CORE', 7); +SELECT overlay('Spark SQL', materialize('CORE'), 7), overlayUTF8('Spark SQLå’ŒCH', materialize('CORE'), 7); +SELECT overlay('Spark SQL', 'CORE', materialize(7)), overlayUTF8('Spark SQLå’ŒCH', 'CORE', materialize(7)); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), 7), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('CORE'), 7); +SELECT overlay(materialize('Spark SQL'), 'CORE', materialize(7)), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'CORE', materialize(7)); +SELECT overlay('Spark SQL', materialize('CORE'), materialize(7)), overlayUTF8('Spark SQLå’ŒCH', materialize('CORE'), materialize(7)); +SELECT overlay(materialize('Spark SQL'), materialize('CORE'), materialize(7)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('CORE'), materialize(7)); + +SELECT 'Positive test 4 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'ANSI ', 7, 0), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, 0), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, 0); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), 0), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), 0); +SELECT overlay('Spark SQL', 'ANSI ', 7, materialize(0)), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), 7, 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), 0), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), 0); +SELECT overlay(materialize('Spark SQL'), 'ANSI ', 7, materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, materialize(0)); +SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), 0), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), 0); +SELECT overlay('Spark SQL', materialize('ANSI '), 7, materialize(0)), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, materialize(0)); +SELECT overlay('Spark SQL', 'ANSI ', materialize(7), materialize(0)), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), materialize(0)); +SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)); + +SELECT 'Positive test 5 with various combinations of const/non-const columns'; +SELECT overlay('Spark SQL', 'tructured', 2, 4), overlayUTF8('Spark SQLå’ŒCH', 'tructured', 2, 4); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, 4), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', 2, 4); +SELECT overlay('Spark SQL', materialize('tructured'), 2, 4), overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), 2, 4); +SELECT overlay('Spark SQL', 'tructured', materialize(2), 4), overlayUTF8('Spark SQLå’ŒCH', 'tructured', materialize(2), 4); +SELECT overlay('Spark SQL', 'tructured', 2, materialize(4)), overlayUTF8('Spark SQLå’ŒCH', 'tructured', 2, materialize(4)); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), 2, 4), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('tructured'), 2, 4); +SELECT overlay(materialize('Spark SQL'), 'tructured', materialize(2), 4), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', materialize(2), 4); +SELECT overlay(materialize('Spark SQL'), 'tructured', 2, materialize(4)), overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', 2, materialize(4)); +SELECT overlay('Spark SQL', materialize('tructured'), materialize(2), 4), overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), materialize(2), 4); +SELECT overlay('Spark SQL', materialize('tructured'), 2, materialize(4)), overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), 2, materialize(4)); +SELECT overlay('Spark SQL', 'tructured', materialize(2), materialize(4)), overlayUTF8('Spark SQLå’ŒCH', 'tructured', materialize(2), materialize(4)); +SELECT overlay(materialize('Spark SQL'), materialize('tructured'), materialize(2), materialize(4)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('tructured'), materialize(2), materialize(4)); diff --git a/tests/queries/0_stateless/03206_overlay_utf8.reference b/tests/queries/0_stateless/03206_overlay_utf8.reference deleted file mode 100644 index 19878c97184..00000000000 --- a/tests/queries/0_stateless/03206_overlay_utf8.reference +++ /dev/null @@ -1,168 +0,0 @@ -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark_SQLå’ŒCH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark CORECH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Spark ANSI SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH -Structured SQLå’ŒCH diff --git a/tests/queries/0_stateless/03206_overlay_utf8.sql b/tests/queries/0_stateless/03206_overlay_utf8.sql deleted file mode 100644 index 00b756c8b5b..00000000000 --- a/tests/queries/0_stateless/03206_overlay_utf8.sql +++ /dev/null @@ -1,60 +0,0 @@ -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); - -SELECT overlayUTF8('Spark SQLå’ŒCH', '_', 6) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), '_', 6) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('_'), 6) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', '_', materialize(6)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('_'), 6) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), '_', materialize(6)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('_'), materialize(6)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('_'), materialize(6)) from numbers(3); - -SELECT overlayUTF8('Spark SQLå’ŒCH', 'CORE', 7) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'CORE', 7) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('CORE'), 7) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'CORE', materialize(7)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('CORE'), 7) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'CORE', materialize(7)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('CORE'), materialize(7)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('CORE'), materialize(7)) from numbers(3); - -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), 7, 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', materialize(7), 0) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'ANSI ', 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), 0) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), 7, materialize(0)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', materialize(7), materialize(0)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)) from numbers(3); - -SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', 2, 4) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', 2, 4) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), 2, 4) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', materialize(2), 4) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('tructured'), 2, 4) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', materialize(2), 4) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), 'tructured', 2, materialize(4)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), materialize(2), 4) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', materialize('tructured'), 2, materialize(4)) from numbers(3); -SELECT overlayUTF8('Spark SQLå’ŒCH', 'tructured', materialize(2), materialize(4)) from numbers(3); -SELECT overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('tructured'), materialize(2), materialize(4)) from numbers(3); From 2a8c9b8518175d81632cf7ca48c10522b737e6b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2024 07:59:14 +0200 Subject: [PATCH 1134/1722] Fix tests --- .../0_stateless/01825_new_type_json_ghdata_insert_select.sh | 3 ++- tests/queries/0_stateless/01825_type_json_ghdata.sh | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh index 2afec5ba7fe..568ba2bd185 100755 --- a/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_new_type_json_ghdata_insert_select.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest, long +# Tags: no-fasttest, no-s3-storage, long +# ^ no-s3-storage: it is memory-hungry CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01825_type_json_ghdata.sh b/tests/queries/0_stateless/01825_type_json_ghdata.sh index 2686e2c8eb1..7e952de6c08 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-s3-storage, long +# ^ no-s3-storage: it is memory-hungry CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e623ad041f4937b0e7ed22f3159acfee6c0147b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Jul 2024 16:44:17 +0200 Subject: [PATCH 1135/1722] Make C-z ignorance configurable (ignore_shell_suspend) in clickhouse-client C-z is extermelly useful for some users (like myself), so provide a way to configure it in client and avoid it's ignorance in clickhouse-disks (I hope it is OK since it is not that known utility and it does not have it's own configuration, while cli option is useless, one should remeber about it). Honestly I've never seen any interactive client that forbids C-z, so ignoring it my default looks strange to me. Signed-off-by: Azat Khuzhin --- programs/client/clickhouse-client.xml | 3 +++ programs/disks/DisksApp.cpp | 1 + programs/disks/DisksApp.h | 2 +- programs/disks/DisksClient.cpp | 1 - programs/disks/DisksClient.h | 3 +-- programs/keeper-client/KeeperClient.cpp | 1 + src/Client/ClientBase.cpp | 1 + src/Client/ReplxxLineReader.cpp | 4 +++- src/Client/ReplxxLineReader.h | 1 + 9 files changed, 12 insertions(+), 5 deletions(-) diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index 9ce7d1cb223..6eb8976a6ef 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -53,6 +53,9 @@ --> + + + If --- src/Functions/LowerUpperUTF8Impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 8469bedde0c..36ee1723269 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -57,7 +57,7 @@ struct LowerUpperUTF8Impl input.toUTF8String(output); /// For valid UTF-8 input strings, ICU sometimes produces output with an extra '\0 at the end. Only the data before that - /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this + /// '\0' is valid. If the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this /// case, the behavior is also reasonable. size_t valid_size = output.size(); if (!output.empty() && output.back() == '\0') From 2f6ad1271cfbd9aa62ad2365e70314aba4da21b9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 00:27:02 +0200 Subject: [PATCH 1271/1722] fix tests + exception --- src/Storages/VirtualColumnUtils.cpp | 2 +- .../test_storage_azure_blob_storage/test.py | 10 +-- tests/integration/test_storage_hdfs/test.py | 9 +-- .../03203_hive_style_partitioning.reference | 2 - .../03203_hive_style_partitioning.sh | 61 +++---------------- 5 files changed, 15 insertions(+), 69 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index ca82a1ce67a..f0d276e4e56 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -162,7 +162,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return; if (storage_columns.size() == 1) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot implement partition by all columns in a file"); + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot use hive partitioning for file {}: it contains only partition columns. Disable use_hive_partitioning setting to read this file", path); auto local_type = storage_columns.get(name).type; storage_columns.remove(name); desc.addEphemeral(name, local_type, ""); diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 637dbd38262..a3172329a99 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1518,14 +1518,14 @@ def test_hive_partitioning_with_one_parameter(cluster): ) query = ( - f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " + f"SELECT column2, _file, _path, column1 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}')" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} ).splitlines() == [ - "Elizabeth\tGordon\tsample.csv\t{bucket}/{max_path}".format( + "Gordon\tsample.csv\t{bucket}/{max_path}\tElizabeth".format( bucket="cont", max_path=path ) ] @@ -1560,7 +1560,7 @@ def test_hive_partitioning_with_all_parameters(cluster): f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}');" ) - pattern = r"DB::Exception: Cannot implement partition by all columns in a file" + pattern = r"DB::Exception: Cannot use hive partitioning for file" with pytest.raises(Exception, match=pattern): azure_query(node, query, settings={"use_hive_partitioning": 1}) @@ -1572,7 +1572,7 @@ def test_hive_partitioning_without_setting(cluster): table_format = "column1 String, column2 String" values_1 = f"('Elizabeth', 'Gordon')" values_2 = f"('Emilia', 'Gregor')" - path = "a/column1=Elizabeth/column2=Gordon/sample.csv" + path = "a/column1=Elizabeth/column2=Gordon/column3=Gordon/sample.csv" azure_query( node, @@ -1582,7 +1582,7 @@ def test_hive_partitioning_without_setting(cluster): ) query = ( - f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " + f"SELECT column1, column2, _file, _path, column3 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " f"blob_path='{path}', format='CSV', structure='{table_format}');" ) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ad2e7084791..ea8c4efa745 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1268,11 +1268,6 @@ def test_hive_partitioning_with_one_parameter(started_cluster): ) assert r == f"Elizabeth\n" - r = node1.query( - "SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", - settings={"use_hive_partitioning": 1}, - ) - assert r == f"Gordon\n" def test_hive_partitioning_with_all_parameters(started_cluster): @@ -1285,11 +1280,11 @@ def test_hive_partitioning_with_all_parameters(started_cluster): == f"Elizabeth\tGordon\n" ) - pattern = r"DB::Exception: Cannot implement partition by all columns in a file" + pattern = r"DB::Exception: Cannot use hive partitioning for file" with pytest.raises(QueryRuntimeException, match=pattern): node1.query( - f"SELECT _column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", + f"SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", settings={"use_hive_partitioning": 1}, ) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index b5eaef7f51e..af52dcd9b88 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -35,8 +35,6 @@ Cross Elizabeth Array(Int64) LowCardinality(Float64) 101 2070 -4081 -2070 2070 b 1 diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 41b215578f0..4e165446c34 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -11,22 +11,10 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE FILE HIVE PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; SELECT number, date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; SELECT array, float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; @@ -37,7 +25,6 @@ SELECT count(*) FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01 $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT _identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') LIMIT 2; SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ @@ -61,21 +48,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE URL PARTITIONING'" $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; - -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM url('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM url('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10;""" @@ -92,24 +65,10 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3 PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; - -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Gordon/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = _column0; +SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; """ $CLICKHOUSE_CLIENT -n -q """ @@ -123,13 +82,7 @@ $CLICKHOUSE_LOCAL -q "SELECT 'TESTING THE S3CLUSTER PARTITIONING'" $CLICKHOUSE_CLIENT -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = _column0; - -SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column1 = _column1; - -SELECT *, _column0, _column1 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; -SELECT *, _column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/column1=Schmidt/sample.parquet') WHERE column0 = _column0 AND column1 = _column1; +SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth'; """ From 8a89cd31a1e7770479af6eaf1b4211ef4ece1795 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Aug 2024 00:29:32 +0200 Subject: [PATCH 1272/1722] Fix Upgrade Check: move some settings to 24.9 section --- src/Core/SettingsChangesHistory.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index fb59577b0f0..5e831c6301c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -72,11 +72,13 @@ static std::initializer_list Date: Thu, 22 Aug 2024 00:48:29 +0200 Subject: [PATCH 1273/1722] fix black --- tests/integration/test_storage_hdfs/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ea8c4efa745..a75c13b9ea6 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1269,7 +1269,6 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert r == f"Elizabeth\n" - def test_hive_partitioning_with_all_parameters(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( From 0f3c7ae8c202f475fe55f33f45e9bca92155d52c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:15:16 +0200 Subject: [PATCH 1274/1722] feat: add docs --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ffd9fae7f45..308e285c4bd 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2942 +personal_ws-1.1 en 2983 AArch ACLs ALTERs @@ -957,6 +957,7 @@ ThreadPoolRemoteFSReaderThreads ThreadPoolRemoteFSReaderThreadsActive ThreadsActive ThreadsInOvercommitTracker +TimeSeries Timeunit TinyLog Tkachenko @@ -1098,12 +1099,12 @@ addressToLineWithInlines addressToSymbol adviced agg +aggThrow aggregatefunction aggregatingmergetree aggregatio aggretate aggthrow -aggThrow aiochclient allocator alphaTokens @@ -1875,8 +1876,8 @@ joinGet joinGetOrNull json jsonMergePatch -jsonasstring jsonasobject +jsonasstring jsoncolumns jsoncolumnsmonoblock jsoncompact @@ -1917,8 +1918,8 @@ kurtSamp kurtosis kurtpop kurtsamp -laion lagInFrame +laion lang laravel largestTriangleThreeBuckets @@ -2020,7 +2021,6 @@ maxMap maxintersections maxintersectionsposition maxmap -minMappedArrays maxmind mdadm meanZTest @@ -2213,8 +2213,8 @@ parseReadableSizeOrZero parseTimeDelta parseable parsers -partitionId partitionID +partitionId pathFull pclmulqdq pcre @@ -2443,6 +2443,7 @@ rewritable rightPad rightPadUTF rightUTF +ripeMD risc riscv ro @@ -2694,7 +2695,6 @@ themself threadpool throwIf timeDiff -TimeSeries timeSeriesData timeSeriesMetrics timeSeriesTags From 54caf1f84e3c3b5076adf29b49f4ee548f243091 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 1275/1722] fix: wrap in conditional preprocessor directives --- src/Functions/FunctionsHashing.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 5111ee2bd90..ec39cf1e2cf 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -192,6 +192,7 @@ T combineHashesFunc(T t1, T t2) return HashFunction::apply(reinterpret_cast(hashes), sizeof(hashes)); } +#if USE_SSL struct RipeMD160Impl { static constexpr auto name = "ripeMD160"; @@ -218,7 +219,7 @@ struct RipeMD160Impl static constexpr bool use_int_hash_for_pods = false; }; - +#endif struct SipHash64Impl { @@ -1647,6 +1648,7 @@ using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL using FunctionHalfMD5 = FunctionAnyHash; +using FunctionRipeMD160Hash = FunctionAnyHash; #endif using FunctionSipHash128 = FunctionAnyHash; using FunctionSipHash128Keyed = FunctionAnyHash; @@ -1676,7 +1678,6 @@ using FunctionXXH3 = FunctionAnyHash; using FunctionWyHash64 = FunctionAnyHash; -using FunctionRipeMD160Hash = FunctionAnyHash; } #pragma clang diagnostic pop From 92e153585ded4f15e1292613584ff35a55c735f3 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Wed, 21 Aug 2024 19:19:07 -0700 Subject: [PATCH 1276/1722] Update README.md Add latest meetups from Alexey tour. --- README.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/README.md b/README.md index 17b6dcd2ac1..5e66b9da73e 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,13 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. +The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey Milovidov: + * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 +* [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 +* [New York Meetup (Ramp)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 +* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From 9d0b3e3937cca32bc8bc922876fb8e6ac53a3de9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 22 Aug 2024 11:32:59 +0800 Subject: [PATCH 1277/1722] change as request --- .../functions/string-replace-functions.md | 145 +++++++++--------- src/Functions/overlay.cpp | 8 +- ...new_functions_must_be_documented.reference | 2 - 3 files changed, 76 insertions(+), 79 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 55e97688b18..0cc6b0b27d5 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -8,6 +8,78 @@ sidebar_label: Replacing in Strings [General strings functions](string-functions.md) and [functions for searching in strings](string-search-functions.md) are described separately. +## overlay + +Replace part of the string `input` with another string `replace`, starting at the 1-based index `offset`. + +**Syntax** + +```sql +overlay(s, replace, offset[, length]) +``` + +**Parameters** + +- `input`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). If `offset` is negative, it is counted from the end of the `input` string. +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). `length` specifies the length of the snippet within input to be replaced. If `length` is not specified, the number of bytes removed from `input` equals the length of `replace`; otherwise `length` bytes are removed. + +**Returned value** + +- A [String](../data-types/string.md) data type value. + +**Example** + +```sql +SELECT overlay('ClickHouse SQL', 'CORE', 12) AS res; +``` + +Result: + +```text +┌─res─────────────┠+│ ClickHouse CORE │ +└─────────────────┘ +``` + +## overlayUTF8 + +Replace part of the string `input` with another string `replace`, starting at the 1-based index `offset`. + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +overlayUTF8(s, replace, offset[, length]) +``` + +**Parameters** + +- `s`: A string type [String](../data-types/string.md). +- `replace`: A string type [String](../data-types/string.md). +- `offset`: An integer type [Int](../data-types/int-uint.md). If `offset` is negative, it is counted from the end of the `input` string. +- `length`: Optional. An integer type [Int](../data-types/int-uint.md). `length` specifies the length of the snippet within input to be replaced. If `length` is not specified, the number of characters removed from `input` equals the length of `replace`; otherwise `length` characters are removed. + +**Returned value** + +- A [String](../data-types/string.md) data type value. + +**Example** + +```sql +SELECT overlayUTF8('ClickHouse是一款OLAPæ•°æ®åº“', 'å¼€æº', 12, 2) AS res; +``` + +Result: + +```text +┌─res────────────────────────┠+│ ClickHouse是开æºOLAPæ•°æ®åº“ │ +└────────────────────────────┘ +``` + ## replaceOne Replaces the first occurrence of the substring `pattern` in `haystack` by the `replacement` string. @@ -248,76 +320,3 @@ select printf('%%%s %s %d', 'Hello', 'World', 2024); │ %Hello World 2024 │ └──────────────────────────────────────────────┘ ``` - -## overlay - -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. - - -**Syntax** - -```sql -overlay(s, replace, offset[, length]) -``` - -**Parameters** - -- `s`: A string type [String](../data-types/string.md). -- `replace`: A string type [String](../data-types/string.md). -- `offset`: An integer type [Int](../data-types/int-uint.md). -- `length`: Optional. An integer type [Int](../data-types/int-uint.md). - -**Returned value** - -- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. - -**Example** - -```sql -SELECT overlay('Spark SQL', 'CORE', 7) AS res; -``` - -Result: - -```text - ┌─res────────┠- │ Spark CORE │ - └────────────┘ -``` - -## overlayUTF8 - -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of characters removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of characters is removed. - -Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. - -**Syntax** - -```sql -overlayUTF8(s, replace, offset[, length]) -``` - -**Parameters** - -- `s`: A string type [String](../data-types/string.md). -- `replace`: A string type [String](../data-types/string.md). -- `offset`: An integer type [Int](../data-types/int-uint.md). -- `length`: Optional. An integer type [Int](../data-types/int-uint.md). - -**Returned value** - -- A [String](../data-types/string.md) data type value. If `offset` is negative the offset is counted starting from the back. `length` specifies the length of the snippet within input to be replaced. - -**Example** - -```sql -SELECT overlayUTF8('ClickHouse是一款OLAPæ•°æ®åº“', 'å¼€æº', 12, 2) AS res; -``` - -Result: - -```text -┌─res────────────────────────┠-│ ClickHouse是开æºOLAPæ•°æ®åº“ │ -└────────────────────────────┘ -``` diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 20988c775a5..497ebb9c9cd 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -201,14 +201,14 @@ private: { if (offset > 0) { - if (static_cast(offset) > input_size + 1) [[unlikely]] + if (static_cast(offset) > input_size + 1) return input_size; else return offset - 1; } else { - if (input_size < -static_cast(offset)) [[unlikely]] + if (input_size < -static_cast(offset)) return 0; else return input_size + offset; @@ -704,14 +704,14 @@ REGISTER_FUNCTION(Overlay) { factory.registerFunction>( {.description = R"( -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. +Replace a part of a string `input` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `input` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. )", .categories{"String"}}, FunctionFactory::Case::Insensitive); factory.registerFunction>( {.description = R"( -Replace a part of a string `s` with another string `replace`, starting at 1-based index `offset`. By default, the number of bytes removed from `s` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of bytes is removed. +Replace a part of a string `input` with another string `replace`, starting at 1-based index `offset`. By default, the number of characters removed from `input` equals the length of `replace`. If `length` (the optional fourth argument) is specified, a different number of characters is removed. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. )", diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 6495b6619f9..c39f1fb1ce9 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -512,8 +512,6 @@ nullIf nullIn nullInIgnoreSet or -overlay -overlayUTF8 parseDateTime parseDateTime32BestEffort parseDateTime32BestEffortOrNull From 3ff9522b69ec7e51119f445152ffb9678a0f124f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 22 Aug 2024 12:49:10 +0800 Subject: [PATCH 1278/1722] change as request --- src/Functions/overlay.cpp | 165 +++++++++--------- .../0_stateless/03205_overlay.reference | 58 +++--- tests/queries/0_stateless/03205_overlay.sql | 11 +- 3 files changed, 115 insertions(+), 119 deletions(-) diff --git a/src/Functions/overlay.cpp b/src/Functions/overlay.cpp index 497ebb9c9cd..df8b825eabe 100644 --- a/src/Functions/overlay.cpp +++ b/src/Functions/overlay.cpp @@ -53,139 +53,132 @@ public: if (input_rows_count == 0) return ColumnString::create(); - const size_t number_of_arguments = arguments.size(); - bool has_three_args = number_of_arguments == 3; + bool has_four_args = (arguments.size() == 4); - ColumnPtr column_offset = arguments[2].column; - ColumnPtr column_length; - if (!has_three_args) - column_length = arguments[3].column; + ColumnPtr col_input = arguments[0].column; + const auto * col_input_const = checkAndGetColumn(col_input.get()); + const auto * col_input_string = checkAndGetColumn(col_input.get()); + bool input_is_const = (col_input_const != nullptr); - const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); - const ColumnConst * column_length_const = nullptr; - if (!has_three_args) - column_length_const = checkAndGetColumn(column_length.get()); + ColumnPtr col_replace = arguments[1].column; + const auto * col_replace_const = checkAndGetColumn(col_replace.get()); + const auto * col_replace_string = checkAndGetColumn(col_replace.get()); + bool replace_is_const = (col_replace_const != nullptr); + ColumnPtr col_offset = arguments[2].column; + const ColumnConst * col_offset_const = checkAndGetColumn(col_offset.get()); bool offset_is_const = false; - bool length_is_const = false; Int64 offset = -1; - Int64 length = -1; - if (column_offset_const) + if (col_offset_const) { - offset = column_offset_const->getInt(0); + offset = col_offset_const->getInt(0); offset_is_const = true; } - if (column_length_const) + ColumnPtr col_length = has_four_args ? arguments[3].column : nullptr; + const ColumnConst * col_length_const = has_four_args ? checkAndGetColumn(col_length.get()) : nullptr; + bool length_is_const = false; + Int64 length = -1; + if (col_length_const) { - length = column_length_const->getInt(0); + length = col_length_const->getInt(0); length_is_const = true; } - auto res_col = ColumnString::create(); auto & res_data = res_col->getChars(); auto & res_offsets = res_col->getOffsets(); + res_offsets.resize_exact(input_rows_count); - - ColumnPtr column_input = arguments[0].column; - ColumnPtr column_replace = arguments[1].column; - - const auto * column_input_const = checkAndGetColumn(column_input.get()); - const auto * column_input_string = checkAndGetColumn(column_input.get()); - if (column_input_const) + if (col_input_const) { - StringRef input = column_input_const->getDataAt(0); + StringRef input = col_input_const->getDataAt(0); res_data.reserve((input.size + 1) * input_rows_count); } else { - res_data.reserve(column_input_string->getChars().size()); + res_data.reserve(col_input_string->getChars().size()); } - const auto * column_replace_const = checkAndGetColumn(column_replace.get()); - const auto * column_replace_string = checkAndGetColumn(column_replace.get()); - bool input_is_const = (column_input_const != nullptr); - bool replace_is_const = (column_replace_const != nullptr); -#define OVERLAY_EXECUTE_CASE(THREE_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ +#define OVERLAY_EXECUTE_CASE(HAS_FOUR_ARGS, OFFSET_IS_CONST, LENGTH_IS_CONST) \ if (input_is_const && replace_is_const) \ - constantConstant( \ + constantConstant( \ input_rows_count, \ - column_input_const->getDataAt(0), \ - column_replace_const->getDataAt(0), \ - column_offset, \ - column_length, \ + col_input_const->getDataAt(0), \ + col_replace_const->getDataAt(0), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); \ else if (input_is_const && !replace_is_const) \ - constantVector( \ + constantVector( \ input_rows_count, \ - column_input_const->getDataAt(0), \ - column_replace_string->getChars(), \ - column_replace_string->getOffsets(), \ - column_offset, \ - column_length, \ + col_input_const->getDataAt(0), \ + col_replace_string->getChars(), \ + col_replace_string->getOffsets(), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); \ else if (!input_is_const && replace_is_const) \ - vectorConstant( \ + vectorConstant( \ input_rows_count, \ - column_input_string->getChars(), \ - column_input_string->getOffsets(), \ - column_replace_const->getDataAt(0), \ - column_offset, \ - column_length, \ + col_input_string->getChars(), \ + col_input_string->getOffsets(), \ + col_replace_const->getDataAt(0), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); \ else \ - vectorVector( \ + vectorVector( \ input_rows_count, \ - column_input_string->getChars(), \ - column_input_string->getOffsets(), \ - column_replace_string->getChars(), \ - column_replace_string->getOffsets(), \ - column_offset, \ - column_length, \ + col_input_string->getChars(), \ + col_input_string->getOffsets(), \ + col_replace_string->getChars(), \ + col_replace_string->getOffsets(), \ + col_offset, \ + col_length, \ offset, \ length, \ res_data, \ res_offsets); - if (has_three_args) + if (!has_four_args) { if (offset_is_const) { - OVERLAY_EXECUTE_CASE(true, true, false) + OVERLAY_EXECUTE_CASE(false, true, false) } else { - OVERLAY_EXECUTE_CASE(true, false, false) + OVERLAY_EXECUTE_CASE(false, false, false) } } else { if (offset_is_const && length_is_const) { - OVERLAY_EXECUTE_CASE(false, true, true) + OVERLAY_EXECUTE_CASE(true, true, true) } else if (offset_is_const && !length_is_const) { - OVERLAY_EXECUTE_CASE(false, true, false) + OVERLAY_EXECUTE_CASE(true, true, false) } else if (!offset_is_const && length_is_const) { - OVERLAY_EXECUTE_CASE(false, false, true) + OVERLAY_EXECUTE_CASE(true, false, true) } else { - OVERLAY_EXECUTE_CASE(false, false, false) + OVERLAY_EXECUTE_CASE(true, false, false) } } #undef OVERLAY_EXECUTE_CASE @@ -224,7 +217,7 @@ private: return bytes; } - template + template void constantConstant( size_t rows, const StringRef & input, @@ -236,7 +229,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { constantConstant( rows, input, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -250,12 +243,12 @@ private: size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (has_three_args) + else if constexpr (!has_four_args) { valid_length = replace_size; } @@ -273,14 +266,14 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!has_three_args && !length_is_const) + if constexpr (has_four_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -332,11 +325,12 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } - template + template void vectorConstant( size_t rows, const ColumnString::Chars & input_data, @@ -349,7 +343,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { vectorConstant( rows, input_data, input_offsets, replace, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -359,12 +353,12 @@ private: size_t replace_size = getSliceSize(reinterpret_cast(replace.data), replace.size); Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; } - else if constexpr (has_three_args) + else if constexpr (!has_four_args) { valid_length = replace_size; } @@ -388,14 +382,14 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (!has_three_args && !length_is_const) + if constexpr (has_four_args && !length_is_const) { length = column_length->getInt(i); valid_length = length >= 0 ? length : replace_size; } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -449,11 +443,12 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } - template + template void constantVector( size_t rows, const StringRef & input, @@ -466,7 +461,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { constantVector( rows, input, replace_data, replace_offsets, column_offset, column_length, const_offset, -1, res_data, res_offsets); @@ -480,7 +475,7 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; @@ -502,7 +497,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (has_three_args) + if constexpr (!has_four_args) { valid_length = replace_size; } @@ -513,7 +508,7 @@ private: } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -563,11 +558,12 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } - template + template void vectorVector( size_t rows, const ColumnString::Chars & input_data, @@ -581,7 +577,7 @@ private: ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) const { - if (!has_three_args && length_is_const && const_length < 0) + if (has_four_args && length_is_const && const_length < 0) { vectorVector( rows, @@ -600,7 +596,7 @@ private: Int64 length = 0; // maybe negative size_t valid_length = 0; // not negative - if constexpr (!has_three_args && length_is_const) + if constexpr (has_four_args && length_is_const) { assert(const_length >= 0); valid_length = const_length; @@ -629,7 +625,7 @@ private: valid_offset = getValidOffset(offset, input_size); } - if constexpr (has_three_args) + if constexpr (!has_four_args) { valid_length = replace_size; } @@ -640,7 +636,7 @@ private: } size_t prefix_size = valid_offset; - size_t suffix_size = prefix_size + valid_length > input_size ? 0 : input_size - prefix_size - valid_length; + size_t suffix_size = (prefix_size + valid_length > input_size) ? 0 : (input_size - prefix_size - valid_length); if constexpr (!is_utf8) { @@ -693,6 +689,7 @@ private: /// add zero terminator res_data[res_offset] = 0; ++res_offset; + res_offsets[i] = res_offset; } } diff --git a/tests/queries/0_stateless/03205_overlay.reference b/tests/queries/0_stateless/03205_overlay.reference index 67a699944e0..4be3baadaea 100644 --- a/tests/queries/0_stateless/03205_overlay.reference +++ b/tests/queries/0_stateless/03205_overlay.reference @@ -25,34 +25,34 @@ Spark ANSI SQL Spark ANSI SQLå’ŒCH Spark ANSI SQL Spark ANSI SQLå’ŒCH Spark ANSI SQL Spark ANSI SQLå’ŒCH Spark ANSI SQL Spark ANSI SQLå’ŒCH -Test with different offset values --12 _park SQL _park SQLå’ŒCH --11 _park SQL S_ark SQLå’ŒCH --10 _park SQL Sp_rk SQLå’ŒCH --9 _park SQL Spa_k SQLå’ŒCH --8 S_ark SQL Spar_ SQLå’ŒCH --7 Sp_rk SQL Spark_SQLå’ŒCH --6 Spa_k SQL Spark _QLå’ŒCH --5 Spar_ SQL Spark S_Lå’ŒCH --4 Spark_SQL Spark SQ_å’ŒCH --3 Spark _QL Spark SQL_CH --2 Spark S_L Spark SQLå’Œ_H --1 Spark SQ_ Spark SQLå’ŒC_ -0 Spark SQL_ Spark SQLå’ŒCH_ -1 _park SQL _park SQLå’ŒCH -2 S_ark SQL S_ark SQLå’ŒCH -3 Sp_rk SQL Sp_rk SQLå’ŒCH -4 Spa_k SQL Spa_k SQLå’ŒCH -5 Spar_ SQL Spar_ SQLå’ŒCH -6 Spark_SQL Spark_SQLå’ŒCH -7 Spark _QL Spark _QLå’ŒCH -8 Spark S_L Spark S_Lå’ŒCH -9 Spark SQ_ Spark SQ_å’ŒCH -10 Spark SQL_ Spark SQL_CH -11 Spark SQL_ Spark SQLå’Œ_H -12 Spark SQL_ Spark SQLå’ŒC_ -13 Spark SQL_ Spark SQLå’ŒCH_ -Test with different length values +Test with special offset values +-12 __ark SQL 之park SQLå’ŒCH +-11 __ark SQL S之ark SQLå’ŒCH +-10 __ark SQL Sp之rk SQLå’ŒCH +-9 __ark SQL Spa之k SQLå’ŒCH +-8 S__rk SQL Spar之 SQLå’ŒCH +-7 Sp__k SQL Spark之SQLå’ŒCH +-6 Spa__ SQL Spark 之QLå’ŒCH +-5 Spar__SQL Spark S之Lå’ŒCH +-4 Spark__QL Spark SQ之和CH +-3 Spark __L Spark SQL之CH +-2 Spark S__ Spark SQL和之H +-1 Spark SQ__ Spark SQLå’ŒC之 +0 Spark SQL__ Spark SQLå’ŒCH之 +1 __ark SQL 之park SQLå’ŒCH +2 S__rk SQL S之ark SQLå’ŒCH +3 Sp__k SQL Sp之rk SQLå’ŒCH +4 Spa__ SQL Spa之k SQLå’ŒCH +5 Spar__SQL Spar之 SQLå’ŒCH +6 Spark__QL Spark之SQLå’ŒCH +7 Spark __L Spark 之QLå’ŒCH +8 Spark S__ Spark S之Lå’ŒCH +9 Spark SQ__ Spark SQ之和CH +10 Spark SQL__ Spark SQL之CH +11 Spark SQL__ Spark SQL和之H +12 Spark SQL__ Spark SQLå’ŒC之 +13 Spark SQL__ Spark SQLå’ŒCH之 +Test with special length values -1 Spark ANSI Spark ANSI H 0 Spark ANSI SQL Spark ANSI SQLå’ŒCH 1 Spark ANSI QL Spark ANSI QLå’ŒCH @@ -61,7 +61,7 @@ Test with different length values 4 Spark ANSI Spark ANSI CH 5 Spark ANSI Spark ANSI H 6 Spark ANSI Spark ANSI -Test with different input and replace values +Test with special input and replace values _ _ Spark SQL Spark SQLå’ŒCH ANSI ANSI diff --git a/tests/queries/0_stateless/03205_overlay.sql b/tests/queries/0_stateless/03205_overlay.sql index 4d0b5ecbe03..765b29f93ec 100644 --- a/tests/queries/0_stateless/03205_overlay.sql +++ b/tests/queries/0_stateless/03205_overlay.sql @@ -1,5 +1,4 @@ SELECT 'Negative test of overlay'; -SELECT overlay('hello', 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT overlay('hello', 'world'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT overlay('hello', 'world', 2, 3, 'extra'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT overlay(123, 'world', 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } @@ -35,13 +34,13 @@ SELECT overlay(materialize('Spark SQL'), 'ANSI ', materialize(7), materialize(0) SELECT overlay('Spark SQL', materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8('Spark SQLå’ŒCH', materialize('ANSI '), materialize(7), materialize(0)); SELECT overlay(materialize('Spark SQL'), materialize('ANSI '), materialize(7), materialize(0)), overlayUTF8(materialize('Spark SQLå’ŒCH'), materialize('ANSI '), materialize(7), materialize(0)); -SELECT 'Test with different offset values'; -WITH number - 12 as offset SELECT offset, overlay('Spark SQL', '_', offset), overlayUTF8('Spark SQLå’ŒCH', '_', offset) from numbers(26); +SELECT 'Test with special offset values'; +WITH number - 12 AS offset SELECT offset, overlay('Spark SQL', '__', offset), overlayUTF8('Spark SQLå’ŒCH', '之', offset) FROM numbers(26); -SELECT 'Test with different length values'; -WITH number - 1 as length SELECT length, overlay('Spark SQL', 'ANSI ', 7, length), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, length) from numbers(8); +SELECT 'Test with special length values'; +WITH number - 1 AS length SELECT length, overlay('Spark SQL', 'ANSI ', 7, length), overlayUTF8('Spark SQLå’ŒCH', 'ANSI ', 7, length) FROM numbers(8); -SELECT 'Test with different input and replace values'; +SELECT 'Test with special input and replace values'; SELECT overlay('', '_', 6), overlayUTF8('', '_', 6); SELECT overlay('Spark SQL', '', 6), overlayUTF8('Spark SQLå’ŒCH', '', 6); SELECT overlay('', 'ANSI ', 7, 0), overlayUTF8('', 'ANSI ', 7, 0); From be4439e3ec0a1491f4e333ac848844fd930a6e5b Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 22 Aug 2024 10:30:48 +0300 Subject: [PATCH 1279/1722] Update install.md Added correct commands for russian vwersion of the installation from deb packets --- docs/ru/getting-started/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index aee445da843..4a0ec258c64 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -25,10 +25,10 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su Ð¯Ð½Ð´ÐµÐºÑ Ñ€ÐµÐºÐ¾Ð¼ÐµÐ½Ð´ÑƒÐµÑ‚ иÑпользовать официальные Ñкомпилированные `deb`-пакеты Ð´Ð»Ñ Debian или Ubuntu. Ð”Ð»Ñ ÑƒÑтановки пакетов выполните: ``` bash -sudo apt-get install -y apt-transport-https ca-certificates dirmngr -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 +sudo apt-get install -y apt-transport-https ca-certificates curl gnupg +curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | sudo gpg --dearmor -o /usr/share/keyrings/clickhouse-keyring.gpg -echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ +echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update From 6466f374e0372b22a23d1193e534bd6c94f87b94 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 22 Aug 2024 11:29:33 +0200 Subject: [PATCH 1280/1722] Update geohash.md --- .../en/sql-reference/functions/geo/geohash.md | 31 ++++++++++++++----- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index b6ac7a74092..c4f41fc53da 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -6,7 +6,7 @@ title: "Functions for Working with Geohash" ## Geohash -[Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer is the geohash string, the more precise is the geographic location. +[Geohash](https://en.wikipedia.org/wiki/Geohash) is the geocode system, which subdivides Earth’s surface into buckets of grid shape and encodes each cell into a short string of letters and digits. It is a hierarchical data structure, so the longer the geohash string is, the more precise the geographic location will be. If you need to manually convert geographic coordinates to geohash strings, you can use [geohash.org](http://geohash.org/). @@ -14,26 +14,37 @@ If you need to manually convert geographic coordinates to geohash strings, you c Encodes latitude and longitude as a [geohash](#geohash)-string. +**Syntax** + ``` sql geohashEncode(longitude, latitude, [precision]) ``` **Input values** -- longitude - longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]` -- latitude - latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]` -- precision - Optional, length of the resulting encoded string, defaults to `12`. Integer in range `[1, 12]`. Any value less than `1` or greater than `12` is silently converted to `12`. +- `longitude` — Longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`. [Float](../../data_types/float.md). +- `latitude` — Latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`. [Float](../../data_types/float.md). +- `precision` (optional) — Length of the resulting encoded string. Defaults to `12`. Integer in the range `[1, 12]`. [Int8](../../data-types/int-uint.md). + +:::note +- All coordinate parameters must be of the same type: either `Float32` or `Float64`. +- For the `precision` parameter, any value less than `1` or greater than `12` is silently converted to `12`. +::: **Returned values** -- alphanumeric `String` of encoded coordinate (modified version of the base32-encoding alphabet is used). +- Alphanumeric string of the encoded coordinate (modified version of the base32-encoding alphabet is used). [String](../../data-types/string.md). **Example** +Query: + ``` sql SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res; ``` +Result: + ``` text ┌─res──────────┠│ ezs42d000000 │ @@ -44,13 +55,19 @@ SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res; Decodes any [geohash](#geohash)-encoded string into longitude and latitude. +**Syntax** + +```sql +geohashDecode(hash_str) +``` + **Input values** -- encoded string - geohash-encoded string. +- `hash_str` — Geohash-encoded string. **Returned values** -- (longitude, latitude) - 2-tuple of `Float64` values of longitude and latitude. +- Tuple `(longitude, latitude)` of `Float64` values of longitude and latitude. [Tuple](../../data-types/tuple.md)([Float64](../../data-types/float.md)) **Example** From 526be4759d40efad45a871c3ff150929fcca76be Mon Sep 17 00:00:00 2001 From: Pedro Ferreira Date: Thu, 22 Aug 2024 10:34:05 +0000 Subject: [PATCH 1281/1722] SQL as a set language, doesn't have order in the output, so add ORDER BY or min call for it --- .../integration/test_truncate_database/test_replicated.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_truncate_database/test_replicated.py b/tests/integration/test_truncate_database/test_replicated.py index 59830d44378..d55a0dc7557 100644 --- a/tests/integration/test_truncate_database/test_replicated.py +++ b/tests/integration/test_truncate_database/test_replicated.py @@ -1,5 +1,3 @@ -import time - import pytest from helpers.cluster import ClickHouseCluster @@ -9,7 +7,7 @@ def fill_nodes(nodes, shard): node.query( """ CREATE DATABASE test; - + CREATE TABLE test.test_table(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date); """.format( @@ -50,7 +48,8 @@ def test_truncate_database_replicated(start_cluster): node1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" ) - assert node2.query("SELECT id FROM test.test_table LIMIT 1") == "0\n" + assert node2.query("SELECT min(id) FROM test.test_table") == "0\n" + assert node2.query("SELECT id FROM test.test_table ORDER BY id LIMIT 1") == "0\n" assert node3.query("SHOW DATABASES LIKE 'test'") == "test\n" node3.query("TRUNCATE DATABASE test ON CLUSTER test_cluster SYNC") assert node2.query("SHOW TABLES FROM test") == "" From 95f45d2eaf39a9e8a6373c75749ec57f727be700 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:20:04 +0200 Subject: [PATCH 1282/1722] try to fix tests --- .../test_storage_azure_blob_storage/test.py | 14 +++++------ tests/integration/test_storage_hdfs/test.py | 25 +++---------------- .../03203_hive_style_partitioning.reference | 20 +++++++-------- .../03203_hive_style_partitioning.sh | 14 +++-------- 4 files changed, 23 insertions(+), 50 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index a3172329a99..c1f518e45ce 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1513,14 +1513,14 @@ def test_hive_partitioning_with_one_parameter(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values}", settings={"azure_truncate_on_insert": 1}, ) query = ( f"SELECT column2, _file, _path, column1 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}')" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}')" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} @@ -1533,7 +1533,7 @@ def test_hive_partitioning_with_one_parameter(cluster): query = ( f"SELECT column2 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}');" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');" ) assert azure_query( node, query, settings={"use_hive_partitioning": 1} @@ -1551,14 +1551,14 @@ def test_hive_partitioning_with_all_parameters(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", settings={"azure_truncate_on_insert": 1}, ) query = ( f"SELECT column1, column2, _file, _path FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}');" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');" ) pattern = r"DB::Exception: Cannot use hive partitioning for file" @@ -1577,14 +1577,14 @@ def test_hive_partitioning_without_setting(cluster): azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," - f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", + f" container='cont', blob_path='{path}', format='CSVWithNames', compression='auto', structure='{table_format}') VALUES {values_1}, {values_2}", settings={"azure_truncate_on_insert": 1}, ) query = ( f"SELECT column1, column2, _file, _path, column3 FROM azureBlobStorage(azure_conf2, " f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " - f"blob_path='{path}', format='CSV', structure='{table_format}');" + f"blob_path='{path}', format='CSVWithNames', structure='{table_format}');" ) pattern = re.compile( r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index a75c13b9ea6..31cc8609eb4 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1259,35 +1259,16 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): def test_hive_partitioning_with_one_parameter(started_cluster): hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data(f"/column0=Elizabeth/parquet_1", f"Elizabeth\tGordon\n") - assert hdfs_api.read_data(f"/column0=Elizabeth/parquet_1") == f"Elizabeth\tGordon\n" + hdfs_api.write_data(f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n") + assert hdfs_api.read_data(f"/column0=Elizabeth/file_1") == f"column0,column1\nElizabeth,Gordon\n" r = node1.query( - "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/parquet_1', 'TSV')", + "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/file_1', 'CSVWithNames')", settings={"use_hive_partitioning": 1}, ) assert r == f"Elizabeth\n" -def test_hive_partitioning_with_all_parameters(started_cluster): - hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data( - f"/column0=Elizabeth/column1=Gordon/parquet_2", f"Elizabeth\tGordon\n" - ) - assert ( - hdfs_api.read_data(f"/column0=Elizabeth/column1=Gordon/parquet_2") - == f"Elizabeth\tGordon\n" - ) - - pattern = r"DB::Exception: Cannot use hive partitioning for file" - - with pytest.raises(QueryRuntimeException, match=pattern): - node1.query( - f"SELECT column1 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/column1=Gordon/parquet_2', 'TSV');", - settings={"use_hive_partitioning": 1}, - ) - - def test_hive_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index af52dcd9b88..acdadc2510b 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -1,5 +1,5 @@ TESTING THE FILE HIVE PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -19,8 +19,7 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -35,12 +34,13 @@ Cross Elizabeth Array(Int64) LowCardinality(Float64) 101 2070 +4081 +2070 2070 b 1 -1 TESTING THE URL PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -60,10 +60,9 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth 1 TESTING THE S3 PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -83,8 +82,7 @@ Stanley Gibson Elizabeth Eugenia Greer Elizabeth Jeffery Delgado Elizabeth Clara Cross Elizabeth -Elizabeth Gordon Elizabeth -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -96,7 +94,7 @@ Delgado Elizabeth Cross Elizabeth OK TESTING THE S3CLUSTER PARTITIONING -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth @@ -106,7 +104,7 @@ Gibson Elizabeth Greer Elizabeth Delgado Elizabeth Cross Elizabeth -last Elizabeth + last Elizabeth Frank Elizabeth Moreno Elizabeth Guzman Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 4e165446c34..b3d196924af 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -14,7 +14,7 @@ set use_hive_partitioning = 1; SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10; SELECT number, date FROM file('$CURDIR/data_hive/partitioning/number=42/date=2020-01-01/sample.parquet') LIMIT 1; SELECT array, float FROM file('$CURDIR/data_hive/partitioning/array=[1,2,3]/float=42.42/sample.parquet') LIMIT 1; @@ -29,16 +29,10 @@ SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.c SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ -$CLICKHOUSE_LOCAL -n -q """ -set use_hive_partitioning = 1; - -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; -""" 2>&1 | grep -c "INCORRECT_DATA" - $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" @@ -68,7 +62,7 @@ set use_hive_partitioning = 1; SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; SELECT *, non_existing_column FROM s3('http://localhost:11111/test/hive_partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; +SELECT *, column0 FROM s3('http://localhost:11111/test/hive_partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10; """ $CLICKHOUSE_CLIENT -n -q """ @@ -84,5 +78,5 @@ set use_hive_partitioning = 1; SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; -SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth'; +SELECT *, column0 FROM s3Cluster(test_cluster_one_shard_three_replicas_localhost, 'http://localhost:11111/test/hive_partitioning/column0=Elizabeth/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 10; """ From 62054cae666244fd072a56f70a6df73e68249cb0 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 22 Aug 2024 13:49:16 +0200 Subject: [PATCH 1283/1722] Update geohash.md --- docs/en/sql-reference/functions/geo/geohash.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/geohash.md b/docs/en/sql-reference/functions/geo/geohash.md index c4f41fc53da..ce2e3c43b3e 100644 --- a/docs/en/sql-reference/functions/geo/geohash.md +++ b/docs/en/sql-reference/functions/geo/geohash.md @@ -22,8 +22,8 @@ geohashEncode(longitude, latitude, [precision]) **Input values** -- `longitude` — Longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`. [Float](../../data_types/float.md). -- `latitude` — Latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`. [Float](../../data_types/float.md). +- `longitude` — Longitude part of the coordinate you want to encode. Floating in range`[-180°, 180°]`. [Float](../../data-types/float.md). +- `latitude` — Latitude part of the coordinate you want to encode. Floating in range `[-90°, 90°]`. [Float](../../data-types/float.md). - `precision` (optional) — Length of the resulting encoded string. Defaults to `12`. Integer in the range `[1, 12]`. [Int8](../../data-types/int-uint.md). :::note From 84467077b886cd48c9cd33c69c1935b3f7863dd7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 22 Aug 2024 13:45:13 +0200 Subject: [PATCH 1284/1722] Fix test for role expiration in RoleCache. --- tests/integration/test_role/test.py | 81 +++++++++-------------------- 1 file changed, 26 insertions(+), 55 deletions(-) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index b3b18dc8271..9d15f0f81db 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -418,72 +418,43 @@ def test_function_current_roles(): ) -def test_role_expiration(): - instance.query("CREATE USER ure") +@pytest.mark.parametrize("with_extra_role", [False, True]) +def test_role_expiration(with_extra_role): instance.query("CREATE ROLE rre") - instance.query("GRANT rre TO ure") + instance.query("CREATE USER ure DEFAULT ROLE rre") - instance.query("CREATE TABLE IF NOT EXISTS tre (id Int) Engine=Log") - instance.query("INSERT INTO tre VALUES (0)") + instance.query("CREATE TABLE table1 (id Int) Engine=Log") + instance.query("CREATE TABLE table2 (id Int) Engine=Log") + instance.query("INSERT INTO table1 VALUES (1)") + instance.query("INSERT INTO table2 VALUES (2)") + instance.query("GRANT SELECT ON table1 TO rre") + + assert instance.query("SELECT * FROM table1", user="ure") == "1\n" assert "Not enough privileges" in instance.query_and_get_error( - "SELECT * FROM tre", user="ure" + "SELECT * FROM table2", user="ure" ) - instance.query("GRANT SELECT ON tre TO rre") - - assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - # access_control_improvements/role_cache_expiration_time_seconds value is 2 for the test # so we wait >2 seconds until the role is expired time.sleep(5) - instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") - instance.query("INSERT INTO tre1 VALUES (0)") - instance.query("GRANT SELECT ON tre1 TO rre") + if with_extra_role: + # Expiration of role "rre" from the role cache can be caused by another role being used. + instance.query("CREATE ROLE extra_role") + instance.query("CREATE USER extra_user DEFAULT ROLE extra_role") + instance.query("GRANT SELECT ON table1 TO extra_role") + assert instance.query("SELECT * FROM table1", user="extra_user") == "1\n" - assert instance.query("SELECT * from tre1", user="ure") == "0\n" + instance.query("GRANT SELECT ON table2 TO rre") + assert instance.query("SELECT * FROM table1", user="ure") == "1\n" + assert instance.query("SELECT * FROM table2", user="ure") == "2\n" - instance.query("DROP USER ure") instance.query("DROP ROLE rre") - instance.query("DROP TABLE tre") - instance.query("DROP TABLE tre1") - - -def test_two_roles_expiration(): - instance.query("CREATE USER ure") - instance.query("CREATE ROLE rre") - instance.query("GRANT rre TO ure") - - instance.query("CREATE ROLE rre_second") - - instance.query("CREATE TABLE IF NOT EXISTS tre (id Int) Engine=Log") - instance.query("INSERT INTO tre VALUES (0)") - - assert "Not enough privileges" in instance.query_and_get_error( - "SELECT * FROM tre", user="ure" - ) - - instance.query("GRANT SELECT ON tre TO rre") - - assert instance.query("SELECT * FROM tre", user="ure") == "0\n" - - # access_control_improvements/role_cache_expiration_time_seconds value is 2 for the test - # so we wait >2 seconds until the roles are expired - time.sleep(5) - - instance.query( - "GRANT SELECT ON tre1 TO rre_second" - ) # we expect that both rre and rre_second are gone from cache upon this operation - - instance.query("CREATE TABLE IF NOT EXISTS tre1 (id Int) Engine=Log") - instance.query("INSERT INTO tre1 VALUES (0)") - instance.query("GRANT SELECT ON tre1 TO rre") - - assert instance.query("SELECT * from tre1", user="ure") == "0\n" - instance.query("DROP USER ure") - instance.query("DROP ROLE rre") - instance.query("DROP ROLE rre_second") - instance.query("DROP TABLE tre") - instance.query("DROP TABLE tre1") + instance.query("DROP TABLE table1") + instance.query("DROP TABLE table2") + + if with_extra_role: + instance.query("DROP ROLE extra_role") + instance.query("DROP USER extra_user") From 664e9b3db9d47e45c642ad21e3a5273ab423199a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Aug 2024 13:30:41 +0200 Subject: [PATCH 1285/1722] Add one more test. --- tests/integration/test_role/test.py | 173 ++++++++++++++++++++++++++++ 1 file changed, 173 insertions(+) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 9d15f0f81db..225cab975ff 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -1,5 +1,6 @@ import time import pytest +import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -458,3 +459,175 @@ def test_role_expiration(with_extra_role): if with_extra_role: instance.query("DROP ROLE extra_role") instance.query("DROP USER extra_user") + + +def test_roles_cache(): + # This test takes 20 seconds. + test_time = 20 + + # Three users A, B, C. + users = ["A", "B", "C"] + instance.query("CREATE USER " + ", ".join(users)) + + # Table "tbl" has 10 columns. Each of the users has access to a different set of columns. + num_columns = 10 + columns = [f"x{i}" for i in range(1, num_columns + 1)] + columns_with_types = [column + " Int64" for column in columns] + columns_with_types_comma_separated = ", ".join(columns_with_types) + values = list(range(1, num_columns + 1)) + values_comma_separated = ", ".join([str(value) for value in values]) + instance.query( + f"CREATE TABLE tbl ({columns_with_types_comma_separated}) ENGINE=MergeTree ORDER BY tuple()" + ) + instance.query(f"INSERT INTO tbl VALUES ({values_comma_separated})") + columns_to_values = dict([(f"x{i}", i) for i in range(1, num_columns + 1)]) + + # In this test we create and modify roles multiple times along with updating the following variables. + # Then we check that each of the users has access to the expected set of columns. + roles = [] + users_to_roles = dict([(user, []) for user in users]) + roles_to_columns = {} + + # Checks that each of the users can access the expected set of columns and can't access other columns. + def check(): + for user in random.sample(users, len(users)): + expected_roles = users_to_roles[user] + expected_columns = list( + set(sum([roles_to_columns[role] for role in expected_roles], [])) + ) + expected_result = sorted( + [columns_to_values[column] for column in expected_columns] + ) + query = " UNION ALL ".join( + [ + f"SELECT * FROM viewIfPermitted(SELECT {column} AS c FROM tbl ELSE null('c Int64'))" + for column in columns + ] + ) + result = instance.query(query, user=user).splitlines() + result = sorted([int(value) for value in result]) + ok = result == expected_result + if not ok: + print(f"Show grants for {user}:") + print( + instance.query( + "SHOW GRANTS FOR " + ", ".join([user] + expected_roles) + ) + ) + print(f"Expected result: {expected_result}") + print(f"Got unexpected result: {result}") + assert ok + + # Grants one of our roles a permission to access one of the columns. + def grant_column(): + columns_used_in_roles = sum(roles_to_columns.values(), []) + columns_to_choose = [ + column for column in columns if column not in columns_used_in_roles + ] + if not columns_to_choose or not roles: + return False + column = random.choice(columns_to_choose) + role = random.choice(roles) + instance.query(f"GRANT SELECT({column}) ON tbl TO {role}") + roles_to_columns[role].append(column) + return True + + # Revokes a permission to access one of the granted column from all our roles. + def revoke_column(): + columns_used_in_roles = sum(roles_to_columns.values(), []) + columns_to_choose = list(set(columns_used_in_roles)) + if not columns_to_choose or not roles: + return False + column = random.choice(columns_to_choose) + roles_str = ", ".join(roles) + instance.query(f"REVOKE SELECT({column}) ON tbl FROM {roles_str}") + for role in roles_to_columns: + if column in roles_to_columns[role]: + roles_to_columns[role].remove(column) + return True + + # Creates a role and grants it to one of the users. + def create_role(): + for role in ["R1", "R2", "R3"]: + if role not in roles: + instance.query(f"CREATE ROLE {role}") + roles.append(role) + if role not in roles_to_columns: + roles_to_columns[role] = [] + if "R1" not in users_to_roles["A"]: + instance.query("GRANT R1 TO A") + users_to_roles["A"].append("R1") + elif "R2" not in users_to_roles["B"]: + instance.query("GRANT R2 TO B") + users_to_roles["B"].append("R2") + elif "R3" not in users_to_roles["B"]: + instance.query("GRANT R3 TO R2") + users_to_roles["B"].append("R3") + elif "R3" not in users_to_roles["C"]: + instance.query("GRANT R3 TO C") + users_to_roles["C"].append("R3") + else: + return False + return True + + # Drops one of our roles. + def drop_role(): + if not roles: + return False + role = random.choice(roles) + instance.query(f"DROP ROLE {role}") + roles.remove(role) + for u in users_to_roles: + if role in users_to_roles[u]: + users_to_roles[u].remove(role) + del roles_to_columns[role] + if (role == "R2") and ("R3" in users_to_roles["B"]): + users_to_roles["B"].remove("R3") + return True + + # Modifies some grants or roles randomly. + def modify(): + while True: + rnd = random.random() + if rnd < 0.4: + if grant_column(): + break + elif rnd < 0.5: + if revoke_column(): + break + elif rnd < 0.9: + if create_role(): + break + else: + if drop_role(): + break + + def maybe_modify(): + if random.random() < 0.9: + modify() + modify() + + # Sleeping is necessary in this test because the role cache in ClickHouse has expiration timeout. + def maybe_sleep(): + if random.random() < 0.1: + # "role_cache_expiration_time_seconds" is set to 2 seconds in the test configuration. + # We need a sleep longer than that in this test sometimes. + seconds = random.random() * 5 + print(f"Sleeping {seconds} seconds") + time.sleep(seconds) + + # Main part of the test. + start_time = time.time() + end_time = start_time + test_time + + while time.time() < end_time: + check() + maybe_sleep() + maybe_modify() + maybe_sleep() + + check() + + instance.query("DROP USER " + ", ".join(users)) + instance.query("DROP ROLE " + ", ".join(roles)) + instance.query("DROP TABLE tbl") From 7ef5c366e873c4fd99f257eefbb3a350848e308c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 4 Aug 2024 13:33:50 +0200 Subject: [PATCH 1286/1722] Fix expiration in RoleCache. --- src/Access/RoleCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 2d94df2eea5..cc1f1520b67 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -120,7 +120,7 @@ void RoleCache::collectEnabledRoles(EnabledRoles & enabled_roles, SubscriptionsO SubscriptionsOnRoles new_subscriptions_on_roles; new_subscriptions_on_roles.reserve(subscriptions_on_roles.size()); - auto get_role_function = [this, &subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, subscriptions_on_roles); }; + auto get_role_function = [this, &new_subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, new_subscriptions_on_roles); }; for (const auto & current_role : enabled_roles.params.current_roles) collectRoles(*new_info, skip_ids, get_role_function, current_role, true, false); From 34eca80837837b1856d2d00bb86290abe41ea20c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 22 Aug 2024 12:34:56 +0000 Subject: [PATCH 1287/1722] Speedup test 02150_index_hypothesis_race_long --- .../0_stateless/02150_index_hypothesis_race_long.sh | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh index 5c432350768..92b78a6d27a 100755 --- a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -11,13 +11,21 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDE $CLICKHOUSE_CLIENT -q "INSERT INTO t_index_hypothesis SELECT number, number + 1 FROM numbers(10000000)" -for _ in {0..30}; do +run_query() { output=`$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_index_hypothesis WHERE a = b"` if [[ $output != "0" ]]; then echo "output: $output, expected: 0" exit 1 fi -done +} + +export -f run_query +parallel -j 8 run_query ::: {0..30} + +if [ $? -ne 0 ]; then + echo FAILED + exit 1 +fi echo OK From 54dd3afd49df9c92cd3621a5cec4c7464c341a71 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 14:52:17 +0200 Subject: [PATCH 1288/1722] Turn off fault injection for insert in 01396_inactive_replica_cleanup_nodes_zookeeper --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index bff85b3e29f..9ea15071856 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -23,11 +23,10 @@ $CLICKHOUSE_CLIENT -n --query " DETACH TABLE r2; " -$CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 1 --min_insert_block_size_bytes 1 --max_insert_threads 16 --query "INSERT INTO r1 SELECT * FROM numbers_mt(${SCALE})" +# insert_keeper_fault_injection_probability=0 -- can slowdown insert a lot (produce a lot of parts) +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --max_block_size 1 --min_insert_block_size_rows 1 --min_insert_block_size_bytes 1 --max_insert_threads 16 --query "INSERT INTO r1 SELECT * FROM numbers_mt(${SCALE})" -# Now wait for cleanup thread - for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break; From 7a740819b9551a291827b9d37b8b724612587a20 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 14:53:15 +0200 Subject: [PATCH 1289/1722] Accidentally deleted comment --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 9ea15071856..80e9253af2c 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT -n --query " $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --max_block_size 1 --min_insert_block_size_rows 1 --min_insert_block_size_bytes 1 --max_insert_threads 16 --query "INSERT INTO r1 SELECT * FROM numbers_mt(${SCALE})" +# Now wait for cleanup thread for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break; From b3f084459f60b1e31c32736573af0810dee99230 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 14:53:53 +0200 Subject: [PATCH 1290/1722] fix black --- tests/integration/test_storage_hdfs/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 31cc8609eb4..b18940b7290 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1259,8 +1259,13 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): def test_hive_partitioning_with_one_parameter(started_cluster): hdfs_api = started_cluster.hdfs_api - hdfs_api.write_data(f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n") - assert hdfs_api.read_data(f"/column0=Elizabeth/file_1") == f"column0,column1\nElizabeth,Gordon\n" + hdfs_api.write_data( + f"/column0=Elizabeth/file_1", f"column0,column1\nElizabeth,Gordon\n" + ) + assert ( + hdfs_api.read_data(f"/column0=Elizabeth/file_1") + == f"column0,column1\nElizabeth,Gordon\n" + ) r = node1.query( "SELECT column0 FROM hdfs('hdfs://hdfs1:9000/column0=Elizabeth/file_1', 'CSVWithNames')", @@ -1269,6 +1274,7 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert r == f"Elizabeth\n" + def test_hive_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( From 8d14d8523098a42cd778ef50a9b066508da8919c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 15:24:33 +0200 Subject: [PATCH 1291/1722] fix black --- tests/integration/test_storage_hdfs/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index b18940b7290..7a92e8adb0d 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1274,7 +1274,6 @@ def test_hive_partitioning_with_one_parameter(started_cluster): assert r == f"Elizabeth\n" - def test_hive_partitioning_without_setting(started_cluster): hdfs_api = started_cluster.hdfs_api hdfs_api.write_data( From add4718634317304f652579a9f201c3b81c96a7d Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Thu, 22 Aug 2024 06:37:27 -0700 Subject: [PATCH 1292/1722] Update README.md - Meetups update Fixed one meetup location; Added more meetups --- README.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 5e66b9da73e..c9474ef0fc0 100644 --- a/README.md +++ b/README.md @@ -45,9 +45,17 @@ The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 * [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 -* [New York Meetup (Ramp)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 +* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 * [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 +Other upcoming meetups +* [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 +* [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 +* [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 +* [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 +* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 +* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 + ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" * **Recording available**: [**v24.4 Release Call**](https://www.youtube.com/watch?v=dtUqgcfOGmE) All the features of 24.4, one convenient video! Watch it now! From 91e65feaaedd4806875aed3d4be4f07edeefdb71 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 13:42:30 +0000 Subject: [PATCH 1293/1722] fix virtual columns in Merge engine --- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMerge.cpp | 14 +++++++------- .../02890_describe_table_options.reference | 8 ++++++++ 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c4668159759..0b80858800b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,7 +290,7 @@ VirtualColumnsDescription StorageDistributed::createVirtuals() desc.addEphemeral("_shard_num", std::make_shared(), "Deprecated. Use function shardNum instead"); - /// Add virtual columns from table of storage Merges. + /// Add virtual columns from table with Merge engine. desc.addEphemeral("_database", std::make_shared(std::make_shared()), "The name of database which the row comes from"); desc.addEphemeral("_table", std::make_shared(std::make_shared()), "The name of table which the row comes from"); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e88844e2d31..0827321e296 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -642,10 +642,6 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } } - else - { - - } auto child = createPlanForTable( nested_storage_snaphsot, @@ -657,6 +653,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ row_policy_data_opt, modified_context, current_streams); + child.plan.addInterpreterContext(modified_context); if (child.plan.isInitialized()) @@ -914,12 +911,14 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo modified_query_info.table_expression = replacement_table_expression; modified_query_info.planner_context->getOrCreateTableExpressionData(replacement_table_expression); - auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); - if (storage_snapshot_->storage.supportsSubcolumns()) - get_column_options.withSubcolumns(); + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All) + .withExtendedObjects() + .withSubcolumns(storage_snapshot_->storage.supportsSubcolumns()); std::unordered_map column_name_to_node; + /// Consider only non-virtual columns of storage while checking for _table and _database columns. + /// I.e. always override virtual columns with these names from underlying table (if any). if (!storage_snapshot_->tryGetColumn(get_column_options, "_table")) { auto table_name_node = std::make_shared(current_storage_id.table_name); @@ -946,6 +945,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo column_name_to_node.emplace("_database", function_node); } + get_column_options.withVirtuals(); auto storage_columns = storage_snapshot_->metadata->getColumns(); bool with_aliases = /* common_processed_stage == QueryProcessingStage::FetchColumns && */ !storage_columns.getAliases().empty(); diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 9181cb27cb0..b77ef4a0fdf 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -54,6 +54,8 @@ _row_exists UInt8 Persisted mask created by lightweight delete that show wheth _block_number UInt64 Persisted original number of block that was assigned at insert Delta, LZ4 1 _block_offset UInt64 Persisted original number of row in block that was assigned at insert Delta, LZ4 1 _shard_num UInt32 Deprecated. Use function shardNum instead 1 +_database LowCardinality(String) The name of database which the row comes from 1 +_table LowCardinality(String) The name of table which the row comes from 1 SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options; id UInt64 index column 0 0 @@ -87,6 +89,8 @@ _row_exists UInt8 Persisted mask created by lightweight delete that show wheth _block_number UInt64 Persisted original number of block that was assigned at insert Delta, LZ4 0 1 _block_offset UInt64 Persisted original number of row in block that was assigned at insert Delta, LZ4 0 1 _shard_num UInt32 Deprecated. Use function shardNum instead 0 1 +_database LowCardinality(String) The name of database which the row comes from 0 1 +_table LowCardinality(String) The name of table which the row comes from 0 1 arr.size0 UInt64 1 0 t.a String ZSTD(1) 1 0 t.b UInt64 ZSTD(1) 1 0 @@ -144,6 +148,8 @@ _row_exists UInt8 1 _block_number UInt64 1 _block_offset UInt64 1 _shard_num UInt32 1 +_database LowCardinality(String) 1 +_table LowCardinality(String) 1 SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options; id UInt64 0 0 @@ -177,6 +183,8 @@ _row_exists UInt8 0 1 _block_number UInt64 0 1 _block_offset UInt64 0 1 _shard_num UInt32 0 1 +_database LowCardinality(String) 0 1 +_table LowCardinality(String) 0 1 arr.size0 UInt64 1 0 t.a String 1 0 t.b UInt64 1 0 From ce33943b430a9ad512f4942083889dea4decb778 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Aug 2024 15:50:59 +0200 Subject: [PATCH 1294/1722] Fix flaky check --- tests/docker_scripts/stateless_runner.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/docker_scripts/stateless_runner.sh b/tests/docker_scripts/stateless_runner.sh index 40a63f74a6b..d8921a04458 100755 --- a/tests/docker_scripts/stateless_runner.sh +++ b/tests/docker_scripts/stateless_runner.sh @@ -339,7 +339,7 @@ export -f run_tests if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. - NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests \ + NUM_TRIES=1 USE_DATABASE_ORDINARY=1 run_tests \ | sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||: fi From a9e793532ae308767da3bc4da74d9631cd85eb90 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 16:34:14 +0200 Subject: [PATCH 1295/1722] fix shutdown for PeriodicLog --- src/Interpreters/PeriodicLog.cpp | 3 ++- src/Interpreters/PeriodicLog.h | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 9d2891e11eb..15970ca5b81 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Functions/DateTimeTransforms.h" namespace DB { @@ -27,7 +28,7 @@ template void PeriodicLog::shutdown() { stopCollect(); - this->stopFlushThread(); + Base::shutdown(); } template diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index 08c3f7eb23f..ceac8088d40 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -17,6 +17,7 @@ template class PeriodicLog : public SystemLog { using SystemLog::SystemLog; + using Base = SystemLog; public: using TimePoint = std::chrono::system_clock::time_point; @@ -24,12 +25,12 @@ public: /// Launches a background thread to collect metrics with interval void startCollect(size_t collect_interval_milliseconds_); - /// Stop background thread - void stopCollect(); - void shutdown() final; protected: + /// Stop background thread + void stopCollect(); + virtual void stepFunction(TimePoint current_time) = 0; private: From 5340ac5fbc7fba75d6a743d345c0f79dc466df0b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 14:39:19 +0000 Subject: [PATCH 1296/1722] Update version_date.tsv and changelogs after v24.5.5.41-stable --- docs/changelogs/v24.5.5.41-stable.md | 71 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 + 2 files changed, 73 insertions(+) create mode 100644 docs/changelogs/v24.5.5.41-stable.md diff --git a/docs/changelogs/v24.5.5.41-stable.md b/docs/changelogs/v24.5.5.41-stable.md new file mode 100644 index 00000000000..8ba160e31d7 --- /dev/null +++ b/docs/changelogs/v24.5.5.41-stable.md @@ -0,0 +1,71 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.5.41-stable (441d4a6ebe3) FIXME as compared to v24.5.4.49-stable (63b760955a0) + +#### Improvement +* Backported in [#66768](https://github.com/ClickHouse/ClickHouse/issues/66768): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65350](https://github.com/ClickHouse/ClickHouse/issues/65350): Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#65621](https://github.com/ClickHouse/ClickHouse/issues/65621): Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67902](https://github.com/ClickHouse/ClickHouse/issues/67902): Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66884](https://github.com/ClickHouse/ClickHouse/issues/66884): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#65933](https://github.com/ClickHouse/ClickHouse/issues/65933): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#66301](https://github.com/ClickHouse/ClickHouse/issues/66301): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66328](https://github.com/ClickHouse/ClickHouse/issues/66328): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68252](https://github.com/ClickHouse/ClickHouse/issues/68252): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#66155](https://github.com/ClickHouse/ClickHouse/issues/66155): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66454](https://github.com/ClickHouse/ClickHouse/issues/66454): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66226](https://github.com/ClickHouse/ClickHouse/issues/66226): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66680](https://github.com/ClickHouse/ClickHouse/issues/66680): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66604](https://github.com/ClickHouse/ClickHouse/issues/66604): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66360](https://github.com/ClickHouse/ClickHouse/issues/66360): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68064](https://github.com/ClickHouse/ClickHouse/issues/68064): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68158](https://github.com/ClickHouse/ClickHouse/issues/68158): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#66972](https://github.com/ClickHouse/ClickHouse/issues/66972): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66691](https://github.com/ClickHouse/ClickHouse/issues/66691): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#66969](https://github.com/ClickHouse/ClickHouse/issues/66969): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66720](https://github.com/ClickHouse/ClickHouse/issues/66720): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66951](https://github.com/ClickHouse/ClickHouse/issues/66951): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66757](https://github.com/ClickHouse/ClickHouse/issues/66757): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66948](https://github.com/ClickHouse/ClickHouse/issues/66948): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68115](https://github.com/ClickHouse/ClickHouse/issues/68115): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67633](https://github.com/ClickHouse/ClickHouse/issues/67633): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67481](https://github.com/ClickHouse/ClickHouse/issues/67481): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67814](https://github.com/ClickHouse/ClickHouse/issues/67814): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67197](https://github.com/ClickHouse/ClickHouse/issues/67197): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67379](https://github.com/ClickHouse/ClickHouse/issues/67379): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67501](https://github.com/ClickHouse/ClickHouse/issues/67501): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67886](https://github.com/ClickHouse/ClickHouse/issues/67886): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67576](https://github.com/ClickHouse/ClickHouse/issues/67576): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67850](https://github.com/ClickHouse/ClickHouse/issues/67850): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68272](https://github.com/ClickHouse/ClickHouse/issues/68272): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67807](https://github.com/ClickHouse/ClickHouse/issues/67807): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67836](https://github.com/ClickHouse/ClickHouse/issues/67836): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67991](https://github.com/ClickHouse/ClickHouse/issues/67991): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68207](https://github.com/ClickHouse/ClickHouse/issues/68207): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68091](https://github.com/ClickHouse/ClickHouse/issues/68091): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68122](https://github.com/ClickHouse/ClickHouse/issues/68122): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68171](https://github.com/ClickHouse/ClickHouse/issues/68171): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68337](https://github.com/ClickHouse/ClickHouse/issues/68337): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68667](https://github.com/ClickHouse/ClickHouse/issues/68667): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66387](https://github.com/ClickHouse/ClickHouse/issues/66387): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66426](https://github.com/ClickHouse/ClickHouse/issues/66426): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66544](https://github.com/ClickHouse/ClickHouse/issues/66544): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66859](https://github.com/ClickHouse/ClickHouse/issues/66859): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66875](https://github.com/ClickHouse/ClickHouse/issues/66875): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67059](https://github.com/ClickHouse/ClickHouse/issues/67059): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66945](https://github.com/ClickHouse/ClickHouse/issues/66945): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67252](https://github.com/ClickHouse/ClickHouse/issues/67252): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67412](https://github.com/ClickHouse/ClickHouse/issues/67412): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). +* Update version after release. [#67862](https://github.com/ClickHouse/ClickHouse/pull/67862) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68077](https://github.com/ClickHouse/ClickHouse/issues/68077): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..9063d3ef971 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,6 +6,7 @@ v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +15,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From 1ea0163dfe6b3278d8a5e8d86c31b3d63d7a3780 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 22 Aug 2024 16:42:14 +0200 Subject: [PATCH 1297/1722] Fix issue with maps with arrays as keys --- src/Functions/FunctionsHashing.h | 4 ++-- tests/queries/0_stateless/02534_keyed_siphash.reference | 7 ++++++- tests/queries/0_stateless/02534_keyed_siphash.sql | 5 ++++- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 0cf4246fd66..3da0b2cd9be 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -93,9 +93,9 @@ namespace impl if (is_const) i = 0; assert(key0->size() == key1->size()); - if (offsets != nullptr) + if (offsets != nullptr && i > 0) { - const auto * const begin = offsets->begin(); + const auto * const begin = std::upper_bound(offsets->begin(), offsets->end(), i - 1); const auto * upper = std::upper_bound(begin, offsets->end(), i); if (upper != offsets->end()) i = upper - begin; diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 31c0cae8981..8b147025a05 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -244,5 +244,10 @@ Test emtpy arrays and maps 0AD04BFD000000000000000000000000 4761183170873013810 0AD04BFD000000000000000000000000 +Test maps with arrays as keys 16734549324845627102 -D675BB3D687973A238AB891DD99C7047 +1D03941D808D04810D2363A6C107D622 +16734549324845627102 +16734549324845627102 +1D03941D808D04810D2363A6C107D622 +1D03941D808D04810D2363A6C107D622 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index b499d8ef02b..ba3c4a9156d 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -351,5 +351,8 @@ SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), mapFromArrays([], []))); +SELECT 'Test maps with arrays as keys'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3)); -SELECT hex(sipHash128Keyed((0::UInt64, 0::UInt64), map([0], 1, [2], 3))); +SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3))); +SELECT sipHash64Keyed((materialize(1::UInt64), 2::UInt64), map([0], 1, [2], 3)) FROM numbers(2); +SELECT hex(sipHash128Keyed((materialize(1::UInt64), 2::UInt64), map([0], 1, [2], 3))) FROM numbers(2); From a93d1919804d1c8dc7760f20084ade9a09710a47 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 22 Aug 2024 16:43:38 +0200 Subject: [PATCH 1298/1722] Fix typo in test case --- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 +- tests/queries/0_stateless/02534_keyed_siphash.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 8b147025a05..a05446a494e 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -239,7 +239,7 @@ Check bug found fuzzing Test arrays and maps 608E1FF030C9E206185B112C2A25F1A7 ABB65AE97711A2E053E324ED88B1D08B -Test emtpy arrays and maps +Test empty arrays and maps 4761183170873013810 0AD04BFD000000000000000000000000 4761183170873013810 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index ba3c4a9156d..7cfc82512bd 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -346,7 +346,7 @@ INSERT INTO sipHashKeyed_keys FORMAT VALUES ({'a':'b', 'c':'d'}), ({'e':'f', 'g' SELECT hex(sipHash128ReferenceKeyed((0::UInt64, materialize(0::UInt64)), a)) FROM sipHashKeyed_keys ORDER BY a; DROP TABLE sipHashKeyed_keys; -SELECT 'Test emtpy arrays and maps'; +SELECT 'Test empty arrays and maps'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); From 0dc18247df3a290b4fb312325ff3b2a44a3f8357 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 15:10:24 +0000 Subject: [PATCH 1299/1722] Update version_date.tsv and changelogs after v24.6.3.38-stable --- docs/changelogs/v24.6.3.38-stable.md | 83 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 + 2 files changed, 86 insertions(+) create mode 100644 docs/changelogs/v24.6.3.38-stable.md diff --git a/docs/changelogs/v24.6.3.38-stable.md b/docs/changelogs/v24.6.3.38-stable.md new file mode 100644 index 00000000000..01d7e26e31f --- /dev/null +++ b/docs/changelogs/v24.6.3.38-stable.md @@ -0,0 +1,83 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.3.38-stable (4e33c831589) FIXME as compared to v24.6.2.17-stable (5710a8b5c0c) + +#### Improvement +* Backported in [#66770](https://github.com/ClickHouse/ClickHouse/issues/66770): Make allow_experimental_analyzer be controlled by the initiator for distributed queries. This ensures compatibility and correctness during operations in mixed version clusters. [#65777](https://github.com/ClickHouse/ClickHouse/pull/65777) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#66885](https://github.com/ClickHouse/ClickHouse/issues/66885): Fix unexpeced size of low cardinality column in function calls. [#65298](https://github.com/ClickHouse/ClickHouse/pull/65298) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66303](https://github.com/ClickHouse/ClickHouse/issues/66303): Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). +* Backported in [#66330](https://github.com/ClickHouse/ClickHouse/issues/66330): Add missing settings `input_format_csv_skip_first_lines/input_format_tsv_skip_first_lines/input_format_csv_try_infer_numbers_from_strings/input_format_csv_try_infer_strings_from_quoted_tuples` in schema inference cache because they can change the resulting schema. It prevents from incorrect result of schema inference with these settings changed. [#65980](https://github.com/ClickHouse/ClickHouse/pull/65980) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#66157](https://github.com/ClickHouse/ClickHouse/issues/66157): Fixed buffer overflow bug in `unbin`/`unhex` implementation. [#66106](https://github.com/ClickHouse/ClickHouse/pull/66106) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#66210](https://github.com/ClickHouse/ClickHouse/issues/66210): Disable the `merge-filters` optimization introduced in [#64760](https://github.com/ClickHouse/ClickHouse/issues/64760). It may cause an exception if optimization merges two filter expressions and does not apply a short-circuit evaluation. [#66126](https://github.com/ClickHouse/ClickHouse/pull/66126) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66456](https://github.com/ClickHouse/ClickHouse/issues/66456): Fixed a bug in ZooKeeper client: a session could get stuck in unusable state after receiving a hardware error from ZooKeeper. For example, this might happen due to "soft memory limit" in ClickHouse Keeper. [#66140](https://github.com/ClickHouse/ClickHouse/pull/66140) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66228](https://github.com/ClickHouse/ClickHouse/issues/66228): Fix issue in SumIfToCountIfVisitor and signed integers. [#66146](https://github.com/ClickHouse/ClickHouse/pull/66146) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66183](https://github.com/ClickHouse/ClickHouse/issues/66183): Fix rare case with missing data in the result of distributed query, close [#61432](https://github.com/ClickHouse/ClickHouse/issues/61432). [#66174](https://github.com/ClickHouse/ClickHouse/pull/66174) ([vdimir](https://github.com/vdimir)). +* Backported in [#66271](https://github.com/ClickHouse/ClickHouse/issues/66271): Don't throw `TIMEOUT_EXCEEDED` for `none_only_active` mode of `distributed_ddl_output_mode`. [#66218](https://github.com/ClickHouse/ClickHouse/pull/66218) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#66682](https://github.com/ClickHouse/ClickHouse/issues/66682): Fix handling limit for `system.numbers_mt` when no index can be used. [#66231](https://github.com/ClickHouse/ClickHouse/pull/66231) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66587](https://github.com/ClickHouse/ClickHouse/issues/66587): Fixed how the ClickHouse server detects the maximum number of usable CPU cores as specified by cgroups v2 if the server runs in a container such as Docker. In more detail, containers often run their process in the root cgroup which has an empty name. In that case, ClickHouse ignored the CPU limits set by cgroups v2. [#66237](https://github.com/ClickHouse/ClickHouse/pull/66237) ([filimonov](https://github.com/filimonov)). +* Backported in [#66362](https://github.com/ClickHouse/ClickHouse/issues/66362): Fix the `Not-ready set` error when a subquery with `IN` is used in the constraint. [#66261](https://github.com/ClickHouse/ClickHouse/pull/66261) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68066](https://github.com/ClickHouse/ClickHouse/issues/68066): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68566](https://github.com/ClickHouse/ClickHouse/issues/68566): Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68159](https://github.com/ClickHouse/ClickHouse/issues/68159): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#66613](https://github.com/ClickHouse/ClickHouse/issues/66613): Fix `Column identifier is already registered` error with `group_by_use_nulls=true` and new analyzer. [#66400](https://github.com/ClickHouse/ClickHouse/pull/66400) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66693](https://github.com/ClickHouse/ClickHouse/issues/66693): Fix the VALID UNTIL clause in the user definition resetting after a restart. Closes [#66405](https://github.com/ClickHouse/ClickHouse/issues/66405). [#66409](https://github.com/ClickHouse/ClickHouse/pull/66409) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#66577](https://github.com/ClickHouse/ClickHouse/issues/66577): Fix `Cannot find column` error for queries with constant expression in `GROUP BY` key and new analyzer enabled. [#66433](https://github.com/ClickHouse/ClickHouse/pull/66433) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66721](https://github.com/ClickHouse/ClickHouse/issues/66721): Correctly track memory for `Allocator::realloc`. [#66548](https://github.com/ClickHouse/ClickHouse/pull/66548) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#66670](https://github.com/ClickHouse/ClickHouse/issues/66670): Fix reading of uninitialized memory when hashing empty tuples. This closes [#66559](https://github.com/ClickHouse/ClickHouse/issues/66559). [#66562](https://github.com/ClickHouse/ClickHouse/pull/66562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#66952](https://github.com/ClickHouse/ClickHouse/issues/66952): Fix an invalid result for queries with `WINDOW`. This could happen when `PARTITION` columns have sparse serialization and window functions are executed in parallel. [#66579](https://github.com/ClickHouse/ClickHouse/pull/66579) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66956](https://github.com/ClickHouse/ClickHouse/issues/66956): Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66716](https://github.com/ClickHouse/ClickHouse/issues/66716): Fix removing named collections in local storage. [#66599](https://github.com/ClickHouse/ClickHouse/pull/66599) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#66759](https://github.com/ClickHouse/ClickHouse/issues/66759): Fix `Unknown identifier` and `Column is not under aggregate function` errors for queries with the expression `(column IS NULL).` The bug was triggered by [#65088](https://github.com/ClickHouse/ClickHouse/issues/65088), with the disabled analyzer only. [#66654](https://github.com/ClickHouse/ClickHouse/pull/66654) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66751](https://github.com/ClickHouse/ClickHouse/issues/66751): Fix `Method getResultType is not supported for QUERY query node` error when scalar subquery was used as the first argument of IN (with new analyzer). [#66655](https://github.com/ClickHouse/ClickHouse/pull/66655) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68116](https://github.com/ClickHouse/ClickHouse/issues/68116): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67635](https://github.com/ClickHouse/ClickHouse/issues/67635): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67482](https://github.com/ClickHouse/ClickHouse/issues/67482): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#67816](https://github.com/ClickHouse/ClickHouse/issues/67816): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67199](https://github.com/ClickHouse/ClickHouse/issues/67199): TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. [#67129](https://github.com/ClickHouse/ClickHouse/pull/67129) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#67381](https://github.com/ClickHouse/ClickHouse/issues/67381): Fix error `Cannot convert column because it is non constant in source stream but must be constant in result.` for a query that reads from the `Merge` table over the `Distriburted` table with one shard. [#67146](https://github.com/ClickHouse/ClickHouse/pull/67146) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#67244](https://github.com/ClickHouse/ClickHouse/issues/67244): This closes [#67156](https://github.com/ClickHouse/ClickHouse/issues/67156). This closes [#66447](https://github.com/ClickHouse/ClickHouse/issues/66447). The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. [#67178](https://github.com/ClickHouse/ClickHouse/pull/67178) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#67503](https://github.com/ClickHouse/ClickHouse/issues/67503): Fix crash in DistributedAsyncInsert when connection is empty. [#67219](https://github.com/ClickHouse/ClickHouse/pull/67219) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67887](https://github.com/ClickHouse/ClickHouse/issues/67887): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67578](https://github.com/ClickHouse/ClickHouse/issues/67578): Fix execution of nested short-circuit functions. [#67520](https://github.com/ClickHouse/ClickHouse/pull/67520) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68611](https://github.com/ClickHouse/ClickHouse/issues/68611): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#67852](https://github.com/ClickHouse/ClickHouse/issues/67852): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68275](https://github.com/ClickHouse/ClickHouse/issues/68275): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67808](https://github.com/ClickHouse/ClickHouse/issues/67808): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67838](https://github.com/ClickHouse/ClickHouse/issues/67838): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67993](https://github.com/ClickHouse/ClickHouse/issues/67993): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68208](https://github.com/ClickHouse/ClickHouse/issues/68208): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68093](https://github.com/ClickHouse/ClickHouse/issues/68093): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68124](https://github.com/ClickHouse/ClickHouse/issues/68124): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68221](https://github.com/ClickHouse/ClickHouse/issues/68221): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68173](https://github.com/ClickHouse/ClickHouse/issues/68173): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68339](https://github.com/ClickHouse/ClickHouse/issues/68339): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68396](https://github.com/ClickHouse/ClickHouse/issues/68396): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68668](https://github.com/ClickHouse/ClickHouse/issues/68668): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#66599](https://github.com/ClickHouse/ClickHouse/issues/66599) to 24.6: Fix dropping named collection in local storage"'. [#66922](https://github.com/ClickHouse/ClickHouse/pull/66922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#66332](https://github.com/ClickHouse/ClickHouse/issues/66332): Do not raise a NOT_IMPLEMENTED error when getting s3 metrics with a multiple disk configuration. [#65403](https://github.com/ClickHouse/ClickHouse/pull/65403) ([Elena Torró](https://github.com/elenatorro)). +* Backported in [#66142](https://github.com/ClickHouse/ClickHouse/issues/66142): Fix flaky test_storage_s3_queue tests. [#66009](https://github.com/ClickHouse/ClickHouse/pull/66009) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#66389](https://github.com/ClickHouse/ClickHouse/issues/66389): Disable broken cases from 02911_join_on_nullsafe_optimization. [#66310](https://github.com/ClickHouse/ClickHouse/pull/66310) ([vdimir](https://github.com/vdimir)). +* Backported in [#66428](https://github.com/ClickHouse/ClickHouse/issues/66428): Ignore subquery for IN in DDLLoadingDependencyVisitor. [#66395](https://github.com/ClickHouse/ClickHouse/pull/66395) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#66546](https://github.com/ClickHouse/ClickHouse/issues/66546): Add additional log masking in CI. [#66523](https://github.com/ClickHouse/ClickHouse/pull/66523) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#66861](https://github.com/ClickHouse/ClickHouse/issues/66861): Fix data race in S3::ClientCache. [#66644](https://github.com/ClickHouse/ClickHouse/pull/66644) ([Konstantin Morozov](https://github.com/k-morozov)). +* Backported in [#66877](https://github.com/ClickHouse/ClickHouse/issues/66877): Support one more case in JOIN ON ... IS NULL. [#66725](https://github.com/ClickHouse/ClickHouse/pull/66725) ([vdimir](https://github.com/vdimir)). +* Backported in [#67061](https://github.com/ClickHouse/ClickHouse/issues/67061): Increase asio pool size in case the server is tiny. [#66761](https://github.com/ClickHouse/ClickHouse/pull/66761) ([alesapin](https://github.com/alesapin)). +* Backported in [#66940](https://github.com/ClickHouse/ClickHouse/issues/66940): Small fix in realloc memory tracking. [#66820](https://github.com/ClickHouse/ClickHouse/pull/66820) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67254](https://github.com/ClickHouse/ClickHouse/issues/67254): Followup [#66725](https://github.com/ClickHouse/ClickHouse/issues/66725). [#66869](https://github.com/ClickHouse/ClickHouse/pull/66869) ([vdimir](https://github.com/vdimir)). +* Backported in [#67414](https://github.com/ClickHouse/ClickHouse/issues/67414): CI: Fix build results for release branches. [#67402](https://github.com/ClickHouse/ClickHouse/pull/67402) ([Max K.](https://github.com/maxknv)). +* Update version after release. [#67909](https://github.com/ClickHouse/ClickHouse/pull/67909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68079](https://github.com/ClickHouse/ClickHouse/issues/68079): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..cc168f58862 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,9 +3,11 @@ v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +16,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From 0b9c24f31d548c87deca3334282c14fc78a295ba Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 15 Aug 2024 12:09:50 +0000 Subject: [PATCH 1300/1722] write metadata to disk and keeper in the same format --- src/Storages/ColumnsDescription.cpp | 30 +++++--- src/Storages/ColumnsDescription.h | 6 +- .../__init__.py | 0 .../config/enable_keeper.xml | 26 +++++++ .../config/users.xml | 8 +++ .../test.py | 71 +++++++++++++++++++ 6 files changed, 128 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0d724245b49..0212bbd6fff 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -113,7 +113,15 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && ast_to_str(ttl) == ast_to_str(other.ttl); } -void ColumnDescription::writeText(WriteBuffer & buf) const +String formatASTStateAware(IAST & ast, IAST::FormatState & state) +{ + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true, false); + ast.formatImpl(settings, state, IAST::FormatStateStacked()); + return buf.str(); +} + +void ColumnDescription::writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const { /// NOTE: Serialization format is insane. @@ -126,20 +134,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeChar('\t', buf); DB::writeText(DB::toString(default_desc.kind), buf); writeChar('\t', buf); - writeEscapedString(queryToString(default_desc.expression), buf); + writeEscapedString(formatASTStateAware(*default_desc.expression, state), buf); } - if (!comment.empty()) + if (!comment.empty() && include_comment) { writeChar('\t', buf); DB::writeText("COMMENT ", buf); - writeEscapedString(queryToString(ASTLiteral(Field(comment))), buf); + auto ast = ASTLiteral(Field(comment)); + writeEscapedString(formatASTStateAware(ast, state), buf); } if (codec) { writeChar('\t', buf); - writeEscapedString(queryToString(codec), buf); + writeEscapedString(formatASTStateAware(*codec, state), buf); } if (!settings.empty()) @@ -150,21 +159,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const ASTSetQuery ast; ast.is_standalone = false; ast.changes = settings; - writeEscapedString(queryToString(ast), buf); + writeEscapedString(formatASTStateAware(ast, state), buf); DB::writeText(")", buf); } if (!statistics.empty()) { writeChar('\t', buf); - writeEscapedString(queryToString(statistics.getAST()), buf); + writeEscapedString(formatASTStateAware(*statistics.getAST(), state), buf); } if (ttl) { writeChar('\t', buf); DB::writeText("TTL ", buf); - writeEscapedString(queryToString(ttl), buf); + writeEscapedString(formatASTStateAware(*ttl, state), buf); } writeChar('\n', buf); @@ -895,16 +904,17 @@ void ColumnsDescription::resetColumnTTLs() } -String ColumnsDescription::toString() const +String ColumnsDescription::toString(bool include_comments) const { WriteBufferFromOwnString buf; + IAST::FormatState ast_format_state; writeCString("columns format version: 1\n", buf); DB::writeText(columns.size(), buf); writeCString(" columns:\n", buf); for (const ColumnDescription & column : columns) - column.writeText(buf); + column.writeText(buf, ast_format_state, include_comments); return buf.str(); } diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index f0760160f0a..c89c26501e8 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -104,7 +104,7 @@ struct ColumnDescription bool operator==(const ColumnDescription & other) const; bool operator!=(const ColumnDescription & other) const { return !(*this == other); } - void writeText(WriteBuffer & buf) const; + void writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const; void readText(ReadBuffer & buf); }; @@ -137,7 +137,7 @@ public: /// NOTE Must correspond with Nested::flatten function. void flattenNested(); /// TODO: remove, insert already flattened Nested columns. - bool operator==(const ColumnsDescription & other) const { return columns == other.columns; } + bool operator==(const ColumnsDescription & other) const { return toString(false) == other.toString(false); } bool operator!=(const ColumnsDescription & other) const { return !(*this == other); } auto begin() const { return columns.begin(); } @@ -221,7 +221,7 @@ public: /// Does column has non default specified compression codec bool hasCompressionCodec(const String & column_name) const; - String toString() const; + String toString(bool include_comments = true) const; static ColumnsDescription parse(const String & str); size_t size() const diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml new file mode 100644 index 00000000000..4ca4f604ec3 --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml @@ -0,0 +1,26 @@ + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 20000 + + + + 1 + localhost + 9444 + + + + + + + localhost + 2181 + + 20000 + + \ No newline at end of file diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml new file mode 100644 index 00000000000..c5de0b6819c --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml @@ -0,0 +1,8 @@ + + + + default + + + + \ No newline at end of file diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py new file mode 100644 index 00000000000..e0c15e18c23 --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py @@ -0,0 +1,71 @@ +import pytest +import random +import string + +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=[ + "config/enable_keeper.xml", + "config/users.xml", + ], + stay_alive=True, + with_minio=True, + macros={"shard": 1, "replica": 1}, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def randomize_table_name(table_name, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{table_name}_{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + +@pytest.mark.parametrize("engine", ["ReplicatedMergeTree"]) +def test_aliases_in_default_expr_not_break_table_structure(start_cluster, engine): + """ + Making sure that using aliases in columns' default expressions does not lead to having different columns metadata in ZooKeeper and on disk. + Issue: https://github.com/ClickHouse/clickhouse-private/issues/5150 + """ + + data = '{"event": {"col1-key": "col1-val", "col2-key": "col2-val"}}' + + table_name = randomize_table_name("t") + + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ( + `data` String, + `col1` String DEFAULT JSONExtractString(JSONExtractString(data, 'event') AS event, 'col1-key'), + `col2` String MATERIALIZED JSONExtractString(JSONExtractString(data, 'event') AS event, 'col2-key') + ) + ENGINE = {engine}('/test/{table_name}', '{{replica}}') + ORDER BY col1 + """ + ) + + node.restart_clickhouse() + + node.query( + f""" + INSERT INTO {table_name} (data) VALUES ('{data}'); + """ + ) + assert node.query(f"SELECT data FROM {table_name}").strip() == data + assert node.query(f"SELECT col1 FROM {table_name}").strip() == "col1-val" + assert node.query(f"SELECT col2 FROM {table_name}").strip() == "col2-val" + + node.query(f"DROP TABLE {table_name}") From 859d2bfe273f571458be6f007761bc8c743d589a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 17:18:06 +0200 Subject: [PATCH 1301/1722] move stopFlushThread to SystemLogBase --- src/Common/SystemLogBase.cpp | 19 +++++++++++++++++++ src/Common/SystemLogBase.h | 2 ++ src/Interpreters/PeriodicLog.cpp | 6 +++--- src/Interpreters/PeriodicLog.h | 2 +- src/Interpreters/SystemLog.cpp | 21 +-------------------- src/Interpreters/SystemLog.h | 7 +------ 6 files changed, 27 insertions(+), 30 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 127c8862a35..45f4eb1c5a6 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -273,6 +273,25 @@ void SystemLogBase::startup() saving_thread = std::make_unique([this] { savingThreadFunction(); }); } +template +void SystemLogBase::stopFlushThread() +{ + { + std::lock_guard lock(thread_mutex); + + if (!saving_thread || !saving_thread->joinable()) + return; + + if (is_shutdown) + return; + + is_shutdown = true; + queue->shutdown(); + } + + saving_thread->join(); +} + template void SystemLogBase::add(LogElement element) { diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0d7b04d5c57..0942e920a42 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -216,6 +216,8 @@ public: static consteval bool shouldTurnOffLogger() { return false; } protected: + void stopFlushThread() final; + std::shared_ptr> queue; }; } diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 15970ca5b81..1b285aad3ff 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -11,7 +11,7 @@ void PeriodicLog::startCollect(size_t collect_interval_milliseconds_ { collect_interval_milliseconds = collect_interval_milliseconds_; is_shutdown_metric_thread = false; - flush_thread = std::make_unique([this] { threadFunction(); }); + collecting_thread = std::make_unique([this] { threadFunction(); }); } template @@ -20,8 +20,8 @@ void PeriodicLog::stopCollect() bool old_val = false; if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) return; - if (flush_thread) - flush_thread->join(); + if (collecting_thread) + collecting_thread->join(); } template diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index ceac8088d40..8254a02434a 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -36,7 +36,7 @@ protected: private: void threadFunction(); - std::unique_ptr flush_thread; + std::unique_ptr collecting_thread; size_t collect_interval_milliseconds; std::atomic is_shutdown_metric_thread{false}; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 832c39bfaf8..6a3ec197c6e 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -402,32 +402,13 @@ SystemLog::SystemLog( template void SystemLog::shutdown() { - stopFlushThread(); + Base::stopFlushThread(); auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) table->flushAndShutdown(); } -template -void SystemLog::stopFlushThread() -{ - { - std::lock_guard lock(thread_mutex); - - if (!saving_thread || !saving_thread->joinable()) - return; - - if (is_shutdown) - return; - - is_shutdown = true; - queue->shutdown(); - } - - saving_thread->join(); -} - template void SystemLog::savingThreadFunction() diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 9e1af3578bd..31652c1af67 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -125,8 +125,6 @@ public: void shutdown() override; - void stopFlushThread() override; - /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. * This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created. @@ -136,10 +134,7 @@ public: protected: LoggerPtr log; - using ISystemLog::is_shutdown; - using ISystemLog::saving_thread; - using ISystemLog::thread_mutex; - using Base::queue; + using Base::queue; StoragePtr getStorage() const; From 837f2bba8a136170b6aa8800b6b30849a9310e5f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:23:45 +0200 Subject: [PATCH 1302/1722] init --- .../0_stateless/00080_show_tables_and_system_tables.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index a58f9ddb0ac..02e3645ece0 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -6,8 +6,8 @@ CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.A (A UInt8) ENGINE = TinyLog; CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.B (A UInt8) ENGINE = TinyLog; -SHOW TABLES from {CLICKHOUSE_DATABASE:Identifier}; -SHOW TABLES in system where engine like '%System%' and name in ('numbers', 'one'); +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; +SHOW TABLES IN system WHERE engine LIKE '%System%' AND name IN ('numbers', 'one') AND database = 'system'; SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = currentDatabase() ORDER BY name FORMAT TSVRaw; @@ -16,7 +16,7 @@ SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_log(id UInt64) ENGINE = Log; CREATE MATERIALIZED VIEW {CLICKHOUSE_DATABASE:Identifier}.test_materialized ENGINE = Log AS SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_log; -SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log' and database=currentDatabase(); +SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log' AND database=currentDatabase(); DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; From 51fbc629c6dff4653e687228b0507947516072bb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 15:42:17 +0000 Subject: [PATCH 1303/1722] Update version_date.tsv and changelogs after v24.7.3.47-stable --- docs/changelogs/v24.7.3.47-stable.md | 55 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++ 2 files changed, 59 insertions(+) create mode 100644 docs/changelogs/v24.7.3.47-stable.md diff --git a/docs/changelogs/v24.7.3.47-stable.md b/docs/changelogs/v24.7.3.47-stable.md new file mode 100644 index 00000000000..e5f23a70fe1 --- /dev/null +++ b/docs/changelogs/v24.7.3.47-stable.md @@ -0,0 +1,55 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.3.47-stable (2e50fe27a14) FIXME as compared to v24.7.2.13-stable (6e41f601b2f) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68232](https://github.com/ClickHouse/ClickHouse/issues/68232): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#67969](https://github.com/ClickHouse/ClickHouse/issues/67969): Fixed reading of subcolumns after `ALTER ADD COLUMN` query. [#66243](https://github.com/ClickHouse/ClickHouse/pull/66243) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68068](https://github.com/ClickHouse/ClickHouse/issues/68068): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#67637](https://github.com/ClickHouse/ClickHouse/issues/67637): Fix for occasional deadlock in Context::getDDLWorker. [#66843](https://github.com/ClickHouse/ClickHouse/pull/66843) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#67820](https://github.com/ClickHouse/ClickHouse/issues/67820): Fix possible deadlock on query cancel with parallel replicas. [#66905](https://github.com/ClickHouse/ClickHouse/pull/66905) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67818](https://github.com/ClickHouse/ClickHouse/issues/67818): Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. [#67122](https://github.com/ClickHouse/ClickHouse/pull/67122) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67766](https://github.com/ClickHouse/ClickHouse/issues/67766): Fix crash of `uniq` and `uniqTheta ` with `tuple()` argument. Closes [#67303](https://github.com/ClickHouse/ClickHouse/issues/67303). [#67306](https://github.com/ClickHouse/ClickHouse/pull/67306) ([flynn](https://github.com/ucasfl)). +* Backported in [#67881](https://github.com/ClickHouse/ClickHouse/issues/67881): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68613](https://github.com/ClickHouse/ClickHouse/issues/68613): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#67854](https://github.com/ClickHouse/ClickHouse/issues/67854): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68278](https://github.com/ClickHouse/ClickHouse/issues/68278): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68040](https://github.com/ClickHouse/ClickHouse/issues/68040): Fix creation of view with recursive CTE. [#67587](https://github.com/ClickHouse/ClickHouse/pull/67587) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#68038](https://github.com/ClickHouse/ClickHouse/issues/68038): Fix crash on `percent_rank`. `percent_rank`'s default frame type is changed to `range unbounded preceding and unbounded following`. `IWindowFunction`'s default window frame is considered and now window functions without window frame definition in sql can be put into different `WindowTransfomer`s properly. [#67661](https://github.com/ClickHouse/ClickHouse/pull/67661) ([lgbo](https://github.com/lgbo-ustc)). +* Backported in [#67713](https://github.com/ClickHouse/ClickHouse/issues/67713): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67840](https://github.com/ClickHouse/ClickHouse/issues/67840): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67995](https://github.com/ClickHouse/ClickHouse/issues/67995): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68224](https://github.com/ClickHouse/ClickHouse/issues/68224): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68095](https://github.com/ClickHouse/ClickHouse/issues/68095): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68126](https://github.com/ClickHouse/ClickHouse/issues/68126): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68223](https://github.com/ClickHouse/ClickHouse/issues/68223): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68175](https://github.com/ClickHouse/ClickHouse/issues/68175): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68341](https://github.com/ClickHouse/ClickHouse/issues/68341): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68398](https://github.com/ClickHouse/ClickHouse/issues/68398): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68669](https://github.com/ClickHouse/ClickHouse/issues/68669): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67518](https://github.com/ClickHouse/ClickHouse/issues/67518): Split slow test 03036_dynamic_read_subcolumns. [#66954](https://github.com/ClickHouse/ClickHouse/pull/66954) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67516](https://github.com/ClickHouse/ClickHouse/issues/67516): Split 01508_partition_pruning_long. [#66983](https://github.com/ClickHouse/ClickHouse/pull/66983) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67529](https://github.com/ClickHouse/ClickHouse/issues/67529): Reduce max time of 00763_long_lock_buffer_alter_destination_table. [#67185](https://github.com/ClickHouse/ClickHouse/pull/67185) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67803](https://github.com/ClickHouse/ClickHouse/issues/67803): Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. [#67359](https://github.com/ClickHouse/ClickHouse/pull/67359) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67643](https://github.com/ClickHouse/ClickHouse/issues/67643): [Green CI] Fix potentially flaky test_mask_sensitive_info integration test. [#67506](https://github.com/ClickHouse/ClickHouse/pull/67506) ([Alexey Katsman](https://github.com/alexkats)). +* Backported in [#67609](https://github.com/ClickHouse/ClickHouse/issues/67609): Fix test_zookeeper_config_load_balancing after adding the xdist worker name to the instance. [#67590](https://github.com/ClickHouse/ClickHouse/pull/67590) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#67871](https://github.com/ClickHouse/ClickHouse/issues/67871): Fix 02434_cancel_insert_when_client_dies. [#67600](https://github.com/ClickHouse/ClickHouse/pull/67600) ([vdimir](https://github.com/vdimir)). +* Backported in [#67704](https://github.com/ClickHouse/ClickHouse/issues/67704): Fix 02910_bad_logs_level_in_local in fast tests. [#67603](https://github.com/ClickHouse/ClickHouse/pull/67603) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#67689](https://github.com/ClickHouse/ClickHouse/issues/67689): Fix 01605_adaptive_granularity_block_borders. [#67605](https://github.com/ClickHouse/ClickHouse/pull/67605) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67827](https://github.com/ClickHouse/ClickHouse/issues/67827): Try fix 03143_asof_join_ddb_long. [#67620](https://github.com/ClickHouse/ClickHouse/pull/67620) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#67892](https://github.com/ClickHouse/ClickHouse/issues/67892): Revert "Merge pull request [#66510](https://github.com/ClickHouse/ClickHouse/issues/66510) from canhld94/fix_trivial_count_non_deterministic_func". [#67800](https://github.com/ClickHouse/ClickHouse/pull/67800) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68081](https://github.com/ClickHouse/ClickHouse/issues/68081): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Update version after release. [#68044](https://github.com/ClickHouse/ClickHouse/pull/68044) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68269](https://github.com/ClickHouse/ClickHouse/issues/68269): [Green CI] Fix test 01903_correct_block_size_prediction_with_default. [#68203](https://github.com/ClickHouse/ClickHouse/pull/68203) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#68432](https://github.com/ClickHouse/ClickHouse/issues/68432): tests: make 01600_parts_states_metrics_long better. [#68265](https://github.com/ClickHouse/ClickHouse/pull/68265) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68538](https://github.com/ClickHouse/ClickHouse/issues/68538): CI: Native build for package_aarch64. [#68457](https://github.com/ClickHouse/ClickHouse/pull/68457) ([Max K.](https://github.com/maxknv)). +* Backported in [#68555](https://github.com/ClickHouse/ClickHouse/issues/68555): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..6ef5ace4ba6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,11 +1,14 @@ v24.8.1.2684-lts 2024-08-21 +v24.7.3.47-stable 2024-08-22 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +17,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From 5f61e193401c5fa46db03542cb88ba4188ed00e9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:46:47 +0200 Subject: [PATCH 1304/1722] small fixes --- docs/ru/getting-started/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 4a0ec258c64..5bce41ec07a 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -22,7 +22,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su ### Из deb-пакетов {#install-from-deb-packages} -Ð¯Ð½Ð´ÐµÐºÑ Ñ€ÐµÐºÐ¾Ð¼ÐµÐ½Ð´ÑƒÐµÑ‚ иÑпользовать официальные Ñкомпилированные `deb`-пакеты Ð´Ð»Ñ Debian или Ubuntu. Ð”Ð»Ñ ÑƒÑтановки пакетов выполните: +РекомендуетÑÑ Ð¸Ñпользовать официальные Ñкомпилированные `deb`-пакеты Ð´Ð»Ñ Debian или Ubuntu. Ð”Ð»Ñ ÑƒÑтановки пакетов выполните: ``` bash sudo apt-get install -y apt-transport-https ca-certificates curl gnupg @@ -55,7 +55,7 @@ clickhouse-client # or "clickhouse-client --password" if you've set up a passwor ::: ### Из rpm-пакетов {#from-rpm-packages} -Команда ClickHouse в ЯндекÑе рекомендует иÑпользовать официальные предкомпилированные `rpm`-пакеты Ð´Ð»Ñ CentOS, RedHat и вÑех оÑтальных диÑтрибутивов Linux, оÑнованных на rpm. +Команда ClickHouse рекомендует иÑпользовать официальные предкомпилированные `rpm`-пакеты Ð´Ð»Ñ CentOS, RedHat и вÑех оÑтальных диÑтрибутивов Linux, оÑнованных на rpm. #### УÑтановка официального Ñ€ÐµÐ¿Ð¾Ð·Ð¸Ñ‚Ð¾Ñ€Ð¸Ñ @@ -102,7 +102,7 @@ sudo yum install clickhouse-server clickhouse-client ### Из tgz-архивов {#from-tgz-archives} -Команда ClickHouse в ЯндекÑе рекомендует иÑпользовать предкомпилированные бинарники из `tgz`-архивов Ð´Ð»Ñ Ð²Ñех диÑтрибутивов, где невозможна уÑтановка `deb`- и `rpm`- пакетов. +Команда ClickHouse рекомендует иÑпользовать предкомпилированные бинарники из `tgz`-архивов Ð´Ð»Ñ Ð²Ñех диÑтрибутивов, где невозможна уÑтановка `deb`- и `rpm`- пакетов. ИнтереÑующую верÑию архивов можно Ñкачать вручную Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ `curl` или `wget` из Ñ€ÐµÐ¿Ð¾Ð·Ð¸Ñ‚Ð¾Ñ€Ð¸Ñ https://packages.clickhouse.com/tgz/. ПоÑле Ñтого архивы нужно раÑпаковать и воÑпользоватьÑÑ Ñкриптами уÑтановки. Пример уÑтановки Ñамой Ñвежей верÑии: From 980b02bfd67defbbdf78165e8225fb754d722d7a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:48:57 +0200 Subject: [PATCH 1305/1722] fix compatibility with en version --- docs/ru/getting-started/install.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 5bce41ec07a..f8a660fbec9 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -31,9 +31,17 @@ curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | s echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update +``` +#### УÑтановка ClickHouse server и client + +```bash sudo apt-get install -y clickhouse-server clickhouse-client +``` +#### ЗапуÑк ClickHouse server + +```bash sudo service clickhouse-server start clickhouse-client # or "clickhouse-client --password" if you've set up a password. ``` From 7c3a013d56c1dbd5b72f04f6be61f007004aaefa Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Thu, 22 Aug 2024 16:53:30 +0100 Subject: [PATCH 1306/1722] Update newjson.md --- docs/en/sql-reference/data-types/newjson.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 9e43216df6c..f7fc7e1498e 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -70,7 +70,7 @@ SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json └────────────────────────────────────────────────┘ ``` -CAST from named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. +CAST from `JSON`, named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. ## Reading JSON paths as subcolumns From 28fbd8a4eff4eafa7db99eb37e38376ffda11763 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:56:16 +0200 Subject: [PATCH 1307/1722] fix stateless tests --- .../queries/0_stateless/03203_hive_style_partitioning.reference | 2 -- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index acdadc2510b..a9d856babce 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -34,8 +34,6 @@ Cross Elizabeth Array(Int64) LowCardinality(Float64) 101 2070 -4081 -2070 2070 b 1 diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index b3d196924af..6734c5f14ad 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -32,7 +32,7 @@ SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMI $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; -SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/sample.parquet') LIMIT 10; +SELECT *, non_existing_column FROM file('$CURDIR/data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "UNKNOWN_IDENTIFIER" From 9c0e1df1663dd5c56066dd615fc3cafe6408d308 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 17:58:15 +0200 Subject: [PATCH 1308/1722] Fix flaky test 00989_parallel_parts_loading --- tests/queries/0_stateless/00989_parallel_parts_loading.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index 407e124f137..dc074241ff6 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings, no-random-merge-tree-settings +-- small insert block size can make insert terribly slow, especially with some build like msan DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; From 0bd8ebf62616ce882b0ebc46945c837a5a91ba44 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 22 Aug 2024 17:58:56 +0200 Subject: [PATCH 1309/1722] Update README.md adding community call. resolving recent recordings --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c9474ef0fc0..9099fd48659 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.8 Community Call](https://clickhouse.com/company/events/v24-8-community-release-call) - August 20 +* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 266 ## Upcoming Events @@ -58,7 +58,7 @@ Other upcoming meetups ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v24.4 Release Call**](https://www.youtube.com/watch?v=dtUqgcfOGmE) All the features of 24.4, one convenient video! Watch it now! +* **Recording available**: [**v24.8 LTS Release Call**](https://www.youtube.com/watch?v=AeLmp2jc51k) All the features of 24.8 LTS, one convenient video! Watch it now! ## Interested in joining ClickHouse and making it your full-time job? From 52cdd88eb6d7bbb5d395dd80445655ad47c83c92 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 17:59:10 +0200 Subject: [PATCH 1310/1722] Better comment --- tests/queries/0_stateless/00989_parallel_parts_loading.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index dc074241ff6..3b73e6a0e3c 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,5 +1,5 @@ -- Tags: no-random-settings, no-random-merge-tree-settings --- small insert block size can make insert terribly slow, especially with some build like msan +-- small number of insert threads can make insert terribly slow, especially with some build like msan DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; From e7b89537bf1bb760c6082f04de4668bd1c00f33a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 18:02:42 +0200 Subject: [PATCH 1311/1722] fix style --- src/Interpreters/PeriodicLog.cpp | 1 - src/Interpreters/SystemLog.h | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 1b285aad3ff..22bc14856c4 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Functions/DateTimeTransforms.h" namespace DB { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 31652c1af67..c03f9370068 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -134,7 +134,7 @@ public: protected: LoggerPtr log; - using Base::queue; + using Base::queue; StoragePtr getStorage() const; From 1692360233593e635c5a7797847bdfd8a0ffa33e Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 22 Aug 2024 18:12:38 +0200 Subject: [PATCH 1312/1722] Update README.md 26 and 266 are different --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 9099fd48659..83a5c05c667 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 266 +* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 26 ## Upcoming Events From 4264fbc037accedecebcd8122910e4406e92cd58 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 16:16:47 +0000 Subject: [PATCH 1313/1722] Update version_date.tsv and changelogs after v24.8.2.3-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.8.2.3-lts.md | 12 ++++++++++++ utils/list-versions/version_date.tsv | 5 +++++ 5 files changed, 20 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.8.2.3-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index fc93cee5bbc..6ff7ea43374 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.8.1.2684" +ARG VERSION="24.8.2.3" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 3ceaf2a08b4..c87885d3b49 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.8.1.2684" +ARG VERSION="24.8.2.3" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 76db997821c..6ccf74823e2 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.8.1.2684" +ARG VERSION="24.8.2.3" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.8.2.3-lts.md b/docs/changelogs/v24.8.2.3-lts.md new file mode 100644 index 00000000000..69dfc9961a2 --- /dev/null +++ b/docs/changelogs/v24.8.2.3-lts.md @@ -0,0 +1,12 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.8.2.3-lts (b54f79ed323) FIXME as compared to v24.8.1.2684-lts (161c62fd295) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68670](https://github.com/ClickHouse/ClickHouse/issues/68670): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8556375d543..199c4f822f4 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,11 +1,15 @@ +v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.3.47-stable 2024-08-22 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 +v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 @@ -14,6 +18,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 v24.3.7.30-lts 2024-08-14 v24.3.6.48-lts 2024-08-02 From fa453c3664b18da7a6945e662b881f80fedadc5b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 22 Aug 2024 18:13:45 +0200 Subject: [PATCH 1314/1722] Disable SqlLogic job --- tests/ci/ci_config.py | 7 ++++--- tests/ci/ci_definitions.py | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 58de25f039f..0885f1d9ec2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -498,9 +498,10 @@ class CI: JobNames.SQLANCER_DEBUG: CommonJobConfigs.SQLLANCER_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], ), - JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE], - ), + # TODO: job does not work at all, uncomment and fix + # JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( + # required_builds=[BuildNames.PACKAGE_RELEASE], + # ), JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 1cdb3f1487e..9d95a19790f 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -204,7 +204,7 @@ class JobNames(metaclass=WithIter): PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)" PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)" - SQL_LOGIC_TEST = "Sqllogic test (release)" + # SQL_LOGIC_TEST = "Sqllogic test (release)" SQLANCER = "SQLancer (release)" SQLANCER_DEBUG = "SQLancer (debug)" From ad60876777fdfe178cae9184a89603376c7d2260 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 16:46:41 +0000 Subject: [PATCH 1315/1722] support query paramters in async inserts --- src/Interpreters/AsynchronousInsertQueue.cpp | 5 +++ src/Interpreters/AsynchronousInsertQueue.h | 1 + .../Executors/StreamingFormatExecutor.cpp | 14 ++++++++ .../Executors/StreamingFormatExecutor.h | 3 ++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 10 ++++++ .../Formats/Impl/ValuesBlockInputFormat.h | 1 + .../03228_async_insert_query_params.reference | 13 +++++++ .../03228_async_insert_query_params.sh | 36 +++++++++++++++++++ 8 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/03228_async_insert_query_params.reference create mode 100755 tests/queries/0_stateless/03228_async_insert_query_params.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..5e012d179c8 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -389,6 +389,10 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr if (data_kind == DataKind::Preprocessed) insert_query.format = "Native"; + /// Query parameters make sense only for format Values. + if (insert_query.format == "Values") + entry->query_parameters = query_context->getQueryParameters(); + InsertQuery key{query, query_context->getUserID(), query_context->getCurrentRoles(), settings, data_kind}; InsertDataPtr data_to_process; std::future insert_future; @@ -999,6 +1003,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( "Expected entry with data kind Parsed. Got: {}", entry->chunk.getDataKind()); auto buffer = std::make_unique(*bytes); + executor.setQueryParameters(entry->query_parameters); size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index da14b43d276..17607ac1879 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -147,6 +147,7 @@ private: const String format; MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; + NameToNameMap query_parameters; Entry( DataChunk && chunk_, diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 12dd685a735..5b35ac6020d 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -32,6 +33,19 @@ MutableColumns StreamingFormatExecutor::getResultColumns() return ret_columns; } +void StreamingFormatExecutor::setQueryParameters(const NameToNameMap & parameters) +{ + if (parameters.empty()) + return; + + /// Query parameters make sense only for format Values. + auto * values_format = typeid_cast(format.get()); + if (!values_format) + return; + + values_format->setQueryParameters(parameters); +} + size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) { format->setReadBuffer(buffer); diff --git a/src/Processors/Executors/StreamingFormatExecutor.h b/src/Processors/Executors/StreamingFormatExecutor.h index 3aa90ab0360..f159178df8c 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.h +++ b/src/Processors/Executors/StreamingFormatExecutor.h @@ -39,6 +39,9 @@ public: /// Releases currently accumulated columns. MutableColumns getResultColumns(); + /// Sets query parameters for input format if applicable. + void setQueryParameters(const NameToNameMap & parameters); + private: const Block header; const InputFormatPtr format; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 9839f64b947..e34000df5f4 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -663,6 +663,16 @@ void ValuesBlockInputFormat::resetReadBuffer() IInputFormat::resetReadBuffer(); } +void ValuesBlockInputFormat::setQueryParameters(const NameToNameMap & parameters) +{ + if (parameters.empty()) + return; + + auto context_copy = Context::createCopy(context); + context_copy->setQueryParameters(parameters); + context = std::move(context_copy); +} + ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : IRowSchemaReader(buf, format_settings_), buf(in_) { diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 0abafc896ff..228e8d0d572 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -38,6 +38,7 @@ public: /// TODO: remove context somehow. void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); } + void setQueryParameters(const NameToNameMap & parameters); const BlockMissingValues & getMissingValues() const override { return block_missing_values; } diff --git a/tests/queries/0_stateless/03228_async_insert_query_params.reference b/tests/queries/0_stateless/03228_async_insert_query_params.reference new file mode 100644 index 00000000000..9de99658a53 --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params.reference @@ -0,0 +1,13 @@ +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 diff --git a/tests/queries/0_stateless/03228_async_insert_query_params.sh b/tests/queries/0_stateless/03228_async_insert_query_params.sh new file mode 100755 index 00000000000..6b130389220 --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + DROP TABLE IF EXISTS t_async_insert_params; + CREATE TABLE t_async_insert_params (id UInt64) ENGINE = Memory; +" +cmd_params="--async_insert 1 --async_insert_busy_timeout_max_ms 300000 --async_insert_busy_timeout_min_ms 300000 --wait_for_async_insert 0 --async_insert_use_adaptive_busy_timeout 0" + +$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p1 = 11; INSERT INTO t_async_insert_params VALUES ({p1:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 12; INSERT INTO t_async_insert_params VALUES ({p2:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 1000; INSERT INTO t_async_insert_params VALUES (13);" +$CLICKHOUSE_CLIENT $cmd_params -q 'SET param_p2 = 1000; INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 14};' + +$CLICKHOUSE_CLIENT $cmd_params --param_p1 15 -q "INSERT INTO t_async_insert_params VALUES ({p1:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params --param_p2 16 -q "INSERT INTO t_async_insert_params VALUES ({p2:UInt64});" +$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q "INSERT INTO t_async_insert_params VALUES (17);" +$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 18};' + +url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0" + +${CLICKHOUSE_CURL} -sS "$url¶m_p1=19" -d "INSERT INTO t_async_insert_params VALUES ({p1:UInt64})" +${CLICKHOUSE_CURL} -sS "$url¶m_p2=20" -d "INSERT INTO t_async_insert_params VALUES ({p2:UInt64})" +${CLICKHOUSE_CURL} -sS "$url¶m_p3=21" -d "INSERT INTO t_async_insert_params VALUES ({p3:UInt64})" +${CLICKHOUSE_CURL} -sS "$url¶m_p2=1000" -d "INSERT INTO t_async_insert_params VALUES (22)" +${CLICKHOUSE_CURL} -sS "$url¶m_p2=1000" -d 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 23}' + +$CLICKHOUSE_CLIENT -q " + SYSTEM FLUSH ASYNC INSERT QUEUE; + SELECT id FROM t_async_insert_params ORDER BY id; + DROP TABLE IF EXISTS t_async_insert_params; +" From 06c46ee75bcb94fe02ac68df6a4a044145804d76 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 18:56:50 +0200 Subject: [PATCH 1316/1722] add one more test --- .../0_stateless/03203_hive_style_partitioning.reference | 1 + tests/queries/0_stateless/03203_hive_style_partitioning.sh | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index a9d856babce..0fbc1fb556e 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -37,6 +37,7 @@ Array(Int64) LowCardinality(Float64) 2070 b 1 +1 TESTING THE URL PARTITIONING last Elizabeth Frank Elizabeth diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 6734c5f14ad..8ab18f5edfe 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -29,6 +29,12 @@ SELECT identifier FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.c SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMIT 1; """ +$CLICKHOUSE_LOCAL -n -q """ +set use_hive_partitioning = 1; + +SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; +""" 2>&1 | grep -c "INCORRECT_DATA" + $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 0; From b4a0a9b00cafc86399c62c244b8ada6e0f8d2db4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 22 Aug 2024 16:58:19 +0000 Subject: [PATCH 1317/1722] rework test_async_insert_adaptive_busy_timeout --- .../test.py | 55 ++++++++++--------- .../test_with_table_engine_grant.py | 1 + 2 files changed, 31 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 0ea076b1468..5599786026f 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -104,7 +104,7 @@ def test_with_merge_tree(): _insert_queries_sequentially( table_name, _query_settings, - iterations=100, + iterations=10, max_values_size=1000, array_size_range=[10, 50], ) @@ -125,7 +125,7 @@ def test_with_merge_tree_multithread(): table_name, _query_settings, thread_num=15, - tasks=1000, + tasks=100, max_values_size=1000, array_size_range=[10, 15], ) @@ -152,7 +152,7 @@ def test_with_replicated_merge_tree(): _insert_queries_sequentially( table_name, settings, - iterations=100, + iterations=10, max_values_size=1000, array_size_range=[10, 50], ) @@ -180,7 +180,7 @@ def test_with_replicated_merge_tree_multithread(): table_name, _query_settings, thread_num=15, - tasks=1000, + tasks=100, max_values_size=1000, array_size_range=[10, 15], ) @@ -200,13 +200,13 @@ def test_compare_sequential_inserts_durations_for_adaptive_and_fixed_async_timeo fixed_tm_settings = copy.copy(_query_settings) fixed_tm_settings["async_insert_use_adaptive_busy_timeout"] = 0 - fixed_tm_settings["async_insert_busy_timeout_ms"] = 200 + fixed_tm_settings["async_insert_busy_timeout_ms"] = 100 fixed_tm_run_duration = timeit.timeit( lambda: _insert_queries_sequentially( fixed_tm_table_name, fixed_tm_settings, - iterations=100, + iterations=50, max_values_size=1000, array_size_range=[10, 50], ), @@ -231,13 +231,13 @@ def test_compare_sequential_inserts_durations_for_adaptive_and_fixed_async_timeo adaptive_tm_settings = copy.copy(_query_settings) adaptive_tm_settings["async_insert_busy_timeout_min_ms"] = 10 - adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 1000 + adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 500 adaptive_tm_run_duration = timeit.timeit( lambda: _insert_queries_sequentially( adaptive_tm_table_name, adaptive_tm_settings, - iterations=100, + iterations=50, max_values_size=1000, array_size_range=[10, 50], ), @@ -268,14 +268,14 @@ def test_compare_parallel_inserts_durations_for_adaptive_and_fixed_async_timeout fixed_tm_settings = copy.copy(_query_settings) fixed_tm_settings["async_insert_use_adaptive_busy_timeout"] = 0 - fixed_tm_settings["async_insert_busy_timeout_ms"] = 200 + fixed_tm_settings["async_insert_busy_timeout_ms"] = 500 fixed_tm_run_duration = timeit.timeit( lambda: _insert_queries_in_parallel( fixed_tm_table_name, fixed_tm_settings, thread_num=15, - tasks=1000, + tasks=150, max_values_size=1000, array_size_range=[10, 50], ), @@ -300,14 +300,14 @@ def test_compare_parallel_inserts_durations_for_adaptive_and_fixed_async_timeout adaptive_tm_settings = copy.copy(_query_settings) adaptive_tm_settings["async_insert_busy_timeout_min_ms"] = 10 - adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 200 + adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 500 adaptive_tm_run_duration = timeit.timeit( lambda: _insert_queries_in_parallel( adaptive_tm_table_name, adaptive_tm_settings, thread_num=15, - tasks=100, + tasks=150, max_values_size=1000, array_size_range=[10, 50], ), @@ -344,29 +344,34 @@ def test_change_queries_frequency(): settings = copy.copy(_query_settings) min_ms = 50 + max_ms = 200 + settings["async_insert_busy_timeout_min_ms"] = min_ms - settings["async_insert_busy_timeout_max_ms"] = 2000 + settings["async_insert_busy_timeout_max_ms"] = max_ms + + _insert_queries_sequentially( + table_name, + settings, + iterations=50, + max_values_size=1000, + array_size_range=[10, 50], + ) + + select_log_query = f"SELECT countIf(timeout_milliseconds - {min_ms} < 25) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" + res = node.query(select_log_query) + assert int(res) >= 5 _insert_queries_in_parallel( table_name, settings, - thread_num=15, + thread_num=20, tasks=2000, max_values_size=1000, array_size_range=[10, 15], ) - _insert_queries_sequentially( - table_name, - settings, - iterations=200, - max_values_size=1000, - array_size_range=[10, 50], - ) - - select_log_query = "SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 50" + select_log_query = f"SELECT countIf({max_ms} - timeout_milliseconds < 100) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) - for line in res.splitlines(): - assert int(line) == min_ms + assert int(res) >= 5 node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) diff --git a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index 5fc8f67b75b..ad3d35d8bbd 100644 --- a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -532,6 +532,7 @@ def test_current_database(): assert "Not enough privileges" in instance.query_and_get_error( "SELECT * FROM table", user="A" ) + instance.query("DROP TABLE default.table SYNC") def test_grant_with_replace_option(): From 2a32207e9ee44d52d6fbca7313d847b4eef1c4fb Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 1318/1722] fix: wrap in conditional preprocessor directives --- src/Functions/FunctionsHashingRipe.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionsHashingRipe.cpp b/src/Functions/FunctionsHashingRipe.cpp index 5b06b8ab924..315296b7690 100644 --- a/src/Functions/FunctionsHashingRipe.cpp +++ b/src/Functions/FunctionsHashingRipe.cpp @@ -7,6 +7,7 @@ /// due to excessive resource consumption. namespace DB { +#if USE_SSL REGISTER_FUNCTION(HashingRipe) { factory.registerFunction(FunctionDocumentation{ @@ -18,4 +19,5 @@ REGISTER_FUNCTION(HashingRipe) )"}}, .categories{"Hash"}}); } +#endif } From ef9fbe3006b3023bf47e3a0109490d166071c2aa Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 01:20:46 +0200 Subject: [PATCH 1319/1722] fix: disable running test in fasttest due to missing OpenSSL --- tests/queries/0_stateless/03222_ripeMD160.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03222_ripeMD160.sql b/tests/queries/0_stateless/03222_ripeMD160.sql index 592f9f830dd..9d418376a20 100644 --- a/tests/queries/0_stateless/03222_ripeMD160.sql +++ b/tests/queries/0_stateless/03222_ripeMD160.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest -- Ouput can be verified using: https://emn178.github.io/online-tools/ripemd-160/ SELECT hex(ripeMD160('The quick brown fox jumps over the lazy dog')); From a2ff8e4384f04b2e57d3de93a1ba63971f94794e Mon Sep 17 00:00:00 2001 From: Dergousov Date: Thu, 22 Aug 2024 20:44:52 +0300 Subject: [PATCH 1320/1722] fix: correct return type inconsistencies in docs --- docs/en/sql-reference/functions/hash-functions.md | 5 ++--- docs/ru/sql-reference/functions/hash-functions.md | 5 +++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 9b7ac8af0e3..cd1c85b5f4c 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -689,9 +689,8 @@ SELECT kostikConsistentHash(16045690984833335023, 2); ``` ## ripeMD160 -Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash of a string and returns the resulting set of bytes as [FixedString](../data-types/fixedstring.md). - +Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash value. **Syntax** @@ -705,7 +704,7 @@ ripeMD160('input') **Returned value** -- A [UInt256](../data-types/int-uint.md) hash value +- A [UInt256]((../data-types/int-uint.md)) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. **Example** Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 66d77e66972..b7adcfc1829 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -125,7 +125,8 @@ SELECT hex(sipHash128('foo', '\x01', 3)); ``` ## ripeMD160 -Генерирует [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) хеш Ñтроки и возвращает полученный набор байт в виде [FixedString](../data-types/fixedstring.md). + +Генерирует [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) хеш Ñтроки. **СинтакÑиÑ** @@ -139,7 +140,7 @@ ripeMD160('input') **Возвращаемое значение** -- [UInt256](../data-types/int-uint.md) хеш-значение +- [UInt256](../data-types/int-uint.md), где 160-битный хеш RIPEMD-160 хранитÑÑ Ð² первых 20 байтах. ОÑтавшиеÑÑ 12 байт заполнÑÑŽÑ‚ÑÑ Ð½ÑƒÐ»Ñми. **Пример** ИÑпользуйте функцию [hex](../functions/encoding-functions.md#hex) Ð´Ð»Ñ Ð¿Ñ€ÐµÐ´ÑÑ‚Ð°Ð²Ð»ÐµÐ½Ð¸Ñ Ñ€ÐµÐ·ÑƒÐ»ÑŒÑ‚Ð°Ñ‚Ð° в виде Ñтроки Ñ ÑˆÐµÑтнадцатеричной кодировкой From de2f1adf9167db66469efd8e8b5d2f828f993ec1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2024 18:02:03 +0000 Subject: [PATCH 1321/1722] Fix mergine of aggregated data for grouping sets. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 - src/Planner/Planner.cpp | 2 - .../QueryPlan/MergingAggregatedStep.cpp | 2 +- .../Transforms/MergingAggregatedTransform.cpp | 118 ++++++++++++++++-- .../Transforms/MergingAggregatedTransform.h | 10 +- .../02165_replicated_grouping_sets.reference | 54 ++++++++ .../02165_replicated_grouping_sets.sql | 5 + 7 files changed, 179 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0c79f4310ce..9e5fffac6e4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2010,8 +2010,6 @@ static void executeMergeAggregatedImpl( SortDescription group_by_sort_description) { auto keys = aggregation_keys.getNames(); - if (has_grouping_sets) - keys.insert(keys.begin(), "__grouping_set"); /** There are two modes of distributed aggregation. * diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d3d20c6fba0..c0efed8550f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -504,8 +504,6 @@ void addMergingAggregatedStep(QueryPlan & query_plan, */ auto keys = aggregation_analysis_result.aggregation_keys; - if (!aggregation_analysis_result.grouping_sets_parameters_list.empty()) - keys.insert(keys.begin(), "__grouping_set"); Aggregator::Params params(keys, aggregation_analysis_result.aggregate_descriptions, diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index a5062ac8216..50bd1a882ef 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -48,7 +48,7 @@ MergingAggregatedStep::MergingAggregatedStep( bool memory_bound_merging_of_aggregation_results_enabled_) : ITransformingStep( input_stream_, - params_.getHeader(input_stream_.header, final_), + MergingAggregatedTransform::appendGroupingIfNeeded(input_stream_.header, params_.getHeader(input_stream_.header, final_)), getTraits(should_produce_results_in_order_of_bucket_number_)) , params(std::move(params_)) , final(final_) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 446e60a0b81..114a32b3d83 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -10,13 +11,106 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header) +{ + if (in_header.has("__grouping_set")) + out_header.insert(0, in_header.getByName("__grouping_set")); + + return out_header; +} + MergingAggregatedTransform::MergingAggregatedTransform( Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_) - : IAccumulatingTransform(std::move(header_), params_->getHeader()) - , params(std::move(params_)), max_threads(max_threads_) + : IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params_->getHeader())) + , params(std::move(params_)), max_threads(max_threads_), has_grouping_sets(header_.has("__grouping_set")) { } +void MergingAggregatedTransform::addBlock(Block block) +{ + if (!has_grouping_sets) + { + auto & bucket_to_blocks = grouping_sets[0]; + bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + return; + } + + auto grouping_position = block.getPositionByName("__grouping_set"); + auto grouping_column = block.getByPosition(grouping_position).column; + block.erase(grouping_position); + + const auto * grouping_column_typed = typeid_cast(grouping_column.get()); + if (!grouping_column_typed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName()); + + const auto & grouping_data = grouping_column_typed->getData(); + std::map enumerated_groups; + IColumn::Selector selector; + + size_t num_rows = grouping_data.size(); + UInt64 last_group = grouping_data[0]; + for (size_t row = 1; row < num_rows; ++row) + { + auto group = grouping_data[row]; + if (last_group == group) + continue; + + if (enumerated_groups.empty()) + { + selector.reserve(num_rows); + enumerated_groups.emplace(last_group, enumerated_groups.size()); + } + + selector.resize_fill(row, enumerated_groups[last_group]); + enumerated_groups.emplace(last_group, enumerated_groups.size()); + } + + if (enumerated_groups.empty()) + { + auto & bucket_to_blocks = grouping_sets[last_group]; + bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + return; + } + + selector.resize_fill(num_rows, enumerated_groups[last_group]); + + const size_t num_groups = enumerated_groups.size(); + Blocks splitted_blocks(num_groups); + + for (size_t group_id = 0; group_id < num_groups; ++group_id) + splitted_blocks[group_id] = block.cloneEmpty(); + + size_t columns_in_block = block.columns(); + for (size_t col_idx_in_block = 0; col_idx_in_block < columns_in_block; ++col_idx_in_block) + { + MutableColumns splitted_columns = block.getByPosition(col_idx_in_block).column->scatter(num_groups, selector); + for (size_t group_id = 0; group_id < num_groups; ++group_id) + splitted_blocks[group_id].getByPosition(col_idx_in_block).column = std::move(splitted_columns[group_id]); + } + + for (auto [group, group_id] : enumerated_groups) + { + auto & bucket_to_blocks = grouping_sets[group]; + auto & splitted_block = splitted_blocks[group_id]; + splitted_block.info = block.info; + bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block)); + } +} + +void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) +{ + auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); + for (auto & block : block_list) + { + auto num_rows = block.rows(); + ColumnWithTypeAndName col; + col.type = std::make_shared(); + col.name = "__grouping_set"; + col.column = ColumnUInt64::create(num_rows, group); + block.insert(grouping_position, std::move(col)); + } +} + void MergingAggregatedTransform::consume(Chunk chunk) { if (!consume_started) @@ -46,7 +140,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) block.info.is_overflows = agg_info->is_overflows; block.info.bucket_num = agg_info->bucket_num; - bucket_to_blocks[agg_info->bucket_num].emplace_back(std::move(block)); + addBlock(std::move(block)); } else if (chunk.getChunkInfos().get()) { @@ -54,7 +148,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) block.info.is_overflows = false; block.info.bucket_num = -1; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + addBlock(std::move(block)); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform."); @@ -70,9 +164,19 @@ Chunk MergingAggregatedTransform::generate() /// Exception safety. Make iterator valid in case any method below throws. next_block = blocks.begin(); - /// TODO: this operation can be made async. Add async for IAccumulatingTransform. - params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled); - blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); + for (auto & [group, group_blocks] : grouping_sets) + { + /// TODO: this operation can be made async. Add async for IAccumulatingTransform. + AggregatedDataVariants data_variants; + params->aggregator.mergeBlocks(std::move(group_blocks), data_variants, max_threads, is_cancelled); + auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); + + if (has_grouping_sets) + appendGroupingColumn(group, merged_blocks); + + blocks.splice(blocks.end(), std::move(merged_blocks)); + } + next_block = blocks.begin(); } diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index ade76b2f304..1d801f7a94d 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -15,6 +15,8 @@ public: MergingAggregatedTransform(Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_); String getName() const override { return "MergingAggregatedTransform"; } + static Block appendGroupingIfNeeded(const Block & in_header, Block out_header); + protected: void consume(Chunk chunk) override; Chunk generate() override; @@ -24,8 +26,9 @@ private: LoggerPtr log = getLogger("MergingAggregatedTransform"); size_t max_threads; - AggregatedDataVariants data_variants; - Aggregator::BucketToBlocks bucket_to_blocks; + using GroupingSets = std::unordered_map; + GroupingSets grouping_sets; + const bool has_grouping_sets; UInt64 total_input_rows = 0; UInt64 total_input_blocks = 0; @@ -35,6 +38,9 @@ private: bool consume_started = false; bool generate_started = false; + + void addBlock(Block block); + void appendGroupingColumn(UInt64 group, BlocksList & block_list); }; } diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference index 659cd98368d..4589dc7d7a5 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference @@ -11,3 +11,57 @@ 0 6 4 1 10 4 2 14 4 +['.'] +['.','.'] +['.','.','.'] +['.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +['.'] +['.'] +['.','.'] +['.','.'] +['.','.','.'] +['.','.','.'] +['.','.','.','.'] +['.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +['.','.','.','.','.','.','.','.','.'] +1 +2 +3 +4 +5 +6 +7 +8 +9 +1 +1 +2 +2 +3 +3 +4 +4 +5 +5 +6 +6 +7 +7 +8 +8 +9 +9 diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql index d92d92c3e72..333dab79575 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql @@ -43,3 +43,8 @@ GROUP BY ORDER BY sum_value ASC, count_value ASC; + +SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; From e6b18f2b300c7aec883d60619f432f7e4b4cca3a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 22 Aug 2024 19:05:07 +0000 Subject: [PATCH 1322/1722] black --- .../test_postgresql_replica_database_engine_2/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 428e9ecdb96..0a364d7802b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -61,6 +61,7 @@ pg_manager = PostgresManager() pg_manager2 = PostgresManager() pg_manager_instance2 = PostgresManager() + @pytest.fixture(scope="module") def started_cluster(): try: @@ -918,7 +919,7 @@ def test_failed_load_from_snapshot(started_cluster): def test_symbols_in_publication_name(started_cluster): id = uuid.uuid4() - db = f'test_{id}' + db = f"test_{id}" table = f"test_symbols_in_publication_name" pg_manager3 = PostgresManager() From 0e136ded28dc1191dd344500d031f43d7a5750e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Aug 2024 19:06:04 +0000 Subject: [PATCH 1323/1722] Fixing header. --- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 50bd1a882ef..8332ad73df6 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -154,7 +154,9 @@ void MergingAggregatedStep::describeActions(JSONBuilder::JSONMap & map) const void MergingAggregatedStep::updateOutputStream() { - output_stream = createOutputStream(input_streams.front(), params.getHeader(input_streams.front().header, final), getDataStreamTraits()); + const auto & in_header = input_streams.front().header; + output_stream = createOutputStream(input_streams.front(), + MergingAggregatedTransform::appendGroupingIfNeeded(in_header, params.getHeader(in_header, final)), getDataStreamTraits()); if (is_order_overwritten) /// overwrite order again applyOrder(group_by_sort_description, overwritten_sort_scope); } From f89193fa416cc333f549d72bb8ba453907edc951 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Aug 2024 19:12:19 +0000 Subject: [PATCH 1324/1722] Update version_date.tsv and changelogs after v24.5.5.41-stable --- utils/list-versions/version_date.tsv | 2 -- 1 file changed, 2 deletions(-) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 199c4f822f4..0e25f8d3b62 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,11 +1,9 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 -v24.7.3.47-stable 2024-08-22 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 -v24.6.3.38-stable 2024-08-22 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.5.78-stable 2024-08-05 From 4200b3d5cbbfe065073c40f1e122c44189f3554f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 22 Aug 2024 14:02:25 +0200 Subject: [PATCH 1325/1722] CI: Stress test fix --- tests/clickhouse-test | 2 +- tests/docker_scripts/stress_runner.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4f9380d6f20..ad6173065fe 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3567,7 +3567,7 @@ if __name__ == "__main__": f"Cannot access the specified directory with queries ({args.queries})", file=sys.stderr, ) - sys.exit(1) + assert False, "No --queries provided" CAPTURE_CLIENT_STACKTRACE = args.capture_client_stacktrace diff --git a/tests/docker_scripts/stress_runner.sh b/tests/docker_scripts/stress_runner.sh index 7666398e10b..039c60c8e4e 100755 --- a/tests/docker_scripts/stress_runner.sh +++ b/tests/docker_scripts/stress_runner.sh @@ -10,8 +10,7 @@ dmesg --clear # shellcheck disable=SC1091 source /setup_export_logs.sh -ln -s /repo/tests/clickhouse-test/ci/stress.py /usr/bin/stress -ln -s /repo/tests/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib @@ -266,6 +265,7 @@ fi start_server +cd /repo/tests/ || exit 1 # clickhouse-test can find queries dir from there python3 /repo/tests/ci/stress.py --hung-check --drop-databases --output-folder /test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ && echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \ || echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv From 69f6ea5083f1686becce4ca9fcf47d1404f2d3ed Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 22 Aug 2024 22:07:02 +0200 Subject: [PATCH 1326/1722] Update docs/en/sql-reference/functions/hash-functions.md --- docs/en/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index cd1c85b5f4c..55126640e34 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -704,7 +704,7 @@ ripeMD160('input') **Returned value** -- A [UInt256]((../data-types/int-uint.md)) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. +- A [UInt256](../data-types/int-uint.md) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. **Example** Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. From dc862b1411884a462bba8dcf86a474ccbe57e380 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:40:18 +0200 Subject: [PATCH 1327/1722] fix test --- tests/queries/0_stateless/03203_hive_style_partitioning.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.sh b/tests/queries/0_stateless/03203_hive_style_partitioning.sh index 8ab18f5edfe..60e8a6e9faa 100755 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.sh +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.sh @@ -32,7 +32,7 @@ SELECT a FROM file('$CURDIR/data_hive/partitioning/a=b/a=b/sample.parquet') LIMI $CLICKHOUSE_LOCAL -n -q """ set use_hive_partitioning = 1; -SELECT *, _column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; +SELECT *, column0 FROM file('$CURDIR/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet') LIMIT 10; """ 2>&1 | grep -c "INCORRECT_DATA" $CLICKHOUSE_LOCAL -n -q """ From 1aaf9a08c4e0433dcf2afc97770b5a595bd108ee Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 23:25:38 +0000 Subject: [PATCH 1328/1722] fix async inserts with alter --- src/Interpreters/AsynchronousInsertQueue.cpp | 62 +++++++++++--------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..52f8a8dd42a 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -33,6 +33,8 @@ #include #include #include +#include +#include namespace CurrentMetrics { @@ -308,6 +310,7 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const /* no_squash */ false, /* no_destination */ false, /* async_insert */ false); + auto table = interpreter.getTable(insert_query); auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context); @@ -318,6 +321,10 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const /// InterpreterInsertQuery::getTable() -> ITableFunction::execute(). if (insert_query.table_id) query_context->checkAccess(AccessType::INSERT, insert_query.table_id, sample_block.getNames()); + + insert_query.columns = std::make_shared(); + for (const auto & column : sample_block) + insert_query.columns->children.push_back(std::make_shared(column.name)); } AsynchronousInsertQueue::PushResult @@ -872,36 +879,35 @@ try } }; - Chunk chunk; - auto header = pipeline.getHeader(); - - if (key.data_kind == DataKind::Parsed) - chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); - else - chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); - - ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); - - auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) - { - LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); - queue_shard_flush_time_history.updateWithCurrentTime(); - - bool pulling_pipeline = false; - logQueryFinish( - query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); - }; - - - if (chunk.getNumRows() == 0) - { - finish_entries(); - log_and_add_finish_to_query_log(0, 0); - return; - } - try { + Chunk chunk; + auto header = pipeline.getHeader(); + + if (key.data_kind == DataKind::Parsed) + chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); + else + chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); + + ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); + + auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) + { + LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); + queue_shard_flush_time_history.updateWithCurrentTime(); + + bool pulling_pipeline = false; + logQueryFinish( + query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); + }; + + if (chunk.getNumRows() == 0) + { + finish_entries(); + log_and_add_finish_to_query_log(0, 0); + return; + } + size_t num_rows = chunk.getNumRows(); size_t num_bytes = chunk.bytes(); From 4c790999eb6ad74e3a8f99c072dcc12c956a63d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 23 Aug 2024 02:18:26 +0200 Subject: [PATCH 1329/1722] CI: Force package_debug build on release branches --- .github/workflows/release_branches.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 82826794ea3..ec119b6ff95 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -130,6 +130,7 @@ jobs: with: build_name: package_debug data: ${{ needs.RunConfig.outputs.data }} + force: true BuilderBinDarwin: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} From f5739dfe06db8610818fafb5c3a2c33f59fd0a8d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 23 Aug 2024 02:51:27 +0200 Subject: [PATCH 1330/1722] CI: Make job rerun possible if triggered manually --- tests/ci/ci.py | 7 +++++-- tests/ci/ci_utils.py | 5 +++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index a9ae078b449..d201b6602f5 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -333,7 +333,10 @@ def _pre_action(s3, job_name, batch, indata, pr_info): CI.JobNames.BUILD_CHECK, ): # we might want to rerun build report job rerun_helper = RerunHelper(commit, _get_ext_check_name(job_name)) - if rerun_helper.is_already_finished_by_status(): + if ( + rerun_helper.is_already_finished_by_status() + and not Utils.is_job_triggered_manually() + ): print("WARNING: Rerunning job with GH status ") status = rerun_helper.get_finished_status() assert status @@ -344,7 +347,7 @@ def _pre_action(s3, job_name, batch, indata, pr_info): skip_status = status.state # ci cache check - if not to_be_skipped and not no_cache: + if not to_be_skipped and not no_cache and not Utils.is_job_triggered_manually(): ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() job_config = CI.get_job_config(job_name) if ci_cache.is_successful( diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index a4c0977f47c..e8d9e7dc254 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -18,6 +18,7 @@ class Envs: ) S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") GITHUB_WORKFLOW = os.getenv("GITHUB_WORKFLOW", "") + GITHUB_ACTOR = os.getenv("GITHUB_ACTOR", "") class WithIter(type): @@ -282,3 +283,7 @@ class Utils: ): res = res.replace(*r) return res + + @staticmethod + def is_job_triggered_manually(): + return "robot" not in Envs.GITHUB_ACTOR From 073ef13e36e46cf6f34c0f1cee2dd5212753fd74 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 01:02:46 +0000 Subject: [PATCH 1331/1722] fix --- .../0_stateless/02477_projection_materialize_and_zero_copy.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 1845919890c..2750bdf5c4b 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -1,3 +1,5 @@ +-- Tags: long, no-parallel + DROP TABLE IF EXISTS t; create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 From 60e4bcbbf0b1991b42bcab4b83e55be344e8a659 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Thu, 22 Aug 2024 20:45:28 -0700 Subject: [PATCH 1332/1722] Update README.md Update Raleigh meetup link --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 83a5c05c667..546f08afd3d 100644 --- a/README.md +++ b/README.md @@ -44,7 +44,7 @@ The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 -* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 * [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 * [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 From e5380806653f8d391c6e88664b0096c3c51240f5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 07:09:03 +0000 Subject: [PATCH 1333/1722] Update version_date.tsv and changelogs after v24.5.6.45-stable --- docs/changelogs/v24.5.6.45-stable.md | 33 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 +- 2 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 docs/changelogs/v24.5.6.45-stable.md diff --git a/docs/changelogs/v24.5.6.45-stable.md b/docs/changelogs/v24.5.6.45-stable.md new file mode 100644 index 00000000000..b329ebab27b --- /dev/null +++ b/docs/changelogs/v24.5.6.45-stable.md @@ -0,0 +1,33 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.6.45-stable (bdca8604c29) FIXME as compared to v24.5.5.78-stable (0138248cb62) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67902](https://github.com/ClickHouse/ClickHouse/issues/67902): Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68252](https://github.com/ClickHouse/ClickHouse/issues/68252): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#68064](https://github.com/ClickHouse/ClickHouse/issues/68064): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68158](https://github.com/ClickHouse/ClickHouse/issues/68158): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68115](https://github.com/ClickHouse/ClickHouse/issues/68115): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67886](https://github.com/ClickHouse/ClickHouse/issues/67886): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68272](https://github.com/ClickHouse/ClickHouse/issues/68272): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67807](https://github.com/ClickHouse/ClickHouse/issues/67807): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67836](https://github.com/ClickHouse/ClickHouse/issues/67836): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67991](https://github.com/ClickHouse/ClickHouse/issues/67991): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68207](https://github.com/ClickHouse/ClickHouse/issues/68207): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68091](https://github.com/ClickHouse/ClickHouse/issues/68091): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68122](https://github.com/ClickHouse/ClickHouse/issues/68122): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68171](https://github.com/ClickHouse/ClickHouse/issues/68171): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68337](https://github.com/ClickHouse/ClickHouse/issues/68337): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68667](https://github.com/ClickHouse/ClickHouse/issues/68667): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67862](https://github.com/ClickHouse/ClickHouse/pull/67862) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68077](https://github.com/ClickHouse/ClickHouse/issues/68077): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Backported in [#68756](https://github.com/ClickHouse/ClickHouse/issues/68756): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e25f8d3b62..57a59d7ac49 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,8 +6,8 @@ v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 -v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From 1dc7d37ff7ed295be3ef037bf9b1d04b377892b0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 12:30:25 +0000 Subject: [PATCH 1334/1722] Bump usearch to v2.14.0 --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index e21a5778a0d..7a8967cb442 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4 +Subproject commit 7a8967cb442b08ca20c3dd781414378e65957d37 From c40c8b7adb28403d8020282595ddaa72953acbf6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 14:11:33 +0000 Subject: [PATCH 1335/1722] Enable bf16 + f64 quantization, make bf16 the default --- .../mergetree-family/annindexes.md | 2 +- .../MergeTreeIndexVectorSimilarity.cpp | 4 ++- .../02354_vector_search_queries.reference | 36 +++++++++++++++++++ .../02354_vector_search_queries.sql | 32 +++++++++++++++++ 4 files changed, 72 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 1057ccb5fee..4cf558fc872 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -54,7 +54,7 @@ Parameters: - `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a line between two points in Euclidean space), or `cosineDistance` (the [cosine distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors). -- `quantization`: either `f32`, `f16`, or `i8` for storing the vector with reduced precision (optional, default: `f32`) +- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing the vector with reduced precision (optional, default: `bf16`) - `m`: the number of neighbors per graph node (optional, default: 16) - `ef_construction`: (optional, default: 128) - `ef_search`: (optional, default: 64) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 4c0da28c3c4..ae183d74782 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -52,8 +52,10 @@ const std::unordered_map distanceFunctionT /// Maps from user-facing name to internal name const std::unordered_map quantizationToScalarKind = { + {"f64", unum::usearch::scalar_kind_t::f64_k}, {"f32", unum::usearch::scalar_kind_t::f32_k}, {"f16", unum::usearch::scalar_kind_t::f16_k}, + {"bf16", unum::usearch::scalar_kind_t::bf16_k}, {"i8", unum::usearch::scalar_kind_t::i8_k}}; /// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported across all distance functions. @@ -461,7 +463,7 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) { /// Default parameters: unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet()); - unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k; + unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::bf16_k; UsearchHnswParams usearch_hnsw_params; /// Optional parameters: diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index cb3a8c801b1..faff306ef60 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -56,6 +56,24 @@ Expression (Projection) 1 [2,3.2] 2.3323807824711897 2 [4.2,3.4] 4.427188573446585 0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_f64) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -92,6 +110,24 @@ Expression (Projection) 1 [2,3.2] 2.3323807824711897 2 [4.2,3.4] 4.427188573446585 0 [4.6,2.3] 4.609772130377966 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_bf16) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 +1 [2,3.2] 2.3323807824711897 +2 [4.2,3.4] 4.427188573446585 +0 [4.6,2.3] 4.609772130377966 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index fbf8427d8fe..17939992165 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -75,13 +75,30 @@ SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann ind DROP TABLE tab; SELECT '-- Non-default quantization'; +CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +INSERT INTO tab_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); INSERT INTO tab_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); INSERT INTO tab_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); INSERT INTO tab_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f64 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_f64 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab_f32 @@ -108,6 +125,19 @@ FROM tab_f16 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_bf16 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab_bf16 +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; + WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) FROM tab_i8 @@ -121,8 +151,10 @@ FROM tab_i8 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; +DROP TABLE tab_f64; DROP TABLE tab_f32; DROP TABLE tab_f16; +DROP TABLE tab_bf16; DROP TABLE tab_i8; SELECT '-- Index on Array(Float64) column'; From e1a7bd9163bebf0aeab12d8dd46c729f73b068be Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 07:37:32 +0000 Subject: [PATCH 1336/1722] Update version_date.tsv and changelogs after v24.6.4.42-stable --- docs/changelogs/v24.6.4.42-stable.md | 33 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 ++- 2 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 docs/changelogs/v24.6.4.42-stable.md diff --git a/docs/changelogs/v24.6.4.42-stable.md b/docs/changelogs/v24.6.4.42-stable.md new file mode 100644 index 00000000000..29b6ba095af --- /dev/null +++ b/docs/changelogs/v24.6.4.42-stable.md @@ -0,0 +1,33 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.4.42-stable (c534bb4b4dd) FIXME as compared to v24.6.3.95-stable (8325c920d11) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68066](https://github.com/ClickHouse/ClickHouse/issues/68066): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68566](https://github.com/ClickHouse/ClickHouse/issues/68566): Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68159](https://github.com/ClickHouse/ClickHouse/issues/68159): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68116](https://github.com/ClickHouse/ClickHouse/issues/68116): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67887](https://github.com/ClickHouse/ClickHouse/issues/67887): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68611](https://github.com/ClickHouse/ClickHouse/issues/68611): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68275](https://github.com/ClickHouse/ClickHouse/issues/68275): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67993](https://github.com/ClickHouse/ClickHouse/issues/67993): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68208](https://github.com/ClickHouse/ClickHouse/issues/68208): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68093](https://github.com/ClickHouse/ClickHouse/issues/68093): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68124](https://github.com/ClickHouse/ClickHouse/issues/68124): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68221](https://github.com/ClickHouse/ClickHouse/issues/68221): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68173](https://github.com/ClickHouse/ClickHouse/issues/68173): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68339](https://github.com/ClickHouse/ClickHouse/issues/68339): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68396](https://github.com/ClickHouse/ClickHouse/issues/68396): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68668](https://github.com/ClickHouse/ClickHouse/issues/68668): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67909](https://github.com/ClickHouse/ClickHouse/pull/67909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68079](https://github.com/ClickHouse/ClickHouse/issues/68079): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Backported in [#68758](https://github.com/ClickHouse/ClickHouse/issues/68758): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e25f8d3b62..8ce510f110d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,11 +3,12 @@ v24.8.1.2684-lts 2024-08-21 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 -v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From eec720dab60ea63b033919bbc4c1f6837920a42d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 08:05:27 +0000 Subject: [PATCH 1337/1722] Update version_date.tsv and changelogs after v24.7.4.51-stable --- docs/changelogs/v24.7.4.51-stable.md | 36 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 38 insertions(+) create mode 100644 docs/changelogs/v24.7.4.51-stable.md diff --git a/docs/changelogs/v24.7.4.51-stable.md b/docs/changelogs/v24.7.4.51-stable.md new file mode 100644 index 00000000000..a7cf9790383 --- /dev/null +++ b/docs/changelogs/v24.7.4.51-stable.md @@ -0,0 +1,36 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.4.51-stable (70fe2f6fa52) FIXME as compared to v24.7.3.42-stable (63730bc4293) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68232](https://github.com/ClickHouse/ClickHouse/issues/68232): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#68068](https://github.com/ClickHouse/ClickHouse/issues/68068): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68613](https://github.com/ClickHouse/ClickHouse/issues/68613): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68278](https://github.com/ClickHouse/ClickHouse/issues/68278): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68040](https://github.com/ClickHouse/ClickHouse/issues/68040): Fix creation of view with recursive CTE. [#67587](https://github.com/ClickHouse/ClickHouse/pull/67587) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#68038](https://github.com/ClickHouse/ClickHouse/issues/68038): Fix crash on `percent_rank`. `percent_rank`'s default frame type is changed to `range unbounded preceding and unbounded following`. `IWindowFunction`'s default window frame is considered and now window functions without window frame definition in sql can be put into different `WindowTransfomer`s properly. [#67661](https://github.com/ClickHouse/ClickHouse/pull/67661) ([lgbo](https://github.com/lgbo-ustc)). +* Backported in [#68224](https://github.com/ClickHouse/ClickHouse/issues/68224): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68095](https://github.com/ClickHouse/ClickHouse/issues/68095): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68126](https://github.com/ClickHouse/ClickHouse/issues/68126): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68223](https://github.com/ClickHouse/ClickHouse/issues/68223): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68175](https://github.com/ClickHouse/ClickHouse/issues/68175): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68341](https://github.com/ClickHouse/ClickHouse/issues/68341): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68398](https://github.com/ClickHouse/ClickHouse/issues/68398): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68669](https://github.com/ClickHouse/ClickHouse/issues/68669): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67803](https://github.com/ClickHouse/ClickHouse/issues/67803): Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. [#67359](https://github.com/ClickHouse/ClickHouse/pull/67359) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68081](https://github.com/ClickHouse/ClickHouse/issues/68081): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Update version after release. [#68044](https://github.com/ClickHouse/ClickHouse/pull/68044) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68269](https://github.com/ClickHouse/ClickHouse/issues/68269): [Green CI] Fix test 01903_correct_block_size_prediction_with_default. [#68203](https://github.com/ClickHouse/ClickHouse/pull/68203) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#68432](https://github.com/ClickHouse/ClickHouse/issues/68432): tests: make 01600_parts_states_metrics_long better. [#68265](https://github.com/ClickHouse/ClickHouse/pull/68265) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68538](https://github.com/ClickHouse/ClickHouse/issues/68538): CI: Native build for package_aarch64. [#68457](https://github.com/ClickHouse/ClickHouse/pull/68457) ([Max K.](https://github.com/maxknv)). +* Backported in [#68555](https://github.com/ClickHouse/ClickHouse/issues/68555): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)). +* Backported in [#68760](https://github.com/ClickHouse/ClickHouse/issues/68760): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 57a59d7ac49..d9674ed2366 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,8 +1,10 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.4.51-stable 2024-08-23 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 From 6ba686d2510a2d95ab4332560163d0b4600533a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 23 Aug 2024 09:20:40 +0000 Subject: [PATCH 1338/1722] Split test case and reduce number of random runs to reduce time necessary to run the test --- .../01395_limit_more_cases.reference | 1 - .../0_stateless/01395_limit_more_cases.sh | 24 ++++--------------- .../01395_limit_more_cases_random.reference | 1 + .../01395_limit_more_cases_random.sh | 22 +++++++++++++++++ 4 files changed, 28 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/01395_limit_more_cases_random.reference create mode 100755 tests/queries/0_stateless/01395_limit_more_cases_random.sh diff --git a/tests/queries/0_stateless/01395_limit_more_cases.reference b/tests/queries/0_stateless/01395_limit_more_cases.reference index c9d0dd73ab8..d68b987ea19 100644 --- a/tests/queries/0_stateless/01395_limit_more_cases.reference +++ b/tests/queries/0_stateless/01395_limit_more_cases.reference @@ -254,4 +254,3 @@ 15 13 0 0 0 0 0 0 15 14 0 0 0 0 0 0 15 15 0 0 0 0 0 0 -0 0 0 diff --git a/tests/queries/0_stateless/01395_limit_more_cases.sh b/tests/queries/0_stateless/01395_limit_more_cases.sh index 177147d2142..9709bd74f26 100755 --- a/tests/queries/0_stateless/01395_limit_more_cases.sh +++ b/tests/queries/0_stateless/01395_limit_more_cases.sh @@ -9,8 +9,11 @@ SIZE=13 for OFFSET in {0..15}; do for LIMIT in {0..15}; do echo "SELECT - $OFFSET, $LIMIT, - count() AS c, min(number) AS first, max(number) AS last, + $OFFSET, + $LIMIT, + count() AS c, + min(number) AS first, + max(number) AS last, throwIf(first != ($OFFSET < $SIZE AND $LIMIT > 0 ? $OFFSET : 0)), throwIf(last != ($OFFSET < $SIZE AND $LIMIT > 0 ? least($SIZE - 1, $OFFSET + $LIMIT - 1) : 0)), throwIf((c != 0 OR first != 0 OR last != 0) AND (c != last - first + 1)) @@ -18,20 +21,3 @@ for OFFSET in {0..15}; do " done done | $CLICKHOUSE_CLIENT -n --max_block_size 5 - -# Randomized test - -ITERATIONS=1000 -for _ in $(seq $ITERATIONS); do - SIZE=$(($RANDOM % 100)) - OFFSET=$(($RANDOM % 111)) - LIMIT=$(($RANDOM % 111)) - - echo "WITH count() AS c, min(number) AS first, max(number) AS last - SELECT - throwIf(first != ($OFFSET < $SIZE AND $LIMIT > 0 ? $OFFSET : 0)), - throwIf(last != ($OFFSET < $SIZE AND $LIMIT > 0 ? least($SIZE - 1, $OFFSET + $LIMIT - 1) : 0)), - throwIf((c != 0 OR first != 0 OR last != 0) AND (c != last - first + 1)) - FROM (SELECT * FROM numbers($SIZE) LIMIT $OFFSET, $LIMIT); - " -done | $CLICKHOUSE_CLIENT -n --max_block_size $(($RANDOM % 20 + 1)) | uniq diff --git a/tests/queries/0_stateless/01395_limit_more_cases_random.reference b/tests/queries/0_stateless/01395_limit_more_cases_random.reference new file mode 100644 index 00000000000..06b63ea6c2f --- /dev/null +++ b/tests/queries/0_stateless/01395_limit_more_cases_random.reference @@ -0,0 +1 @@ +0 0 0 diff --git a/tests/queries/0_stateless/01395_limit_more_cases_random.sh b/tests/queries/0_stateless/01395_limit_more_cases_random.sh new file mode 100755 index 00000000000..c2f6b060aab --- /dev/null +++ b/tests/queries/0_stateless/01395_limit_more_cases_random.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SIZE=13 +ITERATIONS=300 +for _ in $(seq $ITERATIONS); do + SIZE=$(($RANDOM % 100)) + OFFSET=$(($RANDOM % 111)) + LIMIT=$(($RANDOM % 111)) + + echo "WITH count() AS c, min(number) AS first, max(number) AS last + SELECT + throwIf(first != ($OFFSET < $SIZE AND $LIMIT > 0 ? $OFFSET : 0)), + throwIf(last != ($OFFSET < $SIZE AND $LIMIT > 0 ? least($SIZE - 1, $OFFSET + $LIMIT - 1) : 0)), + throwIf((c != 0 OR first != 0 OR last != 0) AND (c != last - first + 1)) + FROM (SELECT * FROM numbers($SIZE) LIMIT $OFFSET, $LIMIT); + " +done | $CLICKHOUSE_CLIENT -n --max_block_size $(($RANDOM % 20 + 1)) | uniq From 8c4329964f597b1eb8139990a41360243f9337f9 Mon Sep 17 00:00:00 2001 From: Maxim Dergousov Date: Fri, 23 Aug 2024 12:50:18 +0300 Subject: [PATCH 1339/1722] small cosmetic changes in docs --- docs/en/sql-reference/functions/hash-functions.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 55126640e34..908e288cf59 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -695,7 +695,7 @@ Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash value. **Syntax** ```sql -ripeMD160('input') +ripeMD160(input) ``` **Parameters** @@ -707,6 +707,7 @@ ripeMD160('input') - A [UInt256](../data-types/int-uint.md) hash value where the 160-bit RIPEMD-160 hash is stored in the first 20 bytes. The remaining 12 bytes are zero-padded. **Example** + Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string. Query: From 8cc5d766b5d70f22646e6dc2832f806736c76311 Mon Sep 17 00:00:00 2001 From: Maxim Dergousov Date: Fri, 23 Aug 2024 12:52:55 +0300 Subject: [PATCH 1340/1722] small cosmetic changes in docs --- docs/ru/sql-reference/functions/hash-functions.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index b7adcfc1829..d7b90b09122 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -131,7 +131,7 @@ SELECT hex(sipHash128('foo', '\x01', 3)); **СинтакÑиÑ** ```sql -ripeMD160('input') +ripeMD160(input) ``` **Ðргументы** @@ -143,6 +143,7 @@ ripeMD160('input') - [UInt256](../data-types/int-uint.md), где 160-битный хеш RIPEMD-160 хранитÑÑ Ð² первых 20 байтах. ОÑтавшиеÑÑ 12 байт заполнÑÑŽÑ‚ÑÑ Ð½ÑƒÐ»Ñми. **Пример** + ИÑпользуйте функцию [hex](../functions/encoding-functions.md#hex) Ð´Ð»Ñ Ð¿Ñ€ÐµÐ´ÑÑ‚Ð°Ð²Ð»ÐµÐ½Ð¸Ñ Ñ€ÐµÐ·ÑƒÐ»ÑŒÑ‚Ð°Ñ‚Ð° в виде Ñтроки Ñ ÑˆÐµÑтнадцатеричной кодировкой ЗапроÑ: From b0894bffe62722acee2fa5d832ceda9a75754bde Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 12:01:17 +0200 Subject: [PATCH 1341/1722] change test file location --- .../sample.parquet | Bin 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/{column0=Elizabeth => column0=Elizabeth1}/sample.parquet (100%) diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet similarity index 100% rename from tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth/sample.parquet rename to tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/column0=Elizabeth1/sample.parquet From b1e0469d2b085ecc6ff1628a2c707a89b78bb12c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 10:58:43 +0000 Subject: [PATCH 1342/1722] Check setting in runtime --- src/DataTypes/DataTypeObject.cpp | 17 +++++++++++++++-- src/DataTypes/DataTypeObjectDeprecated.cpp | 4 ---- .../03230_json_alias_new_old_types.reference | 2 ++ .../03230_json_alias_new_old_types.sql | 8 ++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03230_json_alias_new_old_types.reference create mode 100644 tests/queries/0_stateless/03230_json_alias_new_old_types.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index a56764f4e6e..62239372b28 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -1,10 +1,12 @@ #include #include +#include #include #include #include #include #include +#include #include #include @@ -513,13 +515,24 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: static DataTypePtr createJSON(const ASTPtr & arguments) { + auto context = CurrentThread::getQueryContext(); + if (!context) + context = Context::getGlobalContextInstance(); + + if (context->getSettingsRef().use_json_alias_for_old_object_type) + { + if (arguments && !arguments->children.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If ypu want to use new JSON type, set setting allow_experimental_json_type = 1"); + + return std::make_shared("JSON", false); + } + return createObject(arguments, DataTypeObject::SchemaFormat::JSON); } void registerDataTypeJSON(DataTypeFactory & factory) { - if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) - factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive); + factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeObjectDeprecated.cpp b/src/DataTypes/DataTypeObjectDeprecated.cpp index 07f9c116e58..2ef3098811d 100644 --- a/src/DataTypes/DataTypeObjectDeprecated.cpp +++ b/src/DataTypes/DataTypeObjectDeprecated.cpp @@ -78,10 +78,6 @@ static DataTypePtr create(const ASTPtr & arguments) void registerDataTypeObjectDeprecated(DataTypeFactory & factory) { factory.registerDataType("Object", create); - if (Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type) - factory.registerSimpleDataType("JSON", - [] { return std::make_shared("JSON", false); }, - DataTypeFactory::Case::Insensitive); } } diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.reference b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference new file mode 100644 index 00000000000..91fd15697be --- /dev/null +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference @@ -0,0 +1,2 @@ +{"a":42} JSON +{"a":42} Object(\'json\') diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.sql b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql new file mode 100644 index 00000000000..97d64f2a606 --- /dev/null +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql @@ -0,0 +1,8 @@ +set allow_experimental_object_type=1; +set allow_experimental_json_type=1; +set use_json_alias_for_old_object_type=0; +select '{"a" : 42}'::JSON as json, toTypeName(json); +set use_json_alias_for_old_object_type=1; +select '{"a" : 42}'::JSON as json, toTypeName(json); +select '{"a" : 42}'::JSON(max_dynamic_paths=100) as json, toTypeName(json); -- {serverError BAD_ARGUMENTS} + From 5c5c803fad08600d9ca667f2246b479e6bff5eb9 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 11:00:03 +0000 Subject: [PATCH 1343/1722] Update docs --- docs/en/operations/settings/settings.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 19db4be17db..f2e75e34609 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5623,7 +5623,6 @@ Default value: `1GiB`. ## use_json_alias_for_old_object_type When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/newjson.md) type. -This setting requires server restart to take effect when changed. Default value: `false`. From 1165ae756d3a6ca1b9b7c7e9be77f1812390c527 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 12:16:16 +0000 Subject: [PATCH 1344/1722] Make dynamic structure selection more consistent --- src/Columns/ColumnDynamic.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..efb835b2e17 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1182,12 +1182,13 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source if (!canAddNewVariants(0, all_variants.size())) { /// Create list of variants with their sizes and sort it. - std::vector> variants_with_sizes; + std::vector> variants_with_sizes; variants_with_sizes.reserve(all_variants.size()); for (const auto & variant : all_variants) { - if (variant->getName() != getSharedVariantTypeName()) - variants_with_sizes.emplace_back(total_sizes[variant->getName()], variant); + auto variant_name = variant->getName(); + if (variant_name != getSharedVariantTypeName()) + variants_with_sizes.emplace_back(total_sizes[variant_name], variant_name, variant); } std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); @@ -1196,14 +1197,14 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source result_variants.reserve(max_dynamic_types + 1); /// +1 for shared variant. /// Add shared variant. result_variants.push_back(getSharedVariantDataType()); - for (const auto & [size, variant] : variants_with_sizes) + for (const auto & [size, variant_name, variant_type] : variants_with_sizes) { /// Add variant to the resulting variants list until we reach max_dynamic_types. if (canAddNewVariant(result_variants.size())) - result_variants.push_back(variant); + result_variants.push_back(variant_type); /// Add all remaining variants into shared_variants_statistics until we reach its max size. else if (new_statistics.shared_variants_statistics.size() < Statistics::MAX_SHARED_VARIANT_STATISTICS_SIZE) - new_statistics.shared_variants_statistics[variant->getName()] = size; + new_statistics.shared_variants_statistics[variant_name] = size; else break; } From ef7970d2c38a301530721751e7bbde908c84721a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 23 Aug 2024 14:19:42 +0200 Subject: [PATCH 1345/1722] Fix typo --- src/DataTypes/DataTypeObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 62239372b28..bb0bb928b4f 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -522,7 +522,7 @@ static DataTypePtr createJSON(const ASTPtr & arguments) if (context->getSettingsRef().use_json_alias_for_old_object_type) { if (arguments && !arguments->children.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If ypu want to use new JSON type, set setting allow_experimental_json_type = 1"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set setting allow_experimental_json_type = 1"); return std::make_shared("JSON", false); } From 6f5210644b95b41cc9d490d4e117c81bd61a1d06 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 23 Aug 2024 14:43:09 +0200 Subject: [PATCH 1346/1722] Update src/Columns/ColumnObject.cpp Co-authored-by: Alexander Gololobov --- src/Columns/ColumnObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 999c0f6088e..e397b03b69e 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1045,7 +1045,7 @@ void ColumnObject::forEachSubcolumnRecursively(DB::IColumn::RecursiveMutableColu bool ColumnObject::structureEquals(const IColumn & rhs) const { - /// 2 Object columns have equal structure if they have the same typed paths and max_dynamic_paths/max_dynamic_types. + /// 2 Object columns have equal structure if they have the same typed paths and global_max_dynamic_paths/max_dynamic_types. const auto * rhs_object = typeid_cast(&rhs); if (!rhs_object || typed_paths.size() != rhs_object->typed_paths.size() || global_max_dynamic_paths != rhs_object->global_max_dynamic_paths || max_dynamic_types != rhs_object->max_dynamic_types) return false; From 2b20b2d4de78acf4fbb08b3f106ebdf410e4587d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 23 Aug 2024 15:02:43 +0200 Subject: [PATCH 1347/1722] Update src/Columns/ColumnDynamic.cpp Co-authored-by: Dmitry Novik --- src/Columns/ColumnDynamic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index efb835b2e17..ef6cd7dcea2 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1181,7 +1181,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source /// Check if the number of all dynamic types exceeds the limit. if (!canAddNewVariants(0, all_variants.size())) { - /// Create list of variants with their sizes and sort it. + /// Create a list of variants with their sizes and names and then sort it. std::vector> variants_with_sizes; variants_with_sizes.reserve(all_variants.size()); for (const auto & variant : all_variants) From 5d6b861ff055de0d04e0c574bf2ebb1e51215ace Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 13:49:36 +0000 Subject: [PATCH 1348/1722] Fix index with limit=0 --- src/Columns/ColumnVariant.cpp | 2 +- .../03228_variant_permutation_issue.reference | 4 ++++ .../03228_variant_permutation_issue.sql | 15 +++++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 2fea3eca123..c6511695f5c 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -953,7 +953,7 @@ ColumnPtr ColumnVariant::index(const IColumn & indexes, size_t limit) const { /// If we have only NULLs, index will take no effect, just return resized column. if (hasOnlyNulls()) - return cloneResized(limit); + return cloneResized(limit == 0 ? indexes.size(): limit); /// Optimization when we have only one non empty variant and no NULLs. /// In this case local_discriminators column is filled with identical values and offsets column diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.reference b/tests/queries/0_stateless/03228_variant_permutation_issue.reference index 10688253e15..be9cdedaf07 100644 --- a/tests/queries/0_stateless/03228_variant_permutation_issue.reference +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.reference @@ -2,3 +2,7 @@ 3 {"foo2":"bar"} 1 2 {"foo2":"baz"} 2 3 {"foo2":"bar"} 1 +2 {"foo2":"bar"} 1 +3 {"foo2":"bar"} 1 +2 {"foo2":"baz"} 2 +3 {"foo2":"bar"} 1 diff --git a/tests/queries/0_stateless/03228_variant_permutation_issue.sql b/tests/queries/0_stateless/03228_variant_permutation_issue.sql index 088361d6430..81eb2ed69af 100644 --- a/tests/queries/0_stateless/03228_variant_permutation_issue.sql +++ b/tests/queries/0_stateless/03228_variant_permutation_issue.sql @@ -16,3 +16,18 @@ SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id; DROP TABLE test_new_json_type; +CREATE TABLE test_new_json_type(id Nullable(UInt32), data JSON, version UInt64) ENGINE=ReplacingMergeTree(version) ORDER BY id settings allow_nullable_key=1; +INSERT INTO test_new_json_type format JSONEachRow +{"id":1,"data":{"foo1":"bar"},"version":1} +{"id":2,"data":{"foo2":"bar"},"version":1} +{"id":3,"data":{"foo2":"bar"},"version":1} +; + +SELECT * FROM test_new_json_type FINAL WHERE data.foo2 is not null ORDER BY id; + +INSERT INTO test_new_json_type SELECT id, '{"foo2":"baz"}' AS _data, version+1 AS _version FROM test_new_json_type where id=2; + +SELECT * FROM test_new_json_type FINAL PREWHERE data.foo2 IS NOT NULL WHERE data.foo2 IS NOT NULL ORDER BY id ASC NULLS FIRST; + +DROP TABLE test_new_json_type; + From eb25e064acf0684cea2b0503f78659431a7c57b4 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 13:53:41 +0000 Subject: [PATCH 1349/1722] reduce amount --- .../02477_projection_materialize_and_zero_copy.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 2750bdf5c4b..0597ac10cd7 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -3,9 +3,9 @@ DROP TABLE IF EXISTS t; create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 -SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8092, index_granularity_bytes='10Mi'; +SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8192, index_granularity_bytes='10Mi'; -insert into t (c1, c18) select number, -number from numbers(2000000); +insert into t (c1, c18) select number, -number from numbers(500000); alter table t add projection p_norm (select * order by c1); From 61fa4e7a476b3db31c22030470341b131501f3b6 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 23 Aug 2024 16:38:48 +0200 Subject: [PATCH 1350/1722] fix logical err of modify statistics --- src/Storages/AlterCommands.cpp | 2 +- src/Storages/StatisticsDescription.cpp | 6 ++++-- .../integration/test_manipulate_statistics/test.py | 14 ++++++++++++-- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d5780e32db3..67b18217767 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -734,7 +734,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { if (!metadata.columns.has(statistics_column_name)) { - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot add statistics for column {}: this column is not found", statistics_column_name); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot modify statistics for column {}: this column is not found", statistics_column_name); } } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 63c849e3806..acf600dd6f7 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include @@ -115,8 +114,11 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other) { + /// If the statistics is empty, it's possible that we have not assign a column_name. + if (empty() && column_name == "") + column_name = other.column_name; if (other.column_name != column_name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", column_name, other.column_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", other.column_name, column_name); types_to_desc = other.types_to_desc; data_type = other.data_type; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 2541c9b946f..ab5559e18fa 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,11 +6,13 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=True + "node1", user_configs=["config/config.xml"], with_zookeeper=True, + macros={"replica": "a", "shard": "shard1"} ) node2 = cluster.add_instance( - "node2", user_configs=["config/config.xml"], with_zookeeper=True + "node2", user_configs=["config/config.xml"], with_zookeeper=True, + macros={"replica": "b", "shard": "shard1"} ) @@ -183,3 +185,11 @@ def test_replicated_table_ddl(started_cluster): ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) + + +def test_replicated_db(started_cluster): + node1.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") + node2.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") + node1.query("CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()") + node2.query("ALTER TABLE test.test_stats MODIFY COLUMN b Float64") + node2.query("ALTER TABLE test.test_stats MODIFY STATISTICS b TYPE tdigest") From 9ad7cfc71ff6e61a51622740b4aff72857389ecd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 23 Aug 2024 23:00:42 +0800 Subject: [PATCH 1351/1722] host fix lower upper performance issue --- src/Functions/LowerUpperUTF8Impl.h | 74 ++++++++++++++++++++++-------- 1 file changed, 54 insertions(+), 20 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 5da085f48e5..490b53b0f4a 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -6,8 +6,10 @@ #include #include -#include #include +#include +#include +#include #include namespace DB @@ -38,38 +40,70 @@ struct LowerUpperUTF8Impl return; } + + UErrorCode error_code = U_ZERO_ERROR; + UCaseMap * csm = ucasemap_open(nullptr, 0, &error_code); + if (U_FAILURE(error_code)) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Call ucasemap_open error:{}", u_errorName(error_code)); + + // String output; + size_t curr_offset = 0; res_data.resize(data.size()); res_offsets.resize_exact(offsets.size()); - - String output; - size_t curr_offset = 0; - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t i = 0; i < input_rows_count; ++i) { const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); - size_t size = offsets[i] - offsets[i - 1]; + size_t size = offsets[i] - offsets[i - 1] - 1; - icu::UnicodeString input(data_start, static_cast(size), "UTF-8"); + int32_t out_size; if constexpr (upper) - input.toUpper(); + out_size = ucasemap_utf8ToUpper( + csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); else - input.toLower(); + out_size = ucasemap_utf8ToLower( + csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); + // std::cout << size << ":" << out_size << ":" << static_cast(res_data[curr_offset + out_size - 1]) << ":" << error_code + // << std::endl; - output.clear(); - input.toUTF8String(output); + if (error_code == U_BUFFER_OVERFLOW_ERROR) + { + size_t new_size = curr_offset + out_size + 1; + res_data.resize(new_size); - /// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first - /// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this - /// case, the behavior is also reasonable. - const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size()); - size_t valid_size = res_end ? res_end - output.data() + 1 : 0; + error_code = U_ZERO_ERROR; + if constexpr (upper) + out_size = ucasemap_utf8ToUpper( + csm, + reinterpret_cast(&res_data[curr_offset]), + res_data.size() - curr_offset, + data_start, + size, + &error_code); + else + out_size = ucasemap_utf8ToLower( + csm, + reinterpret_cast(&res_data[curr_offset]), + res_data.size() - curr_offset, + data_start, + size, + &error_code); + } - res_data.resize(curr_offset + valid_size + 1); - memcpy(&res_data[curr_offset], output.data(), valid_size); - res_data[curr_offset + valid_size] = 0; + if (error_code != U_ZERO_ERROR) + throw DB::Exception( + ErrorCodes::LOGICAL_ERROR, + "Call {} error:{} input:{} input_size:{}", + upper ? "ucasemap_utf8ToUpper" : "ucasemap_utf8ToLower", + u_errorName(error_code), + std::string_view(data_start, size), + size); - curr_offset += valid_size + 1; + res_data[curr_offset + out_size] = 0; + curr_offset += out_size + 1; res_offsets[i] = curr_offset; } + + res_data.resize(curr_offset); } static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t) From 223de640721a84b1434417379bd47636ae3b8841 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 17:09:28 +0200 Subject: [PATCH 1352/1722] init --- .../__init__.py | 0 .../configs/config.d/cluster.xml | 11 ---- .../configs/config.xml | 9 ---- .../test_incorrect_datetime_format/test.py | 54 ------------------- ...301_test_incorrect_datetime_msan.reference | 3 ++ .../032301_test_incorrect_datetime_msan.sh | 20 +++++++ 6 files changed, 23 insertions(+), 74 deletions(-) delete mode 100644 tests/integration/test_incorrect_datetime_format/__init__.py delete mode 100644 tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml delete mode 100644 tests/integration/test_incorrect_datetime_format/configs/config.xml delete mode 100644 tests/integration/test_incorrect_datetime_format/test.py create mode 100644 tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference create mode 100644 tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh diff --git a/tests/integration/test_incorrect_datetime_format/__init__.py b/tests/integration/test_incorrect_datetime_format/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml b/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml deleted file mode 100644 index a27968fb3d2..00000000000 --- a/tests/integration/test_incorrect_datetime_format/configs/config.d/cluster.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - node - - - - - diff --git a/tests/integration/test_incorrect_datetime_format/configs/config.xml b/tests/integration/test_incorrect_datetime_format/configs/config.xml deleted file mode 100644 index 053b5d30418..00000000000 --- a/tests/integration/test_incorrect_datetime_format/configs/config.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - information - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - diff --git a/tests/integration/test_incorrect_datetime_format/test.py b/tests/integration/test_incorrect_datetime_format/test.py deleted file mode 100644 index 3cdc6781534..00000000000 --- a/tests/integration/test_incorrect_datetime_format/test.py +++ /dev/null @@ -1,54 +0,0 @@ -import logging -import pytest -from helpers.cluster import ClickHouseCluster - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node", - main_configs=[ - "configs/config.d/cluster.xml", - ], - ) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") - - node = cluster.instances["node"] - node.query( - """ - CREATE TABLE tab - ( - a DateTime, - pk String - ) Engine = MergeTree() ORDER BY pk; - """ - ) - - yield cluster - finally: - cluster.shutdown() - - -def test_incorrect_datetime_format(cluster): - """ - Test for an MSan issue which is caused by parsing incorrect datetime string - """ - - node = cluster.instances["node"] - - res = node.query("SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'").strip() - assert res == "0" - - error = node.query_and_get_error( - "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'" - ).strip() - assert "Cannot parse time component of DateTime 09:58:0" in error - - error = node.query_and_get_error( - "SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'" - ).strip() - assert "Cannot convert string '2024-08-0 09:58:09' to type DateTime" in error diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference new file mode 100644 index 00000000000..346e6731867 --- /dev/null +++ b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference @@ -0,0 +1,3 @@ +0 +OK +OK diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh new file mode 100644 index 00000000000..c54c69dce58 --- /dev/null +++ b/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query " +CREATE TABLE tab +( + a DateTime, + pk String +) Engine = MergeTree() ORDER BY pk; +" + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:09'" + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM tab WHERE a = '2024-08-06 09:58:0'" 2>&1 | grep -F -q "Cannot parse time component of DateTime 09:58:0" && echo "OK" || echo "FAIL"; + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM tab WHERE a = '2024-08-0 09:58:09'" 2>&1 | grep -F -q "Cannot convert string '2024-08-0 09:58:09" && echo "OK" || echo "FAIL"; From 7aabd7d2fd4a03ddea5ef311cf89b2eb7520674c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Aug 2024 15:11:51 +0000 Subject: [PATCH 1353/1722] Fix resolving dynamic subcolumns from subqueries in analyzer --- src/Analyzer/Resolve/IdentifierResolver.cpp | 2 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 3 +++ src/Analyzer/Resolve/TableExpressionData.h | 1 + .../03228_dynamic_subcolumns_from_subquery.reference | 4 ++++ .../03228_dynamic_subcolumns_from_subquery.sql | 9 +++++++++ 5 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference create mode 100644 tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 14d4acc7c9b..80e7d1e4445 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -692,7 +692,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( result_column_node = it->second; } /// Check if it's a dynamic subcolumn - else + else if (table_expression_data.supports_subcolumns) { auto [column_name, dynamic_subcolumn_name] = Nested::splitName(identifier_full_name); auto jt = table_expression_data.column_name_to_column_node.find(column_name); diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 004da5ed341..a18c2901a58 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4379,7 +4379,10 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); if (storage_snapshot->storage.supportsSubcolumns()) + { get_column_options.withSubcolumns(); + table_expression_data.supports_subcolumns = true; + } auto column_names_and_types = storage_snapshot->getColumns(get_column_options); table_expression_data.column_names_and_types = NamesAndTypes(column_names_and_types.begin(), column_names_and_types.end()); diff --git a/src/Analyzer/Resolve/TableExpressionData.h b/src/Analyzer/Resolve/TableExpressionData.h index 18cbfa32366..6770672d0c2 100644 --- a/src/Analyzer/Resolve/TableExpressionData.h +++ b/src/Analyzer/Resolve/TableExpressionData.h @@ -36,6 +36,7 @@ struct AnalysisTableExpressionData std::string database_name; std::string table_name; bool should_qualify_columns = true; + bool supports_subcolumns = false; NamesAndTypes column_names_and_types; ColumnNameToColumnNodeMap column_name_to_column_node; std::unordered_set subcolumn_names; /// Subset columns that are subcolumns of other columns diff --git a/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference new file mode 100644 index 00000000000..153ad78f694 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.reference @@ -0,0 +1,4 @@ +str +42 +42 +42 diff --git a/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql new file mode 100644 index 00000000000..a10b0cb2809 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_subcolumns_from_subquery.sql @@ -0,0 +1,9 @@ +set allow_experimental_dynamic_type=1; +set allow_experimental_json_type=1; +set allow_experimental_analyzer=1; + +select d.String from (select 'str'::Dynamic as d); +select json.a from (select '{"a" : 42}'::JSON as json); +select json.a from (select '{"a" : 42}'::JSON(a UInt32) as json); +select json.a.:Int64 from (select materialize('{"a" : 42}')::JSON as json); + From 327828777999447756823fe1e6c99cb867896154 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 23 Aug 2024 15:26:45 +0000 Subject: [PATCH 1354/1722] fix --- .../Optimizations/optimizeReadInOrder.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 5df7d7b4e82..9dc4992cddd 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -255,7 +255,7 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi /// Add more functions to fixed columns. /// Functions result is fixed if all arguments are fixed or constants. -void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) +void enrichFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) { struct Frame { @@ -300,20 +300,20 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) { if (frame.node->function_base->isDeterministicInScopeOfQuery()) { - //std::cerr << "*** enreachFixedColumns check " << frame.node->result_name << std::endl; + //std::cerr << "*** enrichFixedColumns check " << frame.node->result_name << std::endl; bool all_args_fixed_or_const = true; for (const auto * child : frame.node->children) { if (!child->column && !fixed_columns.contains(child)) { - //std::cerr << "*** enreachFixedColumns fail " << child->result_name << ' ' << static_cast(child) << std::endl; + //std::cerr << "*** enrichFixedColumns fail " << child->result_name << ' ' << static_cast(child) << std::endl; all_args_fixed_or_const = false; } } if (all_args_fixed_or_const) { - //std::cerr << "*** enreachFixedColumns add " << frame.node->result_name << ' ' << static_cast(frame.node) << std::endl; + //std::cerr << "*** enrichFixedColumns add " << frame.node->result_name << ' ' << static_cast(frame.node) << std::endl; fixed_columns.insert(frame.node); } } @@ -357,7 +357,7 @@ InputOrderInfoPtr buildInputOrderInfo( } } - enreachFixedColumns(sorting_key_dag, fixed_key_columns); + enrichFixedColumns(sorting_key_dag, fixed_key_columns); } /// This is a result direction we will read from MergeTree @@ -530,7 +530,7 @@ AggregationInputOrder buildInputOrderInfo( } } - enreachFixedColumns(sorting_key_dag, fixed_key_columns); + enrichFixedColumns(sorting_key_dag, fixed_key_columns); for (const auto * output : dag->getOutputs()) { @@ -804,7 +804,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n buildSortingDAG(node, dag, fixed_columns, limit); if (dag && !fixed_columns.empty()) - enreachFixedColumns(*dag, fixed_columns); + enrichFixedColumns(*dag, fixed_columns); if (auto * reading = typeid_cast(reading_node->step.get())) { @@ -858,7 +858,7 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl buildSortingDAG(node, dag, fixed_columns, limit); if (dag && !fixed_columns.empty()) - enreachFixedColumns(*dag, fixed_columns); + enrichFixedColumns(*dag, fixed_columns); if (auto * reading = typeid_cast(reading_node->step.get())) { From 7517ef4cc9f990e1ac949366a1ec8688aba0420f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 16:08:04 +0000 Subject: [PATCH 1355/1722] fix async insert with alter modify column --- src/Interpreters/AsynchronousInsertQueue.cpp | 178 ++++++++++-------- src/Interpreters/AsynchronousInsertQueue.h | 5 + .../03229_async_insert_alter.reference | 8 + .../0_stateless/03229_async_insert_alter.sql | 46 +++++ .../03229_async_insert_alter_http.reference | 8 + .../03229_async_insert_alter_http.sh | 55 ++++++ 6 files changed, 226 insertions(+), 74 deletions(-) create mode 100644 tests/queries/0_stateless/03229_async_insert_alter.reference create mode 100644 tests/queries/0_stateless/03229_async_insert_alter.sql create mode 100644 tests/queries/0_stateless/03229_async_insert_alter_http.reference create mode 100755 tests/queries/0_stateless/03229_async_insert_alter_http.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 52f8a8dd42a..5ed0d6d6257 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -699,6 +699,17 @@ catch (...) tryLogCurrentException("AsynchronousInsertQueue", "Failed to add elements to AsynchronousInsertLog"); } +void convertBlockToHeader(Block & block, const Block & header) +{ + auto converting_dag = ActionsDAG::makeConvertingActions( + block.getColumnsWithTypeAndName(), + header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(block); +} + String serializeQuery(const IAST & query, size_t max_length) { return query.hasSecretParts() @@ -794,6 +805,48 @@ try if (async_insert_log) log_elements.reserve(data->entries.size()); + auto add_entry_to_asynchronous_insert_log = [&]( + const InsertData::EntryPtr & entry, + const NameToNameMap & query_by_format, + const String & parsing_exception, + size_t num_rows, + size_t num_bytes) + { + if (!async_insert_log) + return; + + AsynchronousInsertLogElement elem; + elem.event_time = timeInSeconds(entry->create_time); + elem.event_time_microseconds = timeInMicroseconds(entry->create_time); + elem.database = query_database; + elem.table = query_table; + elem.format = entry->format; + elem.query_id = entry->query_id; + elem.bytes = num_bytes; + elem.rows = num_rows; + elem.exception = parsing_exception; + elem.data_kind = entry->chunk.getDataKind(); + elem.timeout_milliseconds = data->timeout_ms.count(); + elem.flush_query_id = insert_query_id; + + auto it = query_by_format.find(entry->format); + elem.query_for_logging = it != query_by_format.end() ? it->second : key.query_str; + + /// If there was a parsing error, + /// the entry won't be flushed anyway, + /// so add the log element immediately. + if (!elem.exception.empty()) + { + elem.status = AsynchronousInsertLogElement::ParsingError; + async_insert_log->add(std::move(elem)); + } + else + { + elem.status = AsynchronousInsertLogElement::Ok; + log_elements.push_back(elem); + } + }; + try { interpreter = std::make_unique( @@ -822,49 +875,21 @@ try catch (...) { logExceptionBeforeStart(query_for_logging, insert_context, key.query, query_span, start_watch.elapsedMilliseconds()); + + if (async_insert_log) + { + auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); + for (const auto & entry : data->entries) + add_entry_to_asynchronous_insert_log(entry, query_by_format, "", 0, entry->chunk.byteSize()); + + auto exception = getCurrentExceptionMessage(false); + auto flush_time = std::chrono::system_clock::now(); + appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, exception); + } throw; } - auto add_entry_to_asynchronous_insert_log = [&](const auto & entry, - const auto & entry_query_for_logging, - const auto & exception, - size_t num_rows, - size_t num_bytes, - Milliseconds timeout_ms) - { - if (!async_insert_log) - return; - - AsynchronousInsertLogElement elem; - elem.event_time = timeInSeconds(entry->create_time); - elem.event_time_microseconds = timeInMicroseconds(entry->create_time); - elem.query_for_logging = entry_query_for_logging; - elem.database = query_database; - elem.table = query_table; - elem.format = entry->format; - elem.query_id = entry->query_id; - elem.bytes = num_bytes; - elem.rows = num_rows; - elem.exception = exception; - elem.data_kind = entry->chunk.getDataKind(); - elem.timeout_milliseconds = timeout_ms.count(); - elem.flush_query_id = insert_query_id; - - /// If there was a parsing error, - /// the entry won't be flushed anyway, - /// so add the log element immediately. - if (!elem.exception.empty()) - { - elem.status = AsynchronousInsertLogElement::ParsingError; - async_insert_log->add(std::move(elem)); - } - else - { - log_elements.push_back(elem); - } - }; - - auto finish_entries = [&] + auto finish_entries = [&](size_t num_rows, size_t num_bytes) { for (const auto & entry : data->entries) { @@ -877,6 +902,13 @@ try auto flush_time = std::chrono::system_clock::now(); appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, ""); } + + LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); + queue_shard_flush_time_history.updateWithCurrentTime(); + + bool pulling_pipeline = false; + logQueryFinish( + query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); }; try @@ -891,20 +923,9 @@ try ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); - auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) - { - LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); - queue_shard_flush_time_history.updateWithCurrentTime(); - - bool pulling_pipeline = false; - logQueryFinish( - query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); - }; - if (chunk.getNumRows() == 0) { - finish_entries(); - log_and_add_finish_to_query_log(0, 0); + finish_entries(0, 0); return; } @@ -917,7 +938,7 @@ try CompletedPipelineExecutor completed_executor(pipeline); completed_executor.execute(); - log_and_add_finish_to_query_log(num_rows, num_bytes); + finish_entries(num_rows, num_bytes); } catch (...) { @@ -931,8 +952,6 @@ try } throw; } - - finish_entries(); } catch (const Exception & e) { @@ -1013,7 +1032,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, query_for_logging, current_exception, num_rows, num_bytes, data->timeout_ms); + add_to_async_insert_log(entry, {}, current_exception, num_rows, num_bytes); current_exception.clear(); entry->resetChunk(); @@ -1036,19 +1055,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( auto chunk_info = std::make_shared(); auto result_columns = header.cloneEmptyColumns(); - std::unordered_map format_to_query; - - auto get_query_by_format = [&](const String & format) -> const String & - { - auto [it, inserted] = format_to_query.try_emplace(format); - if (!inserted) - return it->second; - - auto query = key.query->clone(); - assert_cast(*query).format = format; - it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); - return it->second; - }; + auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); for (const auto & entry : data->entries) { @@ -1057,17 +1064,19 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected entry with data kind Preprocessed. Got: {}", entry->chunk.getDataKind()); - auto columns = block->getColumns(); + Block block_to_insert = *block; + if (!isCompatibleHeader(block_to_insert, header)) + convertBlockToHeader(block_to_insert, header); + + auto columns = block_to_insert.getColumns(); for (size_t i = 0, s = columns.size(); i < s; ++i) result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - total_rows += block->rows(); + total_rows += block_to_insert.rows(); chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - const auto & query_for_logging = get_query_by_format(entry->format); - add_to_async_insert_log(entry, query_for_logging, "", block->rows(), block->bytes(), data->timeout_ms); - + add_to_async_insert_log(entry, query_by_format, "", block_to_insert.rows(), block_to_insert.bytes()); entry->resetChunk(); } @@ -1076,6 +1085,27 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( return chunk; } +NameToNameMap AsynchronousInsertQueue::getQueriesByFormat( + const ASTPtr & query, + const std::list & entries, + const ContextPtr & insert_context) +{ + std::unordered_map format_to_query; + auto query_copy = query->clone(); + + for (const auto & entry : entries) + { + auto [it, inserted] = format_to_query.try_emplace(entry->format); + if (!inserted) + continue; + + assert_cast(*query_copy).format = entry->format; + it->second = serializeQuery(*query_copy, insert_context->getSettingsRef().log_queries_cut_to_length); + } + + return format_to_query; +} + template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index da14b43d276..9a84fe8bb12 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -293,6 +293,11 @@ private: const ContextPtr & insert_context, LogFunc && add_to_async_insert_log); + static NameToNameMap getQueriesByFormat( + const ASTPtr & query, + const std::list & entries, + const ContextPtr & insert_context); + template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); diff --git a/tests/queries/0_stateless/03229_async_insert_alter.reference b/tests/queries/0_stateless/03229_async_insert_alter.reference new file mode 100644 index 00000000000..f66021d0bfe --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter.reference @@ -0,0 +1,8 @@ +42 24 0 +42 24 0 +43 34 55 +42 24 +43 34 +INSERT INTO default.t_async_insert_alter (id, v1) FORMAT Values Preprocessed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Preprocessed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Preprocessed FlushError diff --git a/tests/queries/0_stateless/03229_async_insert_alter.sql b/tests/queries/0_stateless/03229_async_insert_alter.sql new file mode 100644 index 00000000000..a95bbc6e55f --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter.sql @@ -0,0 +1,46 @@ +-- Tags: no-parallel + +SET wait_for_async_insert = 0; +SET async_insert_busy_timeout_max_ms = 300000; +SET async_insert_busy_timeout_min_ms = 300000; +SET async_insert_use_adaptive_busy_timeout = 0; + +DROP TABLE IF EXISTS t_async_insert_alter; + +CREATE TABLE t_async_insert_alter (id Int64, v1 Int64) ENGINE = MergeTree ORDER BY id SETTINGS async_insert = 1; + +-- ADD COLUMN + +INSERT INTO t_async_insert_alter VALUES (42, 24); + +ALTER TABLE t_async_insert_alter ADD COLUMN value2 Int64; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; + +-- MODIFY COLUMN + +INSERT INTO t_async_insert_alter VALUES (43, 34, 55); + +ALTER TABLE t_async_insert_alter MODIFY COLUMN value2 String; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; + +-- DROP COLUMN + +INSERT INTO t_async_insert_alter VALUES ('100', '200', '300'); + +ALTER TABLE t_async_insert_alter DROP COLUMN value2; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; +SELECT query, data_kind, status FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_alter' ORDER BY event_time_microseconds; + +DROP TABLE t_async_insert_alter; diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.reference b/tests/queries/0_stateless/03229_async_insert_alter_http.reference new file mode 100644 index 00000000000..195701d2b82 --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.reference @@ -0,0 +1,8 @@ +42 24 0 +42 24 0 +43 34 55 +42 24 +43 34 +INSERT INTO default.t_async_insert_alter (id, v1) FORMAT Values Parsed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Parsed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Parsed FlushError diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.sh b/tests/queries/0_stateless/03229_async_insert_alter_http.sh new file mode 100755 index 00000000000..18e68f51285 --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + DROP TABLE IF EXISTS t_async_insert_alter; + CREATE TABLE t_async_insert_alter (id Int64, v1 Int64) ENGINE = MergeTree ORDER BY id SETTINGS async_insert = 1; +" + +url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0" + +# ADD COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES (42, 24)" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter ADD COLUMN value2 Int64; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; +" + +# MODIFY COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES (43, 34, 55)" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter MODIFY COLUMN value2 String; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; +" + +## DROP COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES ('100', '200', '300')" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter DROP COLUMN value2; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; + SELECT query, data_kind, status FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_alter' ORDER BY event_time_microseconds; + + DROP TABLE t_async_insert_alter; +" From eb94847ed9c5d5834c05560fb5e60fc7ab1224ff Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 23 Aug 2024 18:14:02 +0200 Subject: [PATCH 1356/1722] Fix --- .../test/integration/runner/requirements.txt | 2 + .../DataLakes/DeltaLakeMetadata.cpp | 13 +-- tests/integration/test_storage_delta/test.py | 95 +++++++++++++++++++ 3 files changed, 104 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 428986b5562..2c9df73ccca 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -112,3 +112,5 @@ wadllib==1.3.6 websocket-client==0.59.0 wheel==0.37.1 zipp==1.0.0 +deltalake==0.16.0 + diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 7aadba18817..9b972711cb1 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -425,8 +425,9 @@ struct DeltaLakeMetadataImpl { auto field = fields->getObject(static_cast(i)); element_names.push_back(field->getValue("name")); - auto required = field->getValue("required"); - element_types.push_back(getFieldType(field, "type", required)); + + auto is_nullable = field->getValue("nullable"); + element_types.push_back(getFieldType(field, "type", is_nullable)); } return std::make_shared(element_types, element_names); @@ -434,16 +435,16 @@ struct DeltaLakeMetadataImpl if (type_name == "array") { - bool is_nullable = type->getValue("containsNull"); - auto element_type = getFieldType(type, "elementType", is_nullable); + bool element_nullable = type->getValue("containsNull"); + auto element_type = getFieldType(type, "elementType", element_nullable); return std::make_shared(element_type); } if (type_name == "map") { - bool is_nullable = type->getValue("containsNull"); auto key_type = getFieldType(type, "keyType", /* is_nullable */false); - auto value_type = getFieldType(type, "valueType", is_nullable); + bool value_nullable = type->getValue("valueContainsNull"); + auto value_type = getFieldType(type, "valueType", value_nullable); return std::make_shared(key_type, value_type); } diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 75a4b6cc221..d000875d8d1 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -29,6 +29,9 @@ from datetime import datetime from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject +import pyarrow as pa +import pyarrow.parquet as pq +from deltalake.writer import write_deltalake from helpers.s3_tools import ( prepare_s3_bucket, @@ -728,3 +731,95 @@ SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.mini ) == 1 ) + + +def test_complex_types(started_cluster): + node = started_cluster.instances["node1"] + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + + schema = pa.schema( + [ + ("id", pa.int32()), + ("name", pa.string()), + ( + "address", + pa.struct( + [ + ("street", pa.string()), + ("city", pa.string()), + ("state", pa.string()), + ] + ), + ), + ("interests", pa.list_(pa.string())), + ( + "metadata", + pa.map_( + pa.string(), pa.string() + ), # Map with string keys and string values + ), + ] + ) + + # Create sample data + data = [ + pa.array([1, 2, 3], type=pa.int32()), + pa.array(["John Doe", "Jane Smith", "Jake Johnson"], type=pa.string()), + pa.array( + [ + {"street": "123 Elm St", "city": "Springfield", "state": "IL"}, + {"street": "456 Maple St", "city": "Shelbyville", "state": "IL"}, + {"street": "789 Oak St", "city": "Ogdenville", "state": "IL"}, + ], + type=schema.field("address").type, + ), + pa.array( + [ + pa.array(["dancing", "coding", "hiking"]), + pa.array(["dancing", "coding", "hiking"]), + pa.array(["dancing", "coding", "hiking"]), + ], + type=schema.field("interests").type, + ), + pa.array( + [ + {"key1": "value1", "key2": "value2"}, + {"key1": "value3", "key2": "value4"}, + {"key1": "value5", "key2": "value6"}, + ], + type=schema.field("metadata").type, + ), + ] + + endpoint_url = f"http://{started_cluster.minio_ip}:{started_cluster.minio_port}" + aws_access_key_id = "minio" + aws_secret_access_key = "minio123" + + storage_options = { + "AWS_ENDPOINT_URL": endpoint_url, + "AWS_ACCESS_KEY_ID": aws_access_key_id, + "AWS_SECRET_ACCESS_KEY": aws_secret_access_key, + "AWS_ALLOW_HTTP": "true", + "AWS_S3_ALLOW_UNSAFE_RENAME": "true", + } + path = f"s3://root/testtable" + table = pa.Table.from_arrays(data, schema=schema) + + write_deltalake(path, table, storage_options=storage_options) + + assert "1\n2\n3\n" in node.query( + f"SELECT id FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + ) + assert ( + "('123 Elm St','Springfield','IL')\n('456 Maple St','Shelbyville','IL')\n('789 Oak St','Ogdenville','IL')" + in node.query( + f"SELECT address FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + ) + ) + assert ( + "{'key1':'value1','key2':'value2'}\n{'key1':'value3','key2':'value4'}\n{'key1':'value5','key2':'value6'}" + in node.query( + f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + ) + ) From b39c5cdb4021fd2b52fde4d320e421abcd868c77 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 16:17:14 +0000 Subject: [PATCH 1357/1722] fix tests --- .../02790_async_queries_in_query_log.reference | 12 ++++++------ ...03148_async_queries_in_query_log_errors.reference | 6 +++--- .../queries/0_stateless/03229_async_insert_alter.sql | 1 + .../0_stateless/03229_async_insert_alter_http.sh | 1 + 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference index 567e8d4f4b5..84254cc8735 100644 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference @@ -9,7 +9,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -26,7 +26,7 @@ written_rows: 4 written_bytes: 16 result_rows: 4 result_bytes: 16 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -54,7 +54,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -71,7 +71,7 @@ written_rows: 6 written_bytes: 24 result_rows: 6 result_bytes: 24 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -118,7 +118,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -135,7 +135,7 @@ written_rows: 3 written_bytes: 12 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] diff --git a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference index 685d28268f6..7cc2efd50ec 100644 --- a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference +++ b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference @@ -4,7 +4,7 @@ Row 1: ────── database: default table: async_insert_landing -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values format: Values error: DB::Exc*****on: Cannot parse string 'Invalid' as UInt32: populated_flush_query_id: 1 @@ -18,7 +18,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -35,7 +35,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] diff --git a/tests/queries/0_stateless/03229_async_insert_alter.sql b/tests/queries/0_stateless/03229_async_insert_alter.sql index a95bbc6e55f..022e386bef0 100644 --- a/tests/queries/0_stateless/03229_async_insert_alter.sql +++ b/tests/queries/0_stateless/03229_async_insert_alter.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel +-- no-parallel because the test uses FLUSH ASYNC INSERT QUEUE SET wait_for_async_insert = 0; SET async_insert_busy_timeout_max_ms = 300000; diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.sh b/tests/queries/0_stateless/03229_async_insert_alter_http.sh index 18e68f51285..fe72ed3299a 100755 --- a/tests/queries/0_stateless/03229_async_insert_alter_http.sh +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash # Tags: no-parallel +# no-parallel because the test uses FLUSH ASYNC INSERT QUEUE CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From b83c6239acc14d1c8f1644abe6b25debb95cc8fd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 18:21:06 +0200 Subject: [PATCH 1358/1722] rename tests --- ...san.reference => 03231_test_incorrect_datetime_msan.reference} | 0 ...ect_datetime_msan.sh => 03231_test_incorrect_datetime_msan.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{032301_test_incorrect_datetime_msan.reference => 03231_test_incorrect_datetime_msan.reference} (100%) rename tests/queries/0_stateless/{032301_test_incorrect_datetime_msan.sh => 03231_test_incorrect_datetime_msan.sh} (100%) diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference b/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.reference similarity index 100% rename from tests/queries/0_stateless/032301_test_incorrect_datetime_msan.reference rename to tests/queries/0_stateless/03231_test_incorrect_datetime_msan.reference diff --git a/tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh b/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh similarity index 100% rename from tests/queries/0_stateless/032301_test_incorrect_datetime_msan.sh rename to tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh From bea98fed5e2107e621dc3f7578ee3eed14e2c5b5 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 23 Aug 2024 16:45:38 +0000 Subject: [PATCH 1359/1722] chmod +x on test --- tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh diff --git a/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh b/tests/queries/0_stateless/03231_test_incorrect_datetime_msan.sh old mode 100644 new mode 100755 From 80504e7b9b52fec79a89e2fff5881ca397022107 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 19:07:25 +0000 Subject: [PATCH 1360/1722] fix test 03228_virtual_column_merge_dist --- .../queries/0_stateless/03228_virtual_column_merge_dist.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql index caf00a2e407..e58c7f38d3b 100644 --- a/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql +++ b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql @@ -1,3 +1,6 @@ +-- There is a bug in old analyzer with currentDatabase() and distributed query. +SET enable_analyzer = 1; + DROP TABLE IF EXISTS t_local_1; DROP TABLE IF EXISTS t_local_2; DROP TABLE IF EXISTS t_merge; @@ -10,7 +13,7 @@ INSERT INTO t_local_1 VALUES (1); INSERT INTO t_local_2 VALUES (2); CREATE TABLE t_merge AS t_local_1 ENGINE = Merge(currentDatabase(), '^(t_local_1|t_local_2)$'); -CREATE TABLE t_distr AS t_local_1 engine=Distributed('test_shard_localhost', currentDatabase(), t_merge, rand()); +CREATE TABLE t_distr AS t_local_1 ENGINE = Distributed('test_shard_localhost', currentDatabase(), t_merge, rand()); SELECT a, _table FROM t_merge ORDER BY a; SELECT a, _table FROM t_distr ORDER BY a; From 66be44e7aa719681b7f13e31a91f2c105e705a4f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 23 Aug 2024 19:09:09 +0000 Subject: [PATCH 1361/1722] better --- tests/ci/integration_tests_runner.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 074c947fe02..deddcb95cf4 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -33,7 +33,7 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 10 # run whole pytest several times +FLAKY_TRIES_COUNT = 5 # run whole pytest several times FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 @@ -794,7 +794,7 @@ class ClickhouseIntegrationTestsRunner: } # type: Dict tests_times = defaultdict(float) # type: Dict tests_log_paths = defaultdict(list) - + id_counter = 0 for test_to_run in tests_to_run: tries_num = 1 if should_fail else FLAKY_TRIES_COUNT for i in range(tries_num): @@ -805,12 +805,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Running tests for the %s time", i) group_counters, group_test_times, log_paths = self.try_run_test_group( repo_path, - "bugfix" if should_fail else "flaky", + f"bugfix_{id_counter}" if should_fail else f"flaky{id_counter}", [test_to_run], 1, 1, FLAKY_REPEAT_COUNT, ) + id_counter = id_counter + 1 for counter, value in group_counters.items(): logging.info( "Tests from group %s stats, %s count %s", From 83fa51f400fba43cd4bb961419f7dd572ec240a0 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Fri, 23 Aug 2024 21:42:54 +0200 Subject: [PATCH 1362/1722] PR fixes: update documentation on JSONCompactWithProgress format --- docs/en/interfaces/formats.md | 38 ++++++++++++++++++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 8795b71e0ac..f2ca803453e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -930,7 +930,43 @@ Columns that are not present in the block will be filled with default values (yo In this format, ClickHouse outputs each row as a separated, newline-delimited JSON Object. -Each row is either a metadata object, data object, progress information or statistics object. +Each row is either a metadata object, data object, progress information or statistics object: + +1. **Metadata Object (`meta`)** + - Describes the structure of the data rows. + - Fields: `name` (column name), `type` (data type, e.g., `UInt32`, `String`, etc.). + - Example: `{"meta": [{"name":"id", "type":"UInt32"}, {"name":"name", "type":"String"}]}` + - Appears before any data objects. + +2. **Data Object (`data`)** + - Represents a row of query results. + - Fields: An array with values corresponding to the columns defined in the metadata. + - Example: `{"data":["1", "John Doe"]}` + - Appears after the metadata object, one per row. + +3. **Progress Information Object (`progress`)** + - Provides real-time progress feedback during query execution. + - Fields: `read_rows`, `read_bytes`, `written_rows`, `written_bytes`, `total_rows_to_read`, `result_rows`, `result_bytes`, `elapsed_ns`. + - Example: `{"progress":{"read_rows":"8","read_bytes":"168"}}` + - May appear intermittently. + +4. **Statistics Object (`statistics`)** + - Summarizes query execution statistics. + - Fields: `rows`, `rows_before_limit_at_least`, `elapsed`, `rows_read`, `bytes_read`. + - Example: `{"statistics": {"rows":2, "elapsed":0.001995, "rows_read":8}}` + - Appears at the end. + +5. **Exception Object (`exception`)** + - Represents an error that occurred during query execution. + - Fields: A single text field containing the error message. + - Example: `{"exception": "Code: 395. DB::Exception: Value passed to 'throwIf' function is non-zero..."}` + - Appears when an error is encountered. + +6. **Totals Object (`totals`)** + - Provides the totals for each numeric column in the result set. + - Fields: An array with total values corresponding to the columns defined in the metadata. + - Example: `{"totals": ["", "3"]}` + - Appears at the end of the data rows, if applicable. Example: From 7916490afa8d4ac54bcd8b541dfe02ec9a715d3a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 20:26:31 +0000 Subject: [PATCH 1363/1722] add more cases for test --- .../Executors/StreamingFormatExecutor.cpp | 10 ++-------- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- ...ync_insert_query_params_bad_type.reference | 1 + ...228_async_insert_query_params_bad_type.sql | 20 +++++++++++++++++++ 4 files changed, 24 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference create mode 100644 tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 5b35ac6020d..10a7b7fd7f5 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -35,15 +35,9 @@ MutableColumns StreamingFormatExecutor::getResultColumns() void StreamingFormatExecutor::setQueryParameters(const NameToNameMap & parameters) { - if (parameters.empty()) - return; - /// Query parameters make sense only for format Values. - auto * values_format = typeid_cast(format.get()); - if (!values_format) - return; - - values_format->setQueryParameters(parameters); + if (auto * values_format = typeid_cast(format.get())) + values_format->setQueryParameters(parameters); } size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index e34000df5f4..10d0e051665 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -665,7 +665,7 @@ void ValuesBlockInputFormat::resetReadBuffer() void ValuesBlockInputFormat::setQueryParameters(const NameToNameMap & parameters) { - if (parameters.empty()) + if (parameters == context->getQueryParameters()) return; auto context_copy = Context::createCopy(context); diff --git a/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference new file mode 100644 index 00000000000..e965047ad7c --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.reference @@ -0,0 +1 @@ +Hello diff --git a/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql new file mode 100644 index 00000000000..359174245cb --- /dev/null +++ b/tests/queries/0_stateless/03228_async_insert_query_params_bad_type.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS t_async_insert_params; + +CREATE TABLE t_async_insert_params (id UInt64) ENGINE = MergeTree ORDER BY tuple(); + +SET param_p1 = 'Hello'; + +SET async_insert = 1; +SET wait_for_async_insert = 1; + +INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER } +INSERT INTO t_async_insert_params VALUES ({p1:String}); -- { serverError TYPE_MISMATCH } + +ALTER TABLE t_async_insert_params MODIFY COLUMN id String; + +INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER } +INSERT INTO t_async_insert_params VALUES ({p1:String}); + +SELECT * FROM t_async_insert_params ORDER BY id; + +DROP TABLE t_async_insert_params; From a82421719383041a839289093d1882265a068cd1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Aug 2024 20:29:04 +0000 Subject: [PATCH 1364/1722] Done --- ..._rewrite_sum_column_and_constant.reference | 26 +++++++++---------- ...alyzer_rewrite_sum_column_and_constant.sql | 11 ++++---- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference index 802d920aaef..b41635f014e 100644 --- a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference +++ b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.reference @@ -1635,21 +1635,21 @@ QUERY id: 0 JOIN TREE TABLE id: 10, alias: __table1, table_name: default.test_table SELECT sum(float64 + 2) From test_table; -26.5 +26.875 SELECT sum(2 + float64) From test_table; -26.5 +26.875 SELECT sum(float64 - 2) From test_table; -6.5 +6.875 SELECT sum(2 - float64) From test_table; --6.5 +-6.875 SELECT sum(float64) + 2 * count(float64) From test_table; -26.5 +26.875 SELECT 2 * count(float64) + sum(float64) From test_table; -26.5 +26.875 SELECT sum(float64) - 2 * count(float64) From test_table; -6.5 +6.875 SELECT 2 * count(float64) - sum(float64) From test_table; --6.5 +-6.875 EXPLAIN QUERY TREE (SELECT sum(float64 + 2) From test_table); QUERY id: 0 PROJECTION COLUMNS @@ -2463,25 +2463,25 @@ QUERY id: 0 JOIN TREE TABLE id: 12, alias: __table1, table_name: default.test_table SELECT sum(float64 + 2) + sum(float64 + 3) From test_table; -58 +58.75 SELECT sum(float64 + 2) - sum(float64 + 3) From test_table; -5 SELECT sum(float64 - 2) + sum(float64 - 3) From test_table; -8 +8.75 SELECT sum(float64 - 2) - sum(float64 - 3) From test_table; 5 SELECT sum(2 - float64) - sum(3 - float64) From test_table; -5 SELECT (sum(float64) + 2 * count(float64)) + (sum(float64) + 3 * count(float64)) From test_table; -58 +58.75 SELECT (sum(float64) + 2 * count(float64)) - (sum(float64) + 3 * count(float64)) From test_table; -5 SELECT (sum(float64) - 2 * count(float64)) + (sum(float64) - 3 * count(float64)) From test_table; -8 +8.75 SELECT (sum(float64) - 2 * count(float64)) - (sum(float64) - 3 * count(float64)) From test_table; 5 SELECT (2 * count(float64) - sum(float64)) + (3 * count(float64) - sum(float64)) From test_table; --8 +-8.75 EXPLAIN QUERY TREE (SELECT sum(float64 + 2) + sum(float64 + 3) From test_table); QUERY id: 0 PROJECTION COLUMNS diff --git a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql index 5492d061c12..b6fa097abe9 100644 --- a/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql +++ b/tests/queries/0_stateless/02932_analyzer_rewrite_sum_column_and_constant.sql @@ -25,11 +25,12 @@ CREATE TABLE test_table decimal32 Decimal32(5), ) ENGINE=MergeTree ORDER BY uint64; -INSERT INTO test_table VALUES (1, 1.1, 1.11); -INSERT INTO test_table VALUES (2, 2.2, 2.22); -INSERT INTO test_table VALUES (3, 3.3, 3.33); -INSERT INTO test_table VALUES (4, 4.4, 4.44); -INSERT INTO test_table VALUES (5, 5.5, 5.55); +-- Use Float64 numbers divisible by 1/16 (or some other small power of two), so that their sum doesn't depend on summation order. +INSERT INTO test_table VALUES (1, 1.125, 1.11); +INSERT INTO test_table VALUES (2, 2.250, 2.22); +INSERT INTO test_table VALUES (3, 3.375, 3.33); +INSERT INTO test_table VALUES (4, 4.500, 4.44); +INSERT INTO test_table VALUES (5, 5.625, 5.55); -- { echoOn } SELECT sum(uint64 + 1 AS i) from test_table where i > 0; From b772140514aa47ada3556f1d24712eae734a5645 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Fri, 23 Aug 2024 22:56:24 +0200 Subject: [PATCH 1365/1722] Fix PR comments: check exception in the test --- .../0_stateless/03174_json_compact_with_progress.reference | 2 ++ tests/queries/0_stateless/03174_json_compact_with_progress.sh | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.reference b/tests/queries/0_stateless/03174_json_compact_with_progress.reference index b45f296e5c0..fe51e484bc1 100644 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.reference +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.reference @@ -11,3 +11,5 @@ {"data":["c", "1"]} {"totals": ["", "3"]} {"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":30}} +3 +Value passed to 'throwIf' function is non-zero: diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.sh b/tests/queries/0_stateless/03174_json_compact_with_progress.sh index 8f524cdff76..b15dc7cfdb2 100755 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.sh +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.sh @@ -16,4 +16,8 @@ $CLICKHOUSE_CLIENT -q "SELECT 2;" # Check Totals $CLICKHOUSE_CLIENT -q "SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactWithProgress settings max_block_size=2;" | grep -v --text "progress" | sed -E 's/"elapsed":[0-9]+\.[0-9]+/"elapsed":ELAPSED/g' +$CLICKHOUSE_CLIENT -q "SELECT 3;" +# Check exceptions +${CLICKHOUSE_CURL} -sS "$CLICKHOUSE_URL" -d "SELECT throwIf(number = 15), 1::Int64 as a, '\"' from numbers(100) format JSONCompactWithProgress settings output_format_json_quote_64bit_integers=1, max_block_size=10" | grep "exception" | cut -c42-88 + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_table;" From 0f265ce33d857a9c7446698629b6517b71b4a71d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 23 Aug 2024 23:13:53 +0200 Subject: [PATCH 1366/1722] address comments --- src/Interpreters/InterpreterCreateQuery.cpp | 1 - src/Storages/AlterCommands.cpp | 10 ++++------ src/Storages/ColumnsDescription.cpp | 4 ---- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 4 ++-- src/Storages/Statistics/Statistics.cpp | 16 ++++++++-------- src/Storages/Statistics/Statistics.h | 6 ++++-- src/Storages/StatisticsDescription.cpp | 21 +++++---------------- src/Storages/StatisticsDescription.h | 4 ++-- 9 files changed, 26 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 95143031707..467547e6c9e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -700,7 +700,6 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec); } - column.statistics.column_name = column.name; /// We assign column name here for better exception error message. if (col_decl.statistics_desc) { if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 67b18217767..07bc87b0162 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -705,9 +705,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns); - for (const auto & stats : stats_vec) + for (const auto & [stats_column_name, stats] : stats_vec) { - metadata.columns.modify(stats.column_name, + metadata.columns.modify(stats_column_name, [&](ColumnDescription & column) { column.statistics.merge(stats, column.name, column.type, if_not_exists); }); } } @@ -739,9 +739,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns); - for (const auto & stats : stats_vec) + for (const auto & [stats_column_name, stats] : stats_vec) { - metadata.columns.modify(stats.column_name, + metadata.columns.modify(stats_column_name, [&](ColumnDescription & column) { column.statistics.assign(stats); }); } } @@ -866,8 +866,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) rename_visitor.visit(column_to_modify.default_desc.expression); if (column_to_modify.ttl) rename_visitor.visit(column_to_modify.ttl); - if (column_to_modify.name == column_name && !column_to_modify.statistics.empty()) - column_to_modify.statistics.column_name = rename_to; }); } if (metadata.table_ttl.definition_ast) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0d724245b49..fdc3446aa46 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -209,11 +209,7 @@ void ColumnDescription::readText(ReadBuffer & buf) settings = col_ast->settings->as().changes; if (col_ast->statistics_desc) - { statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type); - /// every column has name `x` here, so we have to set the name manually. - statistics.column_name = name; - } } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ce06adf110c..0d34eb7f630 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -62,7 +62,7 @@ static ColumnsStatistics getStatisticsForColumns( const auto * desc = all_columns.tryGet(column.name); if (desc && !desc->statistics.empty()) { - auto statistics = MergeTreeStatisticsFactory::instance().get(desc->statistics); + auto statistics = MergeTreeStatisticsFactory::instance().get(*desc); all_statistics.push_back(std::move(statistics)); } } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b9b5333a61c..1119ca324d6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -546,7 +546,7 @@ static std::set getStatisticsToRecalculate(const StorageMet { if (!col_desc.statistics.empty() && materialized_stats.contains(col_desc.name)) { - stats_to_recalc.insert(stats_factory.get(col_desc.statistics)); + stats_to_recalc.insert(stats_factory.get(col_desc)); } } return stats_to_recalc; @@ -1530,7 +1530,7 @@ private: if (ctx->materialized_statistics.contains(col.name)) { - stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col.statistics)); + stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col)); } else { diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index fd686c5f0aa..6372c804e0e 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -58,8 +58,8 @@ IStatistics::IStatistics(const SingleStatisticsDescription & stat_) { } -ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_) - : stats_desc(stats_desc_) +ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_) + : stats_desc(stats_desc_), column_name(column_name_) { } @@ -176,7 +176,7 @@ String ColumnStatistics::getFileName() const const String & ColumnStatistics::columnName() const { - return stats_desc.column_name; + return column_name; } UInt64 ColumnStatistics::rowCount() const @@ -227,15 +227,15 @@ void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & st } } -ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescription & stats) const +ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnDescription & column_desc) const { - ColumnStatisticsPtr column_stat = std::make_shared(stats); - for (const auto & [type, desc] : stats.types_to_desc) + ColumnStatisticsPtr column_stat = std::make_shared(column_desc.statistics, column_desc.name); + for (const auto & [type, desc] : column_desc.statistics.types_to_desc) { auto it = creators.find(type); if (it == creators.end()) throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type); - auto stat_ptr = (it->second)(desc, stats.data_type); + auto stat_ptr = (it->second)(desc, column_desc.type); column_stat->stats[type] = stat_ptr; } return column_stat; @@ -246,7 +246,7 @@ ColumnsStatistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription & ColumnsStatistics result; for (const auto & col : columns) if (!col.statistics.empty()) - result.push_back(get(col.statistics)); + result.push_back(get(col)); return result; } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 2a30c0de315..98666ed73df 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -54,7 +54,7 @@ using StatisticsPtr = std::shared_ptr; class ColumnStatistics { public: - explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_); + explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_); void serialize(WriteBuffer & buf); void deserialize(ReadBuffer & buf); @@ -73,10 +73,12 @@ public: private: friend class MergeTreeStatisticsFactory; ColumnStatisticsDescription stats_desc; + String column_name; std::map stats; UInt64 rows = 0; /// the number of rows in the column }; +struct ColumnDescription; class ColumnsDescription; using ColumnStatisticsPtr = std::shared_ptr; using ColumnsStatistics = std::vector; @@ -91,7 +93,7 @@ public: using Validator = std::function; using Creator = std::function; - ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const; + ColumnStatisticsPtr get(const ColumnDescription & column_desc) const; ColumnsStatistics getMany(const ColumnsDescription & columns) const; void registerValidator(StatisticsType type, Validator validator); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index acf600dd6f7..64634124758 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -96,16 +96,13 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe { chassert(merging_column_type); - if (column_name.empty()) - column_name = merging_column_name; - data_type = merging_column_type; for (const auto & [stats_type, stats_desc]: other.types_to_desc) { if (!if_not_exists && types_to_desc.contains(stats_type)) { - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, merging_column_name); } else if (!types_to_desc.contains(stats_type)) types_to_desc.emplace(stats_type, stats_desc); @@ -114,12 +111,6 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other) { - /// If the statistics is empty, it's possible that we have not assign a column_name. - if (empty() && column_name == "") - column_name = other.column_name; - if (other.column_name != column_name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", other.column_name, column_name); - types_to_desc = other.types_to_desc; data_type = other.data_type; } @@ -129,7 +120,7 @@ void ColumnStatisticsDescription::clear() types_to_desc.clear(); } -std::vector ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) +std::vector> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns) { const auto * stat_definition_ast = definition_ast->as(); if (!stat_definition_ast) @@ -147,7 +138,7 @@ std::vector ColumnStatisticsDescription::fromAST(co statistics_types.emplace(stat.type, stat); } - std::vector result; + std::vector> result; result.reserve(stat_definition_ast->columns->children.size()); for (const auto & column_ast : stat_definition_ast->columns->children) @@ -159,10 +150,9 @@ std::vector ColumnStatisticsDescription::fromAST(co throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name); const auto & column = columns.getPhysical(physical_column_name); - stats.column_name = column.name; stats.data_type = column.type; stats.types_to_desc = statistics_types; - result.push_back(stats); + result.emplace_back(physical_column_name, stats); } if (result.empty()) @@ -177,14 +167,13 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c if (stat_type_list_ast->children.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column)); ColumnStatisticsDescription stats; - stats.column_name = column.name; for (const auto & ast : stat_type_list_ast->children) { const auto & stat_type = ast->as().name; SingleStatisticsDescription stat(stringToStatisticsType(Poco::toLower(stat_type)), ast->clone()); if (stats.types_to_desc.contains(stat.type)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", column.name, stat_type); stats.types_to_desc.emplace(stat.type, std::move(stat)); } stats.data_type = data_type; diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 03b8fb0d583..46927f1418c 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -55,12 +55,12 @@ struct ColumnStatisticsDescription ASTPtr getAST() const; - static std::vector fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); + /// get a vector of pair + static std::vector> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type); using StatisticsTypeDescMap = std::map; StatisticsTypeDescMap types_to_desc; - String column_name; DataTypePtr data_type; }; From 6fb8f2b4ee10a95104bf6f8880471d24d39095dc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 23 Aug 2024 23:19:03 +0200 Subject: [PATCH 1367/1722] fix black --- .../test_manipulate_statistics/test.py | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index ab5559e18fa..aff943e4d20 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,13 +6,17 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=True, - macros={"replica": "a", "shard": "shard1"} + "node1", + user_configs=["config/config.xml"], + with_zookeeper=True, + macros={"replica": "a", "shard": "shard1"}, ) node2 = cluster.add_instance( - "node2", user_configs=["config/config.xml"], with_zookeeper=True, - macros={"replica": "b", "shard": "shard1"} + "node2", + user_configs=["config/config.xml"], + with_zookeeper=True, + macros={"replica": "b", "shard": "shard1"}, ) @@ -188,8 +192,14 @@ def test_replicated_table_ddl(started_cluster): def test_replicated_db(started_cluster): - node1.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") - node2.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')") - node1.query("CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()") + node1.query( + "CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')" + ) + node2.query( + "CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')" + ) + node1.query( + "CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()" + ) node2.query("ALTER TABLE test.test_stats MODIFY COLUMN b Float64") node2.query("ALTER TABLE test.test_stats MODIFY STATISTICS b TYPE tdigest") From 0a35b111ffb34f3d6a8a9e9bfa712b57b722c447 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 20:03:38 +0000 Subject: [PATCH 1368/1722] fix test 03221_mutation_analyzer_skip_part --- .../03221_mutation_analyzer_skip_part.sh | 46 +++++++++++++++++++ .../03221_mutation_analyzer_skip_part.sql | 21 --------- 2 files changed, 46 insertions(+), 21 deletions(-) create mode 100755 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh delete mode 100644 tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh new file mode 100755 index 00000000000..03fd15f54e2 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash +# Tags: no-random-settings, no-random-merge-tree-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query " + DROP TABLE IF EXISTS t_mutate_skip_part; + + CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) + ENGINE = MergeTree ORDER BY id PARTITION BY key + SETTINGS min_bytes_for_wide_part = 0; + + INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000); + INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000); + + SET mutations_sync = 2; + ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1; + ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0; +" + +# Mutation query may return before the entry is added to part log. +# So, we may have to retry the flush of logs until all entries are actually flushed. +for _ in {1..10}; do + ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + res=$(${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.part_log WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart'") + + if [[ $res -eq 4 ]]; then + break + fi + + sleep 2.0 +done + +${CLICKHOUSE_CLIENT} --query " + SYSTEM FLUSH LOGS; + + -- If part is skipped in mutation and hardlinked then read_rows must be 0. + SELECT part_name, read_rows + FROM system.part_log + WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart' + ORDER BY part_name; + + DROP TABLE IF EXISTS t_mutate_skip_part; +" diff --git a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql b/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql deleted file mode 100644 index bf9a10e2af4..00000000000 --- a/tests/queries/0_stateless/03221_mutation_analyzer_skip_part.sql +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE IF EXISTS t_mutate_skip_part; - -CREATE TABLE t_mutate_skip_part (key UInt64, id UInt64, v1 UInt64, v2 UInt64) ENGINE = MergeTree ORDER BY id PARTITION BY key; - -INSERT INTO t_mutate_skip_part SELECT 1, number, number, number FROM numbers(10000); -INSERT INTO t_mutate_skip_part SELECT 2, number, number, number FROM numbers(10000); - -SET mutations_sync = 2; - -ALTER TABLE t_mutate_skip_part UPDATE v1 = 1000 WHERE key = 1; -ALTER TABLE t_mutate_skip_part DELETE WHERE key = 2 AND v2 % 10 = 0; - -SYSTEM FLUSH LOGS; - --- If part is skipped in mutation and hardlinked then read_rows must be 0. -SELECT part_name, read_rows -FROM system.part_log -WHERE database = currentDatabase() AND table = 't_mutate_skip_part' AND event_type = 'MutatePart' -ORDER BY part_name; - -DROP TABLE IF EXISTS t_mutate_skip_part; From 080b8f74be186738813ca9d9e12ed3e327129c33 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Fri, 23 Aug 2024 15:50:56 -0700 Subject: [PATCH 1369/1722] Update README.md Add Austin meetup --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 546f08afd3d..ba212852ea8 100644 --- a/README.md +++ b/README.md @@ -54,6 +54,7 @@ Other upcoming meetups * [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 * [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 * [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 +* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 * [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 ## Recent Recordings From 5fe151529ab58112f8fa8491d2bfff24562ff624 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 24 Aug 2024 07:33:18 +0200 Subject: [PATCH 1370/1722] fix flacky although that is not actually flacky --- tests/integration/test_manipulate_statistics/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index aff943e4d20..3a1c5ad5b96 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -135,8 +135,8 @@ def test_single_node_normal(started_cluster): def test_replicated_table_ddl(started_cluster): - node1.query("DROP TABLE IF EXISTS test_stat") - node2.query("DROP TABLE IF EXISTS test_stat") + node1.query("DROP TABLE IF EXISTS test_stat SYNC") + node2.query("DROP TABLE IF EXISTS test_stat SYNC") node1.query( """ @@ -192,6 +192,8 @@ def test_replicated_table_ddl(started_cluster): def test_replicated_db(started_cluster): + node1.query("DROP DATABASE IF EXISTS test SYNC") + node2.query("DROP DATABASE IF EXISTS test SYNC") node1.query( "CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')" ) From e2aa953e700bfbabbfe69a5749f4d2806bd3610f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 24 Aug 2024 20:45:10 +0800 Subject: [PATCH 1371/1722] Fix empty tuple in array --- src/Functions/array/arrayElement.cpp | 3 +++ tests/queries/0_stateless/03229_empty_tuple_in_array.reference | 1 + tests/queries/0_stateless/03229_empty_tuple_in_array.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03229_empty_tuple_in_array.reference create mode 100644 tests/queries/0_stateless/03229_empty_tuple_in_array.sql diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index 81f3f97979b..d0b2b49cc1c 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1598,6 +1598,9 @@ ColumnPtr FunctionArrayElement::executeTuple(const ColumnsWithTypeAndName & argu const auto & tuple_columns = col_nested->getColumns(); size_t tuple_size = tuple_columns.size(); + if (tuple_size == 0) + return ColumnTuple::create(input_rows_count); + const DataTypes & tuple_types = typeid_cast( *typeid_cast(*arguments[0].type).getNestedType()).getElements(); diff --git a/tests/queries/0_stateless/03229_empty_tuple_in_array.reference b/tests/queries/0_stateless/03229_empty_tuple_in_array.reference new file mode 100644 index 00000000000..6a452c185a8 --- /dev/null +++ b/tests/queries/0_stateless/03229_empty_tuple_in_array.reference @@ -0,0 +1 @@ +() diff --git a/tests/queries/0_stateless/03229_empty_tuple_in_array.sql b/tests/queries/0_stateless/03229_empty_tuple_in_array.sql new file mode 100644 index 00000000000..09ba3595a5a --- /dev/null +++ b/tests/queries/0_stateless/03229_empty_tuple_in_array.sql @@ -0,0 +1 @@ +select [()][0]; From 78c175225b9b4c929ed918e718351c18a166458a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 22 Aug 2024 14:50:10 +0000 Subject: [PATCH 1372/1722] Done --- contrib/replxx | 2 +- src/Client/ReplxxLineReader.cpp | 13 +++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/contrib/replxx b/contrib/replxx index 5d04501f93a..5f696c6eb9a 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 5d04501f93a4fb7f0bb8b73b8f614bc986f9e25b +Subproject commit 5f696c6eb9a88eb9784e8ff1d68bd5f70285dcc5 diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index 78ae6c5eb15..37ceb471e5b 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -299,13 +299,14 @@ ReplxxLineReader::ReplxxLineReader( Patterns delimiters_, const char word_break_characters_[], replxx::Replxx::highlighter_callback_t highlighter_, - [[ maybe_unused ]] std::istream & input_stream_, - [[ maybe_unused ]] std::ostream & output_stream_, - [[ maybe_unused ]] int in_fd_, - [[ maybe_unused ]] int out_fd_, - [[ maybe_unused ]] int err_fd_ + std::istream & input_stream_, + std::ostream & output_stream_, + int in_fd_, + int out_fd_, + int err_fd_ ) : LineReader(history_file_path_, multiline_, std::move(extenders_), std::move(delimiters_), input_stream_, output_stream_, in_fd_) + , rx(input_stream_, output_stream_, in_fd_, out_fd_, err_fd_) , highlighter(std::move(highlighter_)) , word_break_characters(word_break_characters_) , editor(getEditor()) @@ -516,7 +517,7 @@ void ReplxxLineReader::addToHistory(const String & line) rx.history_add(line); // flush changes to the disk - if (!rx.history_save(history_file_path)) + if (history_file_fd >= 0 && !rx.history_save(history_file_path)) rx.print("Saving history failed: %s\n", errnoToString().c_str()); if (history_file_fd >= 0 && locked && 0 != flock(history_file_fd, LOCK_UN)) From 01523cce2a4ba21c9855ab4eb1398986cf66c64b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Aug 2024 12:14:40 +0000 Subject: [PATCH 1373/1722] Bump replxx --- contrib/replxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/replxx b/contrib/replxx index 5f696c6eb9a..711c18e7f4d 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 5f696c6eb9a88eb9784e8ff1d68bd5f70285dcc5 +Subproject commit 711c18e7f4d951255aa8b0851e5a55d5a5fb0ddb From 385c8127cf4b7018a964705d0bdcaf17bdf494e4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 23 Aug 2024 17:25:34 +0200 Subject: [PATCH 1374/1722] Fix FreeBSD build --- cmake/freebsd/toolchain-x86_64.cmake | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmake/freebsd/toolchain-x86_64.cmake b/cmake/freebsd/toolchain-x86_64.cmake index 4635880b4a6..4d814693b39 100644 --- a/cmake/freebsd/toolchain-x86_64.cmake +++ b/cmake/freebsd/toolchain-x86_64.cmake @@ -8,4 +8,7 @@ set (CMAKE_CXX_COMPILER_TARGET "x86_64-pc-freebsd11") set (CMAKE_ASM_COMPILER_TARGET "x86_64-pc-freebsd11") set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/freebsd-x86_64") +# dprintf is used in a patched version of replxx +add_compile_definitions(_WITH_DPRINTF) + set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake From d16388000497251856f62e8ac67ade58c29f8e85 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 25 Aug 2024 00:11:31 -0400 Subject: [PATCH 1375/1722] process possible SSL error on connection reset --- base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp index 4873d259ae5..14c877b30af 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp @@ -311,6 +311,14 @@ int SecureSocketImpl::sendBytes(const void* buffer, int length, int flags) while (mustRetry(rc, remaining_time)); if (rc <= 0) { + // At this stage we still can have last not yet recieved SSL message containing SSL error + // so make a read to force SSL to process possible SSL error + if (SSL_get_error(_pSSL, rc) == SSL_ERROR_SYSCALL && SocketImpl::lastError() == POCO_ECONNRESET) + { + char c = 0; + SSL_read(_pSSL, &c, 1); + } + rc = handleError(rc); if (rc == 0) throw SSLConnectionUnexpectedlyClosedException(); } From f7cc3e9c59947af5b753b154f5b1b59d26fe67d4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Sun, 25 Aug 2024 00:13:12 -0400 Subject: [PATCH 1376/1722] postpone SSL handshake --- src/Client/Connection.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index e89bd7a2bf5..da6e5baa3ad 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -145,6 +145,9 @@ void Connection::connect(const ConnectionTimeouts & timeouts) /// work we need to pass host name separately. It will be send into TLS Hello packet to let /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). static_cast(socket.get())->setPeerHostName(host); + /// we want to postpone SSL handshake until first read or write operation + /// so any errors during negotiation would be properly processed + static_cast(socket.get())->setLazyHandshake(true); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "tcp_secure protocol is disabled because poco library was built without NetSSL support."); #endif From e4aceed36aa6a5dcc2b37d3a184af57c06cb202f Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 25 Aug 2024 07:48:42 +0000 Subject: [PATCH 1377/1722] Add virtual column _headers for url table engine --- base/poco/Net/include/Poco/Net/HTTPResponse.h | 4 ++ base/poco/Net/src/HTTPResponse.cpp | 9 +++++ src/IO/ReadWriteBufferFromHTTP.cpp | 14 +++++++ src/IO/ReadWriteBufferFromHTTP.h | 5 +++ src/Storages/StorageURL.cpp | 40 ++++++++++++++++--- src/Storages/StorageURL.h | 4 ++ ...3228_url_engine_response_headers.reference | 2 + .../03228_url_engine_response_headers.sql | 7 ++++ 8 files changed, 80 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03228_url_engine_response_headers.reference create mode 100644 tests/queries/0_stateless/03228_url_engine_response_headers.sql diff --git a/base/poco/Net/include/Poco/Net/HTTPResponse.h b/base/poco/Net/include/Poco/Net/HTTPResponse.h index 3c444c3d38c..623dfb783f3 100644 --- a/base/poco/Net/include/Poco/Net/HTTPResponse.h +++ b/base/poco/Net/include/Poco/Net/HTTPResponse.h @@ -18,7 +18,9 @@ #define Net_HTTPResponse_INCLUDED +#include #include + #include "Poco/Net/HTTPCookie.h" #include "Poco/Net/HTTPMessage.h" #include "Poco/Net/Net.h" @@ -180,6 +182,8 @@ namespace Net /// May throw an exception in case of a malformed /// Set-Cookie header. + void getHeaders(std::map & headers) const; + void write(std::ostream & ostr) const; /// Writes the HTTP response to the given /// output stream. diff --git a/base/poco/Net/src/HTTPResponse.cpp b/base/poco/Net/src/HTTPResponse.cpp index e0017321595..4d58d05a43e 100644 --- a/base/poco/Net/src/HTTPResponse.cpp +++ b/base/poco/Net/src/HTTPResponse.cpp @@ -209,6 +209,15 @@ void HTTPResponse::getCookies(std::vector& cookies) const } } +void HTTPResponse::getHeaders(std::map & headers) const +{ + headers.clear(); + for (const auto & it : *this) + { + headers.emplace(it.first, it.second); + } +} + void HTTPResponse::write(std::ostream& ostr) const { diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index a7bc0d4845c..826862bb5b3 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -443,6 +443,7 @@ std::unique_ptr ReadWriteBufferFromHTTP::initialize() } response.getCookies(cookies); + response.getHeaders(response_headers); content_encoding = response.get("Content-Encoding", ""); // Remember file size. It'll be used to report eof in next nextImpl() call. @@ -680,6 +681,19 @@ std::string ReadWriteBufferFromHTTP::getResponseCookie(const std::string & name, return def; } +Map ReadWriteBufferFromHTTP::getResponseHeaders() const +{ + Map map; + for (const auto & header : response_headers) + { + Tuple elem; + elem.emplace_back(header.first); + elem.emplace_back(header.second); + map.emplace_back(elem); + } + return map; +} + void ReadWriteBufferFromHTTP::setNextCallback(NextCallback next_callback_) { next_callback = next_callback_; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 1c9bda53008..cca22c64317 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -90,6 +90,9 @@ private: std::unique_ptr impl; std::vector cookies; + + std::map response_headers; + HTTPHeaderEntries http_header_entries; std::function next_callback; @@ -187,6 +190,8 @@ public: HTTPFileInfo getFileInfo(); static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); + + Map getResponseHeaders() const; }; using ReadWriteBufferFromHTTPPtr = std::unique_ptr; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6442891cf23..67b6ec4c9bf 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -44,10 +44,11 @@ #include #include +#include +#include +#include #include #include -#include -#include namespace ProfileEvents { @@ -167,7 +168,19 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings)); + auto virtual_columns_desc = VirtualColumnUtils::getVirtualsForFileLikeStorage( + storage_metadata.getColumns(), context_, getSampleURI(uri, context_), format_settings); + if (!storage_metadata.getColumns().has("_headers")) + { + virtual_columns_desc.addEphemeral( + "_headers", + std::make_shared( + std::make_shared(std::make_shared()), + std::make_shared(std::make_shared())), + ""); + } + + setVirtuals(virtual_columns_desc); } @@ -292,11 +305,13 @@ StorageURLSource::StorageURLSource( const URIParams & params, bool glob_url, bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false), WithContext(context_) + : SourceWithKeyCondition(info.source_header, false) + , WithContext(context_) , name(std::move(name_)) , columns_description(info.columns_description) , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) + , need_headers_virtual_column(info.requested_virtual_columns.contains("_headers")) + , requested_virtual_columns(info.requested_virtual_columns.eraseNames({"_headers"})) , block_for_format(info.format_header) , uri_iterator(uri_iterator_) , format(format_) @@ -436,6 +451,20 @@ Chunk StorageURLSource::generate() .path = curr_uri.getPath(), .size = current_file_size, }, getContext(), columns_description); + if (need_headers_virtual_column) + { + if (!http_response_headers_initialized) + { + http_response_headers = dynamic_cast(read_buf.get())->getResponseHeaders(); + http_response_headers_initialized = true; + } + + auto type = std::make_shared( + std::make_shared(std::make_shared()), + std::make_shared(std::make_shared())); + + chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)); + } return chunk; } @@ -446,6 +475,7 @@ Chunk StorageURLSource::generate() reader.reset(); input_format.reset(); read_buf.reset(); + http_response_headers_initialized = false; total_rows_in_file = 0; } return {}; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 19daf843431..6f1d544629a 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -220,6 +220,7 @@ private: String name; ColumnsDescription columns_description; NamesAndTypesList requested_columns; + bool need_headers_virtual_column; NamesAndTypesList requested_virtual_columns; Block block_for_format; std::shared_ptr uri_iterator; @@ -233,6 +234,9 @@ private: Poco::Net::HTTPBasicCredentials credentials; + Map http_response_headers; + bool http_response_headers_initialized = false; + std::unique_ptr read_buf; std::shared_ptr input_format; std::unique_ptr pipeline; diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.reference b/tests/queries/0_stateless/03228_url_engine_response_headers.reference new file mode 100644 index 00000000000..f28952972b8 --- /dev/null +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.reference @@ -0,0 +1,2 @@ +Map(LowCardinality(String), LowCardinality(String)) +1 1 diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql new file mode 100644 index 00000000000..ff8e47611f4 --- /dev/null +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -0,0 +1,7 @@ +SELECT toTypeName(_headers) +FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); + +SELECT + *, + mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'] +FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); From e1e692a968e5ccce64c43a90478903648f95bb27 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 25 Aug 2024 08:00:16 +0000 Subject: [PATCH 1378/1722] Fix and update docs --- docs/en/engines/table-engines/special/url.md | 1 + docs/en/sql-reference/table-functions/url.md | 1 + src/Storages/StorageURL.cpp | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index c906830d0e9..bbc28daef48 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -109,6 +109,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +- `_headers` - HTTP response headers. Type: `Map(LowCardinality(String), LowCardinality(String))`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index b4027594e7c..7a354ea0364 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -54,6 +54,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. - `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. +- `_headers` - HTTP response headers. Type: `Map(LowCardinality(String), LowCardinality(String))`. ## Hive-style partitioning {#hive-style-partitioning} diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f7c3147d091..3fb00886540 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -465,7 +465,7 @@ Chunk StorageURLSource::generate() std::make_shared(std::make_shared()), std::make_shared(std::make_shared())); - chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)); + chunk.addColumn(type->createColumnConst(chunk.getNumRows(), http_response_headers)->convertToFullColumnIfConst()); } return chunk; } From 06f0267ed142d76109b2d6d48d3c413bdaf0fc17 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 25 Aug 2024 11:27:43 +0200 Subject: [PATCH 1379/1722] Do not show statistics if exception happens --- ...JSONCompactWithProgressRowOutputFormat.cpp | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index 0814e5f45d6..cb3398196f4 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -117,18 +117,16 @@ void JSONCompactWithProgressRowOutputFormat::writeProgress() void JSONCompactWithProgressRowOutputFormat::finalizeImpl() { - JSONUtils::writeCompactAdditionalInfo( - row_count, - statistics.rows_before_limit, - statistics.applied_limit, - statistics.watch, - statistics.progress, - settings.write_statistics && exception_message.empty(), - *ostr); - - if (!exception_message.empty()) - { - writeCString("\n", *ostr); + if (exception_message.empty()) { + JSONUtils::writeCompactAdditionalInfo( + row_count, + statistics.rows_before_limit, + statistics.applied_limit, + statistics.watch, + statistics.progress, + settings.write_statistics && exception_message.empty(), + *ostr); + } else { JSONUtils::writeCompactObjectStart(*ostr); JSONUtils::writeException(exception_message, *ostr, settings, 0); JSONUtils::writeCompactObjectEnd(*ostr); From 58aab9b8c789090834dd1db7bb00568db7b66cac Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 25 Aug 2024 11:29:07 +0200 Subject: [PATCH 1380/1722] Remove wrong bracket in the meta section --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 2 +- .../0_stateless/03174_json_compact_with_progress.reference | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index cb3398196f4..b5cbd6fb232 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -19,7 +19,7 @@ void JSONCompactWithProgressRowOutputFormat::writePrefix() JSONUtils::writeCompactObjectStart(*ostr); JSONUtils::writeCompactMetadata(names, types, settings, *ostr); JSONUtils::writeCompactObjectEnd(*ostr); - writeCString("}\n", *ostr); + writeCString("\n", *ostr); } void JSONCompactWithProgressRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) diff --git a/tests/queries/0_stateless/03174_json_compact_with_progress.reference b/tests/queries/0_stateless/03174_json_compact_with_progress.reference index fe51e484bc1..cdbe7cfcb3e 100644 --- a/tests/queries/0_stateless/03174_json_compact_with_progress.reference +++ b/tests/queries/0_stateless/03174_json_compact_with_progress.reference @@ -1,11 +1,11 @@ 1 -{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}]}} +{"meta": [{"name":"value", "type":"UInt8"}, {"name":"name", "type":"String"}]} {"data":[1, "a"]} {"data":[2, "b"]} {"data":[3, "c"]} {"statistics": {"rows":3, "elapsed":ELAPSED, "rows_read":3, "bytes_read":33}} 2 -{"meta": [{"name":"name", "type":"String"}, {"name":"c", "type":"UInt64"}]}} +{"meta": [{"name":"name", "type":"String"}, {"name":"c", "type":"UInt64"}]} {"data":["a", "1"]} {"data":["b", "1"]} {"data":["c", "1"]} From af3f600b5cca7cadb304e0f69fc281b7015deca9 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 25 Aug 2024 11:35:35 +0200 Subject: [PATCH 1381/1722] Fix commas in the statistics section --- src/Formats/JSONUtils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 0918658cf6f..6c2c6fc3700 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -563,17 +563,17 @@ namespace JSONUtils writeCompactObjectStart(out, 0, "statistics"); writeTitle("rows", out, 0, ""); writeIntText(rows, out); - writeFieldCompactDelimiter(out); if (applied_limit) { + writeFieldCompactDelimiter(out); writeTitle("rows_before_limit_at_least", out, 0, ""); writeIntText(rows_before_limit, out); - writeFieldCompactDelimiter(out); } if (write_statistics) { + writeFieldCompactDelimiter(out); writeTitle("elapsed", out, 0, ""); writeText(watch.elapsedSeconds(), out); writeFieldCompactDelimiter(out); From 0e4f2b17cc75a1f09994ad583df69fd067e582f8 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 25 Aug 2024 11:38:29 +0200 Subject: [PATCH 1382/1722] Remove unnecessary check --- .../Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index b5cbd6fb232..d1d7355724f 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -124,7 +124,7 @@ void JSONCompactWithProgressRowOutputFormat::finalizeImpl() statistics.applied_limit, statistics.watch, statistics.progress, - settings.write_statistics && exception_message.empty(), + settings.write_statistics, *ostr); } else { JSONUtils::writeCompactObjectStart(*ostr); From 312d32aaf972e01a88fceb2fea60fce424bfc49f Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Sun, 25 Aug 2024 12:14:18 +0200 Subject: [PATCH 1383/1722] Fix formatting --- .../Impl/JSONCompactWithProgressRowOutputFormat.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp index d1d7355724f..e90864ecdf3 100644 --- a/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactWithProgressRowOutputFormat.cpp @@ -117,7 +117,8 @@ void JSONCompactWithProgressRowOutputFormat::writeProgress() void JSONCompactWithProgressRowOutputFormat::finalizeImpl() { - if (exception_message.empty()) { + if (exception_message.empty()) + { JSONUtils::writeCompactAdditionalInfo( row_count, statistics.rows_before_limit, @@ -126,7 +127,9 @@ void JSONCompactWithProgressRowOutputFormat::finalizeImpl() statistics.progress, settings.write_statistics, *ostr); - } else { + } + else + { JSONUtils::writeCompactObjectStart(*ostr); JSONUtils::writeException(exception_message, *ostr, settings, 0); JSONUtils::writeCompactObjectEnd(*ostr); From f38f95a144fa8840bc19647af3be9aa83a505196 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 25 Aug 2024 14:26:21 +0200 Subject: [PATCH 1384/1722] Update base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp --- base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp index 14c877b30af..eaf267d8a8b 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp @@ -311,7 +311,7 @@ int SecureSocketImpl::sendBytes(const void* buffer, int length, int flags) while (mustRetry(rc, remaining_time)); if (rc <= 0) { - // At this stage we still can have last not yet recieved SSL message containing SSL error + // At this stage we still can have last not yet received SSL message containing SSL error // so make a read to force SSL to process possible SSL error if (SSL_get_error(_pSSL, rc) == SSL_ERROR_SYSCALL && SocketImpl::lastError() == POCO_ECONNRESET) { From 9bcafbc90f66602749d3a66cb3631ebb50ce4483 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 10:46:18 +0800 Subject: [PATCH 1385/1722] improve_upper_lower --- src/Functions/LowerUpperUTF8Impl.h | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 490b53b0f4a..0b36caad817 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -4,20 +4,21 @@ #if USE_ICU -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; } template From 6c0102e971499bd44e4efe437f5e54028ba0ff53 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 16:28:56 +0800 Subject: [PATCH 1386/1722] fix failed uts --- src/Functions/LowerUpperUTF8Impl.h | 59 ++++++++----------- .../0_stateless/01431_utf8_ubsan.reference | 4 +- ...71_lower_upper_utf8_row_overlaps.reference | 4 +- 3 files changed, 28 insertions(+), 39 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 0b36caad817..5ba6c793d42 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -31,7 +31,7 @@ struct LowerUpperUTF8Impl ColumnString::Offsets & res_offsets, size_t input_rows_count) { - if (data.empty()) + if (input_rows_count == 0) return; bool all_ascii = isAllASCII(data.data(), data.size()); @@ -41,67 +41,56 @@ struct LowerUpperUTF8Impl return; } + res_data.resize(data.size()); + res_offsets.resize_exact(input_rows_count); UErrorCode error_code = U_ZERO_ERROR; - UCaseMap * csm = ucasemap_open(nullptr, 0, &error_code); + UCaseMap * case_map = ucasemap_open("", U_FOLD_CASE_DEFAULT, &error_code); if (U_FAILURE(error_code)) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Call ucasemap_open error:{}", u_errorName(error_code)); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Error calling ucasemap_open: {}", u_errorName(error_code)); - // String output; size_t curr_offset = 0; - res_data.resize(data.size()); - res_offsets.resize_exact(offsets.size()); - for (size_t i = 0; i < input_rows_count; ++i) + for (size_t row_i = 0; row_i < input_rows_count; ++row_i) { - const auto * data_start = reinterpret_cast(&data[offsets[i - 1]]); - size_t size = offsets[i] - offsets[i - 1] - 1; + const auto * src = reinterpret_cast(&data[offsets[row_i - 1]]); + size_t src_size = offsets[row_i] - offsets[row_i - 1] - 1; - int32_t out_size; + int32_t dst_size; if constexpr (upper) - out_size = ucasemap_utf8ToUpper( - csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); + dst_size = ucasemap_utf8ToUpper( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); else - out_size = ucasemap_utf8ToLower( - csm, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, data_start, size, &error_code); + dst_size = ucasemap_utf8ToLower( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); // std::cout << size << ":" << out_size << ":" << static_cast(res_data[curr_offset + out_size - 1]) << ":" << error_code // << std::endl; if (error_code == U_BUFFER_OVERFLOW_ERROR) { - size_t new_size = curr_offset + out_size + 1; + size_t new_size = curr_offset + dst_size + 1; res_data.resize(new_size); error_code = U_ZERO_ERROR; if constexpr (upper) - out_size = ucasemap_utf8ToUpper( - csm, - reinterpret_cast(&res_data[curr_offset]), - res_data.size() - curr_offset, - data_start, - size, - &error_code); + dst_size = ucasemap_utf8ToUpper( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); else - out_size = ucasemap_utf8ToLower( - csm, - reinterpret_cast(&res_data[curr_offset]), - res_data.size() - curr_offset, - data_start, - size, - &error_code); + dst_size = ucasemap_utf8ToLower( + case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); } if (error_code != U_ZERO_ERROR) throw DB::Exception( ErrorCodes::LOGICAL_ERROR, - "Call {} error:{} input:{} input_size:{}", + "Error calling {}: {} input: {} input_size: {}", upper ? "ucasemap_utf8ToUpper" : "ucasemap_utf8ToLower", u_errorName(error_code), - std::string_view(data_start, size), - size); + std::string_view(src, src_size), + src_size); - res_data[curr_offset + out_size] = 0; - curr_offset += out_size + 1; - res_offsets[i] = curr_offset; + res_data[curr_offset + dst_size] = 0; + curr_offset += dst_size + 1; + res_offsets[row_i] = curr_offset; } res_data.resize(curr_offset); diff --git a/tests/queries/0_stateless/01431_utf8_ubsan.reference b/tests/queries/0_stateless/01431_utf8_ubsan.reference index dc785e57851..c98c950d535 100644 --- a/tests/queries/0_stateless/01431_utf8_ubsan.reference +++ b/tests/queries/0_stateless/01431_utf8_ubsan.reference @@ -1,2 +1,2 @@ -EFBFBD -EFBFBD +FF +FF diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index deabef61a88..a3bac432482 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo -- MONOGRAM FOR YANG with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV; -1,"�","�" +1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -1,"�","�" +1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" -- NOTE: regression test for introduced bug -- https://github.com/ClickHouse/ClickHouse/issues/42756 From 53ace69ede72f106dc60f504e66a25ae402af6dc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 16:47:47 +0800 Subject: [PATCH 1387/1722] fix failed performance tests --- src/Functions/LowerUpperUTF8Impl.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 5ba6c793d42..7d60bd54d2f 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -62,10 +62,8 @@ struct LowerUpperUTF8Impl else dst_size = ucasemap_utf8ToLower( case_map, reinterpret_cast(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code); - // std::cout << size << ":" << out_size << ":" << static_cast(res_data[curr_offset + out_size - 1]) << ":" << error_code - // << std::endl; - if (error_code == U_BUFFER_OVERFLOW_ERROR) + if (error_code == U_BUFFER_OVERFLOW_ERROR || error_code == U_STRING_NOT_TERMINATED_WARNING) { size_t new_size = curr_offset + dst_size + 1; res_data.resize(new_size); From 6e584dd541527e9171e245b5a95bd6c5bc388deb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 26 Aug 2024 11:33:08 +0200 Subject: [PATCH 1388/1722] Fix test --- tests/integration/test_storage_delta/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d000875d8d1..0aa846478ea 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -795,6 +795,7 @@ def test_complex_types(started_cluster): endpoint_url = f"http://{started_cluster.minio_ip}:{started_cluster.minio_port}" aws_access_key_id = "minio" aws_secret_access_key = "minio123" + table_name = randomize_table_name("test_complex_types") storage_options = { "AWS_ENDPOINT_URL": endpoint_url, @@ -803,23 +804,23 @@ def test_complex_types(started_cluster): "AWS_ALLOW_HTTP": "true", "AWS_S3_ALLOW_UNSAFE_RENAME": "true", } - path = f"s3://root/testtable" + path = f"s3://root/{table_name}" table = pa.Table.from_arrays(data, schema=schema) write_deltalake(path, table, storage_options=storage_options) assert "1\n2\n3\n" in node.query( - f"SELECT id FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + f"SELECT id FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) assert ( "('123 Elm St','Springfield','IL')\n('456 Maple St','Shelbyville','IL')\n('789 Oak St','Ogdenville','IL')" in node.query( - f"SELECT address FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + f"SELECT address FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) ) assert ( "{'key1':'value1','key2':'value2'}\n{'key1':'value3','key2':'value4'}\n{'key1':'value5','key2':'value6'}" in node.query( - f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/testtable' , 'minio', 'minio123')" + f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) ) From 7d8942527355dedf19dc03246cb80ac5fc9e171a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 26 Aug 2024 13:15:26 +0200 Subject: [PATCH 1389/1722] Speedup test 00653_verification_monotonic_data_load --- .../00653_verification_monotonic_data_load.sh | 58 ++++++++++--------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index 7766d7720e1..2a26ec84582 100755 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -47,38 +47,42 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO date_test_table VALUES (1), (2), (2), CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --optimize_use_implicit_projections 0" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "OK" & +wait -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> UInt16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +wait -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: UInt32 -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "OK" & +wait -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> Int8" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> UInt8" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +wait - -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Date -> Int16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" & +wait CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') From aeac6c549873e27d180ad26ecc2572c268b528c9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 26 Aug 2024 13:15:52 +0200 Subject: [PATCH 1390/1722] fix reference --- ...verification_monotonic_data_load.reference | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference b/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference index 8900af059b8..81260c0921c 100644 --- a/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference +++ b/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference @@ -1,26 +1,26 @@ -no monotonic int case: String -> UInt64 -no monotonic int case: FixedString -> UInt64 -monotonic int case: Int32 -> Int64 -monotonic int case: Int32 -> UInt64 -monotonic int case: Int32 -> Int32 -monotonic int case: Int32 -> UInt32 -monotonic int case: Int32 -> Int16 -monotonic int case: Int32 -> UInt16 -monotonic int case: UInt32 -> Int64 -monotonic int case: UInt32 -> UInt64 -monotonic int case: UInt32 -> Int32 -monotonic int case: UInt32 -> UInt32 -monotonic int case: UInt32 -> Int16 -monotonic int case: UInt32 -> UInt16 -monotonic int case: Enum16 -> Int32 -monotonic int case: Enum16 -> UInt32 -monotonic int case: Enum16 -> Int16 -monotonic int case: Enum16 -> UInt16 -monotonic int case: Enum16 -> Int8 -monotonic int case: Enum16 -> UInt8 -monotonic int case: Date -> Int32 -monotonic int case: Date -> UInt32 -monotonic int case: Date -> Int16 -monotonic int case: Date -> UInt16 -monotonic int case: Date -> Int8 -monotonic int case: Date -> UInt8 +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK From 9339c0593b354dbb54b418425c6cfe1ac37e556c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 26 Aug 2024 13:37:37 +0200 Subject: [PATCH 1391/1722] handle timeout in flaky check --- tests/ci/integration_tests_runner.py | 67 ++++++++++++++-------------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index deddcb95cf4..a7d5a8c4cf6 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -873,21 +873,45 @@ class ClickhouseIntegrationTestsRunner: def run_impl(self, repo_path, build_path): stopwatch = Stopwatch() if self.flaky_check or self.bugfix_validate_check: - return self.run_flaky_check( - repo_path, build_path, should_fail=self.bugfix_validate_check + result_state, status_text, test_result, tests_log_paths = ( + self.run_flaky_check( + repo_path, build_path, should_fail=self.bugfix_validate_check + ) + ) + else: + result_state, status_text, test_result, tests_log_paths = ( + self.run_normal_check(build_path, repo_path) ) - self._install_clickhouse(build_path) + if self.soft_deadline_time < time.time(): + status_text = "Timeout, " + status_text + result_state = "failure" + if timeout_expired: + logging.error( + "Job killed by external timeout signal - setting status to failure!" + ) + status_text = "Job timeout expired, " + status_text + result_state = "failure" + # add mock test case to make timeout visible in job report and in ci db + test_result.insert( + 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") + ) + + if "(memory)" in self.params["context_name"]: + result_state = "success" + + return result_state, status_text, test_result, tests_log_paths + + def run_normal_check(self, build_path, repo_path): + self._install_clickhouse(build_path) logging.info("Pulling images") self._pre_pull_images(repo_path) - logging.info( "Dump iptables before run %s", subprocess.check_output("sudo iptables -nvL", shell=True), ) all_tests = self._get_all_tests(repo_path) - if self.run_by_hash_total != 0: grouped_tests = self.group_test_by_file(all_tests) all_filtered_by_hash_tests = [] @@ -895,7 +919,6 @@ class ClickhouseIntegrationTestsRunner: if stringhash(group) % self.run_by_hash_total == self.run_by_hash_num: all_filtered_by_hash_tests += tests_in_group all_tests = all_filtered_by_hash_tests - parallel_skip_tests = self._get_parallel_tests_skip_list(repo_path) logging.info( "Found %s tests first 3 %s", len(all_tests), " ".join(all_tests[:3]) @@ -927,14 +950,12 @@ class ClickhouseIntegrationTestsRunner: len(not_found_tests), " ".join(not_found_tests[:3]), ) - grouped_tests = self.group_test_by_file(filtered_sequential_tests) i = 0 for par_group in chunks(filtered_parallel_tests, PARALLEL_GROUP_SIZE): grouped_tests[f"parallel{i}"] = par_group i += 1 logging.info("Found %s tests groups", len(grouped_tests)) - counters = { "ERROR": [], "PASSED": [], @@ -945,14 +966,11 @@ class ClickhouseIntegrationTestsRunner: } # type: Dict tests_times = defaultdict(float) tests_log_paths = defaultdict(list) - items_to_run = list(grouped_tests.items()) - logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): logging.info("Shuffling test groups") random.shuffle(items_to_run) - for group, tests in items_to_run: if timeout_expired: print("Timeout expired - break tests execution") @@ -980,7 +998,6 @@ class ClickhouseIntegrationTestsRunner: if len(counters["FAILED"]) + len(counters["ERROR"]) >= 20: logging.info("Collected more than 20 failed/error tests, stopping") break - if counters["FAILED"] or counters["ERROR"]: logging.info( "Overall status failure, because we have tests in FAILED or ERROR state" @@ -989,7 +1006,6 @@ class ClickhouseIntegrationTestsRunner: else: logging.info("Overall success!") result_state = "success" - test_result = [] for state in ( "ERROR", @@ -1009,33 +1025,14 @@ class ClickhouseIntegrationTestsRunner: (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] - failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) status_text = f"fail: {failed_sum}, passed: {len(counters['PASSED'])}" - if self.soft_deadline_time < time.time(): - status_text = "Timeout, " + status_text - result_state = "failure" - - if timeout_expired: - logging.error( - "Job killed by external timeout signal - setting status to failure!" - ) - status_text = "Job timeout expired, " + status_text - result_state = "failure" - # add mock test case to make timeout visible in job report and in ci db - test_result.insert( - 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") - ) - if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" - if "(memory)" in self.params["context_name"]: - result_state = "success" - - return result_state, status_text, test_result, [] + return result_state, status_text, test_result, tests_log_paths def write_results(results_file, status_file, results, status): @@ -1068,7 +1065,9 @@ def run(): logging.info("Clearing dmesg before run") subprocess.check_call("sudo -E dmesg --clear", shell=True) - state, description, test_results, _ = runner.run_impl(repo_path, build_path) + state, description, test_results, _test_log_paths = runner.run_impl( + repo_path, build_path + ) logging.info("Tests finished") if IS_CI: From 91d14e6631adbc071dbd20e2129c2b3db2797f90 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:49:32 +0200 Subject: [PATCH 1392/1722] Update 03230_json_alias_new_old_types.sql --- tests/queries/0_stateless/03230_json_alias_new_old_types.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.sql b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql index 97d64f2a606..06d4790e0f9 100644 --- a/tests/queries/0_stateless/03230_json_alias_new_old_types.sql +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.sql @@ -1,7 +1,7 @@ set allow_experimental_object_type=1; set allow_experimental_json_type=1; set use_json_alias_for_old_object_type=0; -select '{"a" : 42}'::JSON as json, toTypeName(json); +select materialize('{"a" : 42}')::JSON as json, toTypeName(json); set use_json_alias_for_old_object_type=1; select '{"a" : 42}'::JSON as json, toTypeName(json); select '{"a" : 42}'::JSON(max_dynamic_paths=100) as json, toTypeName(json); -- {serverError BAD_ARGUMENTS} From f1130f3130d3e4702ccd7f8fce493e49ff79ea4b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:49:45 +0200 Subject: [PATCH 1393/1722] Update 03230_json_alias_new_old_types.reference --- .../0_stateless/03230_json_alias_new_old_types.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_json_alias_new_old_types.reference b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference index 91fd15697be..f03e0117618 100644 --- a/tests/queries/0_stateless/03230_json_alias_new_old_types.reference +++ b/tests/queries/0_stateless/03230_json_alias_new_old_types.reference @@ -1,2 +1,2 @@ -{"a":42} JSON +{"a":"42"} JSON {"a":42} Object(\'json\') From 77061db95595cea33c2e5f84804c1f9a799ec6d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 14:37:02 +0000 Subject: [PATCH 1394/1722] Adding comments and checks. --- .../QueryPlan/MergingAggregatedStep.cpp | 13 +++++++++++++ .../Transforms/MergingAggregatedTransform.cpp | 16 +++++++++++++++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 8332ad73df6..d35c38a4e32 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -10,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static bool memoryBoundMergingWillBeUsed( const DataStream & input_stream, bool memory_bound_merging_of_aggregation_results_enabled, @@ -93,6 +98,10 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c if (memoryBoundMergingWillBeUsed()) { + if (input_streams.front().header.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Memory bound merging of aggregated results is not supported for grouping sets."); + auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), @@ -123,6 +132,10 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c if (!memory_efficient_aggregation) { + if (input_streams.front().header.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Memory efficient merging of aggregated results is not supported for grouping sets."); + /// We union several sources into one, paralleling the work. pipeline.resize(1); diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 114a32b3d83..99fbf3bf4f0 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -13,6 +13,10 @@ namespace ErrorCodes Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header) { + /// __grouping_set is neigher GROUP BY key nor an aggregate function. + /// It behaves like a GROUP BY key, but we cannot append it to keys + /// because it changes hashing method and buckets for two level aggregation. + /// Now, this column is processed "manually" by merging each group separately. if (in_header.has("__grouping_set")) out_header.insert(0, in_header.getByName("__grouping_set")); @@ -39,32 +43,41 @@ void MergingAggregatedTransform::addBlock(Block block) auto grouping_column = block.getByPosition(grouping_position).column; block.erase(grouping_position); + /// Split a block by __grouping_set values. + const auto * grouping_column_typed = typeid_cast(grouping_column.get()); if (!grouping_column_typed) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName()); - const auto & grouping_data = grouping_column_typed->getData(); + /// Enumerate groups and fill the selector. std::map enumerated_groups; IColumn::Selector selector; + const auto & grouping_data = grouping_column_typed->getData(); size_t num_rows = grouping_data.size(); UInt64 last_group = grouping_data[0]; for (size_t row = 1; row < num_rows; ++row) { auto group = grouping_data[row]; + + /// Optimization for equal ranges. if (last_group == group) continue; + /// Optimization for single group. if (enumerated_groups.empty()) { selector.reserve(num_rows); enumerated_groups.emplace(last_group, enumerated_groups.size()); } + /// Fill the last equal range. selector.resize_fill(row, enumerated_groups[last_group]); + /// Enumerate new group if did not see it before. enumerated_groups.emplace(last_group, enumerated_groups.size()); } + /// Optimization for single group. if (enumerated_groups.empty()) { auto & bucket_to_blocks = grouping_sets[last_group]; @@ -72,6 +85,7 @@ void MergingAggregatedTransform::addBlock(Block block) return; } + /// Fill the last equal range. selector.resize_fill(num_rows, enumerated_groups[last_group]); const size_t num_groups = enumerated_groups.size(); From 42e7cc476e4e733839370681366ffde64185ba6c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 14:48:29 +0000 Subject: [PATCH 1395/1722] Fixing typos. --- src/Processors/Transforms/MergingAggregatedTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 99fbf3bf4f0..9b107b70075 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header) { - /// __grouping_set is neigher GROUP BY key nor an aggregate function. + /// __grouping_set is neither GROUP BY key nor an aggregate function. /// It behaves like a GROUP BY key, but we cannot append it to keys /// because it changes hashing method and buckets for two level aggregation. /// Now, this column is processed "manually" by merging each group separately. From e7d17573e1a005c015ba0178cb02696084774ba7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 26 Aug 2024 16:55:45 +0200 Subject: [PATCH 1396/1722] Add assert src/Storages/StorageURL.cpp --- src/Storages/StorageURL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 3fb00886540..1c7439f9a55 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -453,6 +453,7 @@ Chunk StorageURLSource::generate() .size = current_file_size, }, getContext()); + chassert(dynamic_cast(read_buf.get())); if (need_headers_virtual_column) { if (!http_response_headers_initialized) From 5f587af078eb6f9c962ee1ba0dccfefcab400f3a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 15:15:16 +0000 Subject: [PATCH 1397/1722] Review fix. --- src/Processors/Transforms/MergingAggregatedTransform.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 9b107b70075..78fb2f340bf 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -74,7 +74,9 @@ void MergingAggregatedTransform::addBlock(Block block) /// Fill the last equal range. selector.resize_fill(row, enumerated_groups[last_group]); /// Enumerate new group if did not see it before. - enumerated_groups.emplace(last_group, enumerated_groups.size()); + enumerated_groups.emplace(group, enumerated_groups.size()); + + last_group = group; } /// Optimization for single group. From 9d9ef691968f4d93bc90bad9624af2b3390b98e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 26 Aug 2024 15:21:48 +0000 Subject: [PATCH 1398/1722] Fixing check. --- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index d35c38a4e32..7207b5e6c7f 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -132,10 +132,6 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c if (!memory_efficient_aggregation) { - if (input_streams.front().header.has("__grouping_set")) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Memory efficient merging of aggregated results is not supported for grouping sets."); - /// We union several sources into one, paralleling the work. pipeline.resize(1); @@ -145,6 +141,9 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c } else { + if (input_streams.front().header.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Memory efficient merging of aggregated results is not supported for grouping sets."); auto num_merge_threads = memory_efficient_merge_threads ? memory_efficient_merge_threads : max_threads; From 5bad631d468a8cddbd1af40dc43c2d5db2931b46 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 1399/1722] Add config for outout_format_identifers_quoting_style --- src/Core/Settings.h | 2 ++ src/Core/SettingsChangesHistory.cpp | 5 +++++ src/Core/SettingsEnums.cpp | 6 ++++++ src/Core/SettingsEnums.h | 3 +++ src/Interpreters/formatWithPossiblyHidingSecrets.h | 2 +- src/Parsers/IAST.cpp | 5 ++++- src/Parsers/IAST.h | 2 +- 7 files changed, 22 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 23dc2a8fdc5..c33562e25ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1296,6 +1296,8 @@ class IColumn; M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \ M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \ + \ + M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5e831c6301c..76ec440aaf2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,11 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.12", + { + {"output_format_identifier_quoting_style", 1, 1, "New setting."} + } + }, {"24.12", { } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 30e60f246f4..0c43b5256d6 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -244,4 +244,10 @@ IMPLEMENT_SETTING_ENUM( GroupArrayActionWhenLimitReached, ErrorCodes::BAD_ARGUMENTS, {{"throw", GroupArrayActionWhenLimitReached::THROW}, {"discard", GroupArrayActionWhenLimitReached::DISCARD}}) + +IMPLEMENT_SETTING_ENUM(IdentifierQuotingStyle, ErrorCodes::BAD_ARGUMENTS, + {{"None", IdentifierQuotingStyle::None}, + {"Backticks", IdentifierQuotingStyle::Backticks}, + {"DoubleQuotes", IdentifierQuotingStyle::DoubleQuotes}, + {"BackticksMySQL", IdentifierQuotingStyle::BackticksMySQL}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 5c2d8825e76..d358a2d44a0 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -351,6 +352,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateT DECLARE_SETTING_ENUM(SQLSecurityType) +DECLARE_SETTING_ENUM(IdentifierQuotingStyle) + enum class GroupArrayActionWhenLimitReached : uint8_t { THROW, diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 14e84f1d1a4..8ec40d3dcc3 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -26,7 +26,7 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); return settings.query.formatWithPossiblyHidingSensitiveData( - settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names); + settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names, settings.ctx->getSettingsRef().output_format_identifier_quoting_style); } } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 5bd2c92c60a..90cdd330ee8 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,12 +165,15 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const +String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style) const { + WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; settings.print_pretty_type_names = print_pretty_type_names; + settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + settings.identifier_quoting_style = identifier_quoting_style; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 2293d50b0ec..777265aaa7c 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -278,7 +278,7 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const; + String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about From d6ca589ebed7a16f498fbbe4365d9ac07686af08 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 1400/1722] Treat PROJECTION, DICTIONARY, COLUMN, INDEX names as identifiers when formating --- src/Core/SettingsChangesHistory.cpp | 2 +- src/Parsers/ASTColumnDeclaration.cpp | 3 +-- src/Parsers/ASTDictionaryAttributeDeclaration.cpp | 2 +- src/Parsers/ASTIndexDeclaration.cpp | 2 +- src/Parsers/ASTProjectionDeclaration.cpp | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 76ec440aaf2..46f6590b724 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,7 +59,7 @@ static std::initializer_listformatImpl(s, state, frame); } diff --git a/src/Parsers/ASTProjectionDeclaration.cpp b/src/Parsers/ASTProjectionDeclaration.cpp index 60050986161..af79745a88e 100644 --- a/src/Parsers/ASTProjectionDeclaration.cpp +++ b/src/Parsers/ASTProjectionDeclaration.cpp @@ -17,7 +17,7 @@ ASTPtr ASTProjectionDeclaration::clone() const void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << backQuoteIfNeed(name); + settings.writeIdentifier(name); std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); std::string nl_or_nothing = settings.one_line ? "" : "\n"; settings.ostr << settings.nl_or_ws << indent_str << "(" << nl_or_nothing; From e1d66b9e262466695bff2a30cccec047a953e269 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 1401/1722] Add output_format_always_quote_identifiers into the settings --- src/Core/ExternalTable.cpp | 12 ++++++++--- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/executeQuery.cpp | 4 ++-- .../formatWithPossiblyHidingSecrets.h | 7 ++++++- src/Parsers/ASTColumnDeclaration.cpp | 3 ++- src/Parsers/IAST.cpp | 20 ++++++++++++------- src/Parsers/IAST.h | 15 ++++++++++---- 8 files changed, 45 insertions(+), 18 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 4ff0d7092d8..1f8e63db72e 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -17,11 +17,12 @@ #include #include -#include #include #include #include #include +#include +#include "Parsers/IdentifierQuotingStyle.h" namespace DB @@ -85,7 +86,9 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg /// We use `formatWithPossiblyHidingSensitiveData` instead of `getColumnNameWithoutAlias` because `column->type` is an ASTFunction. /// `getColumnNameWithoutAlias` will return name of the function with `(arguments)` even if arguments is empty. if (column) - structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false)); + structure.emplace_back( + column->name, + column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -102,7 +105,10 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); for (size_t i = 0; i < type_list_raw->children.size(); ++i) - structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true, false)); + structure.emplace_back( + "_" + toString(i + 1), + type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData( + 0, true, true, false, false, IdentifierQuotingStyle::Backticks)); } void BaseExternalTable::initSampleBlock() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c33562e25ed..0e58562c745 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1297,6 +1297,7 @@ class IColumn; M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \ \ + M(Bool, output_format_always_quote_identifiers, false, "Always quote identifiers", 0) \ M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 46f6590b724..ac7100d3706 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list executeQueryImpl( /// Verify that AST formatting is consistent: /// If you format AST, parse it back, and format it again, you get the same string. - String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false); + String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks); /// The query can become more verbose after formatting, so: size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0; @@ -811,7 +811,7 @@ static std::tuple executeQueryImpl( chassert(ast2); - String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false); + String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks); if (formatted1 != formatted2) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 8ec40d3dcc3..61f689fb821 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -26,7 +26,12 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); return settings.query.formatWithPossiblyHidingSensitiveData( - settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names, settings.ctx->getSettingsRef().output_format_identifier_quoting_style); + settings.max_length, + settings.one_line, + show_secrets, + settings.ctx->getSettingsRef().print_pretty_type_names, + settings.ctx->getSettingsRef().output_format_always_quote_identifiers, + settings.ctx->getSettingsRef().output_format_identifier_quoting_style); } } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 39b1da68a86..7b5ea49b63d 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -66,7 +66,8 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo { frame.need_parens = false; - format_settings.writeIdentifier(name); + /// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query. + format_settings.writeIdentifier(name, true); if (type) { diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 90cdd330ee8..858630cef2e 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,14 +165,20 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style) const +String IAST::formatWithPossiblyHidingSensitiveData( + size_t max_length, + bool one_line, + bool show_secrets, + bool print_pretty_type_names, + bool always_quote_identifiers, + IdentifierQuotingStyle identifier_quoting_style) const { WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; settings.print_pretty_type_names = print_pretty_type_names; - settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + settings.always_quote_identifiers = always_quote_identifiers; settings.identifier_quoting_style = identifier_quoting_style; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); @@ -211,13 +217,13 @@ String IAST::getColumnNameWithoutAlias() const } -void IAST::FormatSettings::writeIdentifier(const String & name) const +void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoting) const { switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible arguments: always_quote_identifiers = true && " "identifier_quoting_style == IdentifierQuotingStyle::None"); @@ -226,7 +232,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::Backticks: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeBackQuotedString(name, ostr); else writeProbablyBackQuotedString(name, ostr); @@ -234,7 +240,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::DoubleQuotes: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeDoubleQuotedString(name, ostr); else writeProbablyDoubleQuotedString(name, ostr); @@ -242,7 +248,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::BackticksMySQL: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeBackQuotedStringMySQL(name, ostr); else writeProbablyBackQuotedStringMySQL(name, ostr); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 777265aaa7c..c8fb93d3bf6 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -237,7 +237,8 @@ public: { } - void writeIdentifier(const String & name) const; + // If `force_quoting` is true, `name` is always quoted regardless of `always_quote_identifiers` + void writeIdentifier(const String & name, bool force_quoting = false) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. @@ -278,7 +279,13 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks) const; + String formatWithPossiblyHidingSensitiveData( + size_t max_length, + bool one_line, + bool show_secrets, + bool print_pretty_type_names, + bool always_quote_identifiers, + IdentifierQuotingStyle identifier_quoting_style) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about @@ -287,12 +294,12 @@ public: */ String formatForLogging(size_t max_length = 0) const { - return formatWithPossiblyHidingSensitiveData(max_length, true, false, false); + return formatWithPossiblyHidingSensitiveData(max_length, true, false, false, false, IdentifierQuotingStyle::Backticks); } String formatForErrorMessage() const { - return formatWithPossiblyHidingSensitiveData(0, true, false, false); + return formatWithPossiblyHidingSensitiveData(0, true, false, false, false, IdentifierQuotingStyle::Backticks); } virtual bool hasSecretParts() const { return childrenHaveSecretParts(); } From 32e995acbb1ac613217190561fbdda976c4386b5 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 1402/1722] Update settings change history version --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ac7100d3706..982cd0389a9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,7 +57,7 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.12", + {"24.13", { {"output_format_always_quote_identifiers", false, false, "New setting."}, {"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."} From 8e6de7e767e3db62efab622a7cf05e81da5d7866 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 1403/1722] 1) Fix: move new setting to 24.9 key. 2) Fix: fource quoting dictionary attribute names --- src/Core/SettingsChangesHistory.cpp | 8 ++------ src/Parsers/ASTDictionaryAttributeDeclaration.cpp | 2 +- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 982cd0389a9..258065dcfd4 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,12 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.13", - { - {"output_format_always_quote_identifiers", false, false, "New setting."}, - {"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."} - } - }, {"24.12", { } @@ -81,6 +75,8 @@ static std::initializer_list Date: Mon, 26 Aug 2024 17:38:07 +0200 Subject: [PATCH 1404/1722] Revert "Fix prewhere without columns and without adaptive index granularity (almost w/o anything)" --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 11 ---- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +---- .../02967_prewhere_no_columns.reference | 2 - .../0_stateless/02967_prewhere_no_columns.sql | 51 ------------------- ...n_adaptive_granularity_no_errors.reference | 2 - ...198_non_adaptive_granularity_no_errors.sql | 12 ----- .../00166_explain_estimate.reference | 4 +- 7 files changed, 3 insertions(+), 91 deletions(-) delete mode 100644 tests/queries/0_stateless/02967_prewhere_no_columns.reference delete mode 100644 tests/queries/0_stateless/02967_prewhere_no_columns.sql delete mode 100644 tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference delete mode 100644 tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 195aa4fdc10..a1565173b47 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1334,17 +1334,6 @@ void IMergeTreeDataPart::loadRowsCount() auto buf = metadata_manager->read("count.txt"); readIntText(rows_count, *buf); assertEOF(*buf); - - if (!index_granularity.empty() && rows_count < index_granularity.getTotalRows() && index_granularity_info.fixed_index_granularity) - { - /// Adjust last granule size to match the number of rows in the part in case of fixed index_granularity. - index_granularity.popMark(); - index_granularity.appendMark(rows_count % index_granularity_info.fixed_index_granularity); - if (rows_count != index_granularity.getTotalRows()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Index granularity total rows in part {} does not match rows_count: {}, instead of {}", - name, index_granularity.getTotalRows(), rows_count); - } }; if (index_granularity.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 3edcce74b09..8b6735e0fe2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -577,10 +577,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) { - /// With fixed granularity we can have last mark with less rows than granularity - const bool is_last_mark = (mark_num + 1 == index_granularity.getMarksCount()); - if (!index_granularity_info.fixed_index_granularity || !is_last_mark) - throw Exception( + throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" " (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", @@ -844,14 +841,7 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_ /// Without offset rows_written_in_last_mark = 0; } - - if (compute_granularity) - { - index_granularity.popMark(); - index_granularity.appendMark(new_rows_in_last_mark); - } } - } } diff --git a/tests/queries/0_stateless/02967_prewhere_no_columns.reference b/tests/queries/0_stateless/02967_prewhere_no_columns.reference deleted file mode 100644 index df105254618..00000000000 --- a/tests/queries/0_stateless/02967_prewhere_no_columns.reference +++ /dev/null @@ -1,2 +0,0 @@ -105 -105 diff --git a/tests/queries/0_stateless/02967_prewhere_no_columns.sql b/tests/queries/0_stateless/02967_prewhere_no_columns.sql deleted file mode 100644 index efcc952caa2..00000000000 --- a/tests/queries/0_stateless/02967_prewhere_no_columns.sql +++ /dev/null @@ -1,51 +0,0 @@ -CREATE TABLE t_02967 -( - `key` Date, - `value` UInt16 -) -ENGINE = MergeTree -ORDER BY key -SETTINGS - index_granularity_bytes = 0 --8192 --, min_index_granularity_bytes = 2 - , index_granularity = 100 - , min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0 --- --- , min_bytes_for_wide_part = 2 -AS SELECT - number, - repeat(toString(number), 5) -FROM numbers(105.); - - - --- Check with newly inserted data part. It's in-memory structured are filled at insert time. -SELECT - count(ignore(*)) -FROM t_02967 -PREWHERE CAST(ignore() + 1 as UInt8) -GROUP BY - ignore(65535, *), - ignore(255, 256, *) -SETTINGS - --send_logs_level='test', - max_threads=1; - - - --- Reload part form disk to check that in-meory structures where properly serilaized-deserialized -DETACH TABLE t_02967; -ATTACH TABLE t_02967; - - -SELECT - count(ignore(*)) -FROM t_02967 -PREWHERE CAST(ignore() + 1 as UInt8) -GROUP BY - ignore(65535, *), - ignore(255, 256, *) -SETTINGS - --send_logs_level='test', - max_threads=1; - -DROP TABLE t_02967; diff --git a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference deleted file mode 100644 index fcd78da1283..00000000000 --- a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.reference +++ /dev/null @@ -1,2 +0,0 @@ -1000000 -1000000 diff --git a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql b/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql deleted file mode 100644 index 25798ef6d33..00000000000 --- a/tests/queries/0_stateless/03198_non_adaptive_granularity_no_errors.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS data_02051__fuzz_24; - -CREATE TABLE data_02051__fuzz_24 (`key` Int16, `value` String) ENGINE = MergeTree ORDER BY key SETTINGS index_granularity_bytes = 0, min_rows_for_wide_part = 0, min_bytes_for_wide_part=0 AS SELECT number, repeat(toString(number), 5) FROM numbers(1000000.); - -SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*); - -detach table data_02051__fuzz_24; -attach table data_02051__fuzz_24; - -SELECT count(ignore(*)) FROM data_02051__fuzz_24 PREWHERE materialize(1) GROUP BY ignore(*); - -DROP TABLE data_02051__fuzz_24; diff --git a/tests/queries/1_stateful/00166_explain_estimate.reference b/tests/queries/1_stateful/00166_explain_estimate.reference index 85ecd0b9a71..71ddd681581 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.reference +++ b/tests/queries/1_stateful/00166_explain_estimate.reference @@ -1,5 +1,5 @@ test hits 1 57344 7 -test hits 1 8832938 1079 -test hits 1 829354 102 +test hits 1 8839168 1079 +test hits 1 835584 102 test hits 1 8003584 977 test hits 2 581632 71 From d799a0b6ee0a703ed2f9f93f3af1839523b6f9a5 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 26 Aug 2024 18:43:21 +0200 Subject: [PATCH 1405/1722] increase timeout for flaky check --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5453bffd9c6..5de572bc0cb 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,6 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, + timeout=4*3600 # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From cf58ce217a34296c7e3332e0fe147f51fb885959 Mon Sep 17 00:00:00 2001 From: Jordi Date: Mon, 26 Aug 2024 18:49:01 +0200 Subject: [PATCH 1406/1722] Add system.projections table --- .../operations/system-tables/projections.md | 41 +++ .../System/StorageSystemProjections.cpp | 274 ++++++++++++++++++ .../System/StorageSystemProjections.h | 30 ++ src/Storages/System/attachSystemTables.cpp | 2 + .../03230_system_projections.reference | 6 + .../0_stateless/03230_system_projections.sql | 39 +++ 6 files changed, 392 insertions(+) create mode 100644 docs/en/operations/system-tables/projections.md create mode 100644 src/Storages/System/StorageSystemProjections.cpp create mode 100644 src/Storages/System/StorageSystemProjections.h create mode 100644 tests/queries/0_stateless/03230_system_projections.reference create mode 100644 tests/queries/0_stateless/03230_system_projections.sql diff --git a/docs/en/operations/system-tables/projections.md b/docs/en/operations/system-tables/projections.md new file mode 100644 index 00000000000..e9c88f731b2 --- /dev/null +++ b/docs/en/operations/system-tables/projections.md @@ -0,0 +1,41 @@ +--- +slug: /en/operations/system-tables/projections +--- +# projections + +Contains information about existing projections in all the tables. + +Columns: + +- `database` ([String](../../sql-reference/data-types/string.md)) — Database name. +- `table` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `name` ([String](../../sql-reference/data-types/string.md)) — Projection name. +- `type` ([Enum](../../sql-reference/data-types/enum.md)) — Projection type ('Normal' = 0, 'Aggregate' = 1). +- `sorting_key` ([Array(String)](../../sql-reference/data-types/array.md)) — Projection sorting key. +- `query` ([String](../../sql-reference/data-types/string.md)) — Projection query. + +**Example** + +```sql +SELECT * FROM system.projections LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: default +table: landing +name: improved_sorting_key +type: Normal +sorting_key: ['user_id','date'] +query: SELECT * ORDER BY user_id, date + +Row 2: +────── +database: default +table: landing +name: agg_no_key +type: Aggregate +sorting_key: [] +query: SELECT count() +``` diff --git a/src/Storages/System/StorageSystemProjections.cpp b/src/Storages/System/StorageSystemProjections.cpp new file mode 100644 index 00000000000..06c00d764d7 --- /dev/null +++ b/src/Storages/System/StorageSystemProjections.cpp @@ -0,0 +1,274 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +StorageSystemProjections::StorageSystemProjections(const StorageID & table_id_) + : IStorage(table_id_) +{ + auto projection_type_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"Normal", static_cast(ProjectionDescription::Type::Normal)}, + {"Aggregate", static_cast(ProjectionDescription::Type::Aggregate)} + } + ); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(ColumnsDescription( + { + { "database", std::make_shared(), "Database name."}, + { "table", std::make_shared(), "Table name."}, + { "name", std::make_shared(), "Projection name."}, + { "type", std::move(projection_type_datatype), "Projection type."}, + { "sorting_key", std::make_shared(std::make_shared()), "Projection sorting key."}, + { "query", std::make_shared(), "Projection query."}, + })); + setInMemoryMetadata(storage_metadata); +} + +class ProjectionsSource : public ISource +{ +public: + ProjectionsSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ContextPtr context_) + : ISource(header) + , column_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + , database_idx(0) + {} + + String getName() const override { return "Projections"; } + +protected: + Chunk generate() override + { + if (database_idx >= databases->size()) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t rows_count = 0; + while (rows_count < max_block_size) + { + if (tables_it && !tables_it->isValid()) + ++database_idx; + + while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (database) + break; + ++database_idx; + } + + if (database_idx >= databases->size()) + break; + + if (!tables_it || !tables_it->isValid()) + tables_it = database->getTablesIterator(context); + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) + { + auto table_name = tables_it->name(); + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + const auto table = tables_it->table(); + if (!table) + continue; + StorageMetadataPtr metadata_snapshot = table->getInMemoryMetadataPtr(); + if (!metadata_snapshot) + continue; + const auto & projections = metadata_snapshot->getProjections(); + + for (const auto & projection : projections) + { + ++rows_count; + + size_t src_index = 0; + size_t res_index = 0; + + // 'database' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + // 'table' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(table_name); + // 'name' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(projection.name); + // 'type' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(projection.type); + // 'sorting_key' column + if (column_mask[src_index++]) { + auto columns = projection.metadata->getSortingKeyColumns(); + + Array sorting_key; + sorting_key.reserve(columns.size()); + for (const auto & column : columns) + { + sorting_key.push_back(column); + } + res_columns[res_index++]->insert(sorting_key); + } + // 'query' column + if (column_mask[src_index++]) { + res_columns[res_index++]->insert(serializeAST(*projection.definition_ast->children.at(0))); + } + } + } + } + return Chunk(std::move(res_columns), rows_count); + } + +private: + std::vector column_mask; + UInt64 max_block_size; + ColumnPtr databases; + ContextPtr context; + size_t database_idx; + DatabasePtr database; + std::string database_name; + DatabaseTablesIteratorPtr tables_it; +}; + +class ReadFromSystemProjections : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemProjections"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemProjections( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::shared_ptr storage_, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , storage(std::move(storage_)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::shared_ptr storage; + std::vector columns_mask; + const size_t max_block_size; + ExpressionActionsPtr virtual_columns_filter; +}; + +void ReadFromSystemProjections::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + if (filter_actions_dag) + { + Block block_to_filter + { + { ColumnString::create(), std::make_shared(), "database" }, + }; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter); + if (dag) + virtual_columns_filter = VirtualColumnUtils::buildFilterExpression(std::move(*dag), context); + } +} + +void StorageSystemProjections::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /* processed_stage */, + size_t max_block_size, + size_t /* num_streams */) +{ + storage_snapshot->check(column_names); + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, + std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), max_block_size); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromSystemProjections::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & [database_name, database] : databases) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; + + /// Lazy database can contain only very primitive tables, + /// it cannot contain tables with data skipping indices. + /// Skip it to avoid unnecessary tables loading in the Lazy database. + if (database->getEngineName() != "Lazy") + column->insert(database_name); + } + + /// Condition on "database" in a query acts like an index. + Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; + if (virtual_columns_filter) + VirtualColumnUtils::filterBlockWithExpression(virtual_columns_filter, block); + + ColumnPtr & filtered_databases = block.getByPosition(0).column; + pipeline.init(Pipe(std::make_shared( + std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases), context))); +} + +} diff --git a/src/Storages/System/StorageSystemProjections.h b/src/Storages/System/StorageSystemProjections.h new file mode 100644 index 00000000000..46e73cec6cd --- /dev/null +++ b/src/Storages/System/StorageSystemProjections.h @@ -0,0 +1,30 @@ +#pragma once + +#include + + +namespace DB +{ + +/// For system.projections table - describes the projections in tables, similar to system.data_skipping_indices. +class StorageSystemProjections : public IStorage +{ +public: + explicit StorageSystemProjections(const StorageID & table_id_); + + std::string getName() const override { return "StorageSystemProjections"; } + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + bool isSystemStorage() const override { return true; } +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 97eda1db3fa..816ba5095b1 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -51,6 +51,7 @@ #include #include #include +#include #include #include #include @@ -166,6 +167,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "errors", "Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace."); attach(context, system_database, "warnings", "Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server."); attachNoDescription(context, system_database, "data_skipping_indices", "Contains all the information about all the data skipping indices in tables, similar to system.columns."); + attachNoDescription(context, system_database, "projections", "Contains all the information about all the projections in tables, similar to system.data_skipping_indices."); attach(context, system_database, "licenses", "Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources."); attach(context, system_database, "time_zones", "Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse."); attach(context, system_database, "backups", "Contains a list of all BACKUP or RESTORE operations with their current states and other propertis. Note, that table is not persistent and it shows only operations executed after the last server restart."); diff --git a/tests/queries/0_stateless/03230_system_projections.reference b/tests/queries/0_stateless/03230_system_projections.reference new file mode 100644 index 00000000000..22ca272955c --- /dev/null +++ b/tests/queries/0_stateless/03230_system_projections.reference @@ -0,0 +1,6 @@ +default projections improved_sorting_key Normal ['d1','key'] SELECT * ORDER BY d1, key +default projections_2 agg Aggregate ['name'] SELECT name, max(frequency) AS max_frequency GROUP BY name +default projections_2 agg_no_key Aggregate [] SELECT max(frequency) AS max_frequency +1 +2 +improved_sorting_key diff --git a/tests/queries/0_stateless/03230_system_projections.sql b/tests/queries/0_stateless/03230_system_projections.sql new file mode 100644 index 00000000000..37c1e5df8ef --- /dev/null +++ b/tests/queries/0_stateless/03230_system_projections.sql @@ -0,0 +1,39 @@ +DROP TABLE IF EXISTS projections; +DROP TABLE IF EXISTS projections_2; + +CREATE TABLE projections +( + key String, + d1 Int, + PROJECTION improved_sorting_key ( + SELECT * + ORDER BY d1, key + ) +) +Engine=MergeTree() +ORDER BY key; + +CREATE TABLE projections_2 +( + name String, + frequency UInt64, + PROJECTION agg ( + SELECT name, max(frequency) max_frequency + GROUP BY name + ), + PROJECTION agg_no_key ( + SELECT max(frequency) max_frequency + ) +) +Engine=MergeTree() +ORDER BY name; + +SELECT * FROM system.projections WHERE database = currentDatabase(); + +SELECT count(*) FROM system.projections WHERE table = 'projections' AND database = currentDatabase(); +SELECT count(*) FROM system.projections WHERE table = 'projections_2' AND database = currentDatabase(); + +SELECT name FROM system.projections WHERE type = 'Normal' AND database = currentDatabase(); + +DROP TABLE projections; +DROP TABLE projections_2; \ No newline at end of file From 13593340f241ca80a213bb2a16c81912e04bbb1d Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:24:00 +0200 Subject: [PATCH 1407/1722] Add checks --- tests/integration/runner | 45 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/tests/integration/runner b/tests/integration/runner index 0667541b196..0f1e466051f 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -6,6 +6,7 @@ import logging import os import random import shlex +import shutil import signal import string import subprocess @@ -135,6 +136,47 @@ def check_args_and_update_paths(args): ) +def check_iptables_legacy(): + iptables_path = shutil.which("iptables") + + if iptables_path is None: + print("Error: 'iptables' not found in PATH") + sys.exit(1) # Exit with error code 1 + + try: + file_info = os.stat(iptables_path) + file_info_str = str(file_info) + + if "legacy" in file_info_str: + print( + "iptables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + ) + sys.exit(1) + + except FileNotFoundError: + print(f"Error: '{iptables_path}' not found") + sys.exit(1) + + +def check_iptables_forward_accept(ipv6=False): + command = "iptables" + if ipv6: + command = "ip6tables" + output = os.popen(command + "-S FORWARD").read() + + # Check if the output contains '-P FORWARD ACCEPT' + if "-P FORWARD ACCEPT" not in output: + print( + f"'{command} -P FORWARD ACCEPT' is not set. This may cause issues in tests" + ) + + +def chech_prerequisites(): + check_iptables_legacy() + check_iptables_forward_accept(ipv6=False) + check_iptables_forward_accept(ipv6=True) + + def docker_kill_handler_handler(signum, frame): _, _ = signum, frame subprocess.check_call( @@ -163,6 +205,9 @@ if __name__ == "__main__": level=logging.INFO, format="%(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)", ) + + chech_prerequisites() + parser = argparse.ArgumentParser(description="ClickHouse integration tests runner") parser.add_argument( From df7d44bc3adf2fecbd8506e155d0016a9847f7f3 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:27:07 +0200 Subject: [PATCH 1408/1722] Lint --- tests/integration/runner | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 0f1e466051f..a0e85ea9de6 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -141,7 +141,7 @@ def check_iptables_legacy(): if iptables_path is None: print("Error: 'iptables' not found in PATH") - sys.exit(1) # Exit with error code 1 + sys.exit(1) try: file_info = os.stat(iptables_path) @@ -164,7 +164,6 @@ def check_iptables_forward_accept(ipv6=False): command = "ip6tables" output = os.popen(command + "-S FORWARD").read() - # Check if the output contains '-P FORWARD ACCEPT' if "-P FORWARD ACCEPT" not in output: print( f"'{command} -P FORWARD ACCEPT' is not set. This may cause issues in tests" From 34d13bd7467610daec7004640e1ab07da8a03f81 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 26 Aug 2024 19:33:04 +0200 Subject: [PATCH 1409/1722] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5de572bc0cb..0d5f29d705f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,7 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, - timeout=4*3600 # to be able to process many updated tests + timeout=4 * 3600 # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From 07cb5b27eb0f4ef57e507b6ab04ca97585900143 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:41:56 +0200 Subject: [PATCH 1410/1722] Lint --- tests/integration/runner | 47 +++++++++++++++++++++++----------------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index a0e85ea9de6..2224291f3fb 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -138,10 +138,13 @@ def check_args_and_update_paths(args): def check_iptables_legacy(): iptables_path = shutil.which("iptables") + ip6tables_path = shutil.which("ip6tables") if iptables_path is None: print("Error: 'iptables' not found in PATH") sys.exit(1) + if ip6tables_path is None: + print("Error: 'ip6tables' not found in PATH, ignoring") try: file_info = os.stat(iptables_path) @@ -153,29 +156,23 @@ def check_iptables_legacy(): ) sys.exit(1) + if not ip6tables_path: + return + + file_info = os.stat(ip6tables_path) + file_info_str = str(file_info) + + if "legacy" in file_info_str: + print( + "ip6tables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + ) + sys.exit(1) + except FileNotFoundError: print(f"Error: '{iptables_path}' not found") sys.exit(1) -def check_iptables_forward_accept(ipv6=False): - command = "iptables" - if ipv6: - command = "ip6tables" - output = os.popen(command + "-S FORWARD").read() - - if "-P FORWARD ACCEPT" not in output: - print( - f"'{command} -P FORWARD ACCEPT' is not set. This may cause issues in tests" - ) - - -def chech_prerequisites(): - check_iptables_legacy() - check_iptables_forward_accept(ipv6=False) - check_iptables_forward_accept(ipv6=True) - - def docker_kill_handler_handler(signum, frame): _, _ = signum, frame subprocess.check_call( @@ -205,8 +202,6 @@ if __name__ == "__main__": format="%(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)", ) - chech_prerequisites() - parser = argparse.ArgumentParser(description="ClickHouse integration tests runner") parser.add_argument( @@ -355,12 +350,24 @@ if __name__ == "__main__": help="Bind volume to this dir to use for dockerd files", ) + parser.add_argument( + "--ignore-iptables-legacy-check", + action="store_true", + default=False, + help="Ignore iptables-legacy usage check", + ) + parser.add_argument("pytest_args", nargs="*", help="args for pytest command") args = parser.parse_args() check_args_and_update_paths(args) + if not args.ignore_iptables_legacy_check: + check_iptables_legacy() + else: + logging.warning("Skipping iptables-legacy check") + parallel_args = "" if args.parallel: parallel_args += "--dist=loadfile" From 936bbe7d0d1da9c5ddecaf04eff45c3805d0429b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 26 Aug 2024 20:02:03 +0200 Subject: [PATCH 1411/1722] handle trash parts correctly --- src/Storages/MergeTree/MergeList.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index ed58b29d584..b80d7fccc91 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -33,12 +34,12 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta format_version = MERGE_TREE_DATA_OLD_FORMAT_VERSION; /// FIXME why do we need a merge list element for projection parts at all? - bool skip_sanity_checks = future_part->part_info == FAKE_RESULT_PART_FOR_PROJECTION; + bool is_fake_projection_part = future_part->part_info == FAKE_RESULT_PART_FOR_PROJECTION; size_t normal_parts_count = 0; for (const auto & source_part : future_part->parts) { - if (!skip_sanity_checks && !source_part->getParentPart()) + if (!is_fake_projection_part && !source_part->getParentPart()) { ++normal_parts_count; if (!result_part_info.contains(MergeTreePartInfo::fromPartName(source_part->name, format_version))) @@ -57,14 +58,14 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta if (!future_part->parts.empty()) { source_data_version = future_part->parts[0]->info.getDataVersion(); - is_mutation = (result_part_info.level == future_part->parts[0]->info.level); + is_mutation = (result_part_info.level == future_part->parts[0]->info.level) && !is_fake_projection_part; WriteBufferFromString out(partition); const auto & part = future_part->parts[0]; part->partition.serializeText(part->storage, out, {}); } - if (!skip_sanity_checks && is_mutation && normal_parts_count != 1) + if (!is_fake_projection_part && is_mutation && normal_parts_count != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Got {} source parts for mutation {}: {}", future_part->parts.size(), result_part_info.getPartNameV1(), fmt::join(source_part_names, ", ")); From c43fcce77da699012719a3489f6d71b3be3ce88d Mon Sep 17 00:00:00 2001 From: Jordi Date: Mon, 26 Aug 2024 20:04:19 +0200 Subject: [PATCH 1412/1722] Fix style check --- src/Storages/System/StorageSystemProjections.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/System/StorageSystemProjections.cpp b/src/Storages/System/StorageSystemProjections.cpp index 06c00d764d7..7082853e4f9 100644 --- a/src/Storages/System/StorageSystemProjections.cpp +++ b/src/Storages/System/StorageSystemProjections.cpp @@ -135,7 +135,8 @@ protected: if (column_mask[src_index++]) res_columns[res_index++]->insert(projection.type); // 'sorting_key' column - if (column_mask[src_index++]) { + if (column_mask[src_index++]) + { auto columns = projection.metadata->getSortingKeyColumns(); Array sorting_key; @@ -147,7 +148,8 @@ protected: res_columns[res_index++]->insert(sorting_key); } // 'query' column - if (column_mask[src_index++]) { + if (column_mask[src_index++]) + { res_columns[res_index++]->insert(serializeAST(*projection.definition_ast->children.at(0))); } } @@ -254,8 +256,7 @@ void ReadFromSystemProjections::initializePipeline(QueryPipelineBuilder & pipeli if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) continue; - /// Lazy database can contain only very primitive tables, - /// it cannot contain tables with data skipping indices. + /// Lazy database can contain only very primitive tables, it cannot contain tables with projections. /// Skip it to avoid unnecessary tables loading in the Lazy database. if (database->getEngineName() != "Lazy") column->insert(database_name); From fb3c4671f02edfc2e4e79610e643164c288dcffb Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 20:45:25 +0200 Subject: [PATCH 1413/1722] Improve error messages --- tests/integration/runner | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 2224291f3fb..cdd75438b87 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -152,7 +152,12 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( - "iptables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + """ + iptables is in 'legacy' mode. This is not supported. + + Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. + Or, use --ignore-iptables-legacy-check. + """ ) sys.exit(1) @@ -164,7 +169,12 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( - "ip6tables is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode." + """ + ip6tables is in 'legacy' mode. This is not supported. + + Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. + Or, use --ignore-iptables-legacy-check. + """ ) sys.exit(1) From f9f960a5ed65b479257fff70ce2ef7234c61406d Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 26 Aug 2024 22:04:54 +0200 Subject: [PATCH 1414/1722] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0d5f29d705f..e594168ece3 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,7 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, - timeout=4 * 3600 # to be able to process many updated tests + timeout=4 * 3600, # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From 89359c1e6b22d8a0a2ad5e56c5201c8b99c72b67 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 27 Aug 2024 00:53:33 +0000 Subject: [PATCH 1415/1722] Make long_parquet* tests less long --- .../queries/0_stateless/00900_long_parquet.sh | 174 +++++++++--------- .../0_stateless/00900_long_parquet_decimal.sh | 135 +++++++------- .../0_stateless/00900_long_parquet_load.sh | 12 +- 3 files changed, 167 insertions(+), 154 deletions(-) diff --git a/tests/queries/0_stateless/00900_long_parquet.sh b/tests/queries/0_stateless/00900_long_parquet.sh index 5d6317b2787..07d2f24e446 100755 --- a/tests/queries/0_stateless/00900_long_parquet.sh +++ b/tests/queries/0_stateless/00900_long_parquet.sh @@ -8,58 +8,64 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS contributors; + CREATE TABLE contributors (name String) ENGINE = Memory;" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.contributors ORDER BY name DESC FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO contributors FORMAT Parquet" -# random results -${CLICKHOUSE_CLIENT} --query="SELECT * FROM contributors LIMIT 10" > /dev/null -${CLICKHOUSE_CLIENT} --query="DROP TABLE contributors" +${CLICKHOUSE_CLIENT} -n --query=" + -- random results + SELECT * FROM contributors LIMIT 10 FORMAT Null; + DROP TABLE contributors; -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_numbers" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory" + DROP TABLE IF EXISTS parquet_numbers; + CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory;" # less than default block size (65k) ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; + TRUNCATE TABLE parquet_numbers;" # More than default block size ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; + TRUNCATE TABLE parquet_numbers;" ${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers" + TRUNCATE TABLE parquet_numbers;" ${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 1000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10; + DROP TABLE parquet_numbers; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_numbers" - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_events" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory" + DROP TABLE IF EXISTS parquet_events; + CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory;" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_events FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_events" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event; + DROP TABLE parquet_events; -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types3" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types4" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory" -# convert min type -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory" -# convert max type -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory" + DROP TABLE IF EXISTS parquet_types1; + DROP TABLE IF EXISTS parquet_types2; + DROP TABLE IF EXISTS parquet_types3; + DROP TABLE IF EXISTS parquet_types4; + CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory; + CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory; + -- convert min type + CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory; + -- convert max type + CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory; -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9))" + INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9)); -# min -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9))" + -- min + INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9)); -# max -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9))" + -- max + INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9));" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet" @@ -72,8 +78,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8 FORMAT echo diff: diff "${CLICKHOUSE_TMP}"/parquet_all_types_1.dump "${CLICKHOUSE_TMP}"/parquet_all_types_2.dump -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2" -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9))" +${CLICKHOUSE_CLIENT} -n --query=" + TRUNCATE TABLE parquet_types2; + INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9));" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 FORMAT Parquet" @@ -81,70 +88,69 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 values ( 80, ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 FORMAT Parquet" -echo dest: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8" -echo min: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8" -echo max: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT 'dest:'; + SELECT * FROM parquet_types2 ORDER BY int8; + SELECT 'min:'; + SELECT * FROM parquet_types3 ORDER BY int8; + SELECT 'max:'; + SELECT * FROM parquet_types4 ORDER BY int8; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types5" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types6" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)" + DROP TABLE IF EXISTS parquet_types5; + DROP TABLE IF EXISTS parquet_types6; + TRUNCATE TABLE parquet_types2; + CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory; + CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory; + INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" > "${CLICKHOUSE_TMP}"/parquet_all_types_5.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet" echo dest from null: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types6 ORDER BY int8" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_types6 ORDER BY int8; + DROP TABLE parquet_types5; + DROP TABLE parquet_types6; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types5" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types6" + DROP TABLE parquet_types1; + DROP TABLE parquet_types2; + DROP TABLE parquet_types3; + DROP TABLE parquet_types4; + DROP TABLE IF EXISTS parquet_arrays; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types3" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types4" + CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory(); + INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001]); -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_arrays" - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001])" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [])" + INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], []);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays ORDER BY id" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_arrays ORDER BY id; -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_arrays" + DROP TABLE parquet_arrays; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nullable_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory()" -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], [])" + DROP TABLE IF EXISTS parquet_nullable_arrays; + CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory(); + INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], []);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays ORDER BY id" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nullable_arrays" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_nullable_arrays ORDER BY id; + DROP TABLE parquet_nullable_arrays; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nested_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory() " -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []])" + DROP TABLE IF EXISTS parquet_nested_arrays; + CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory(); + INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []]);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nested_arrays" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_nested_arrays; + DROP TABLE parquet_nested_arrays; - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_decimal" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123)" + DROP TABLE IF EXISTS parquet_decimal; + CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory; + INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_decimal FORMAT Arrow" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_decimal" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_decimal; + DROP TABLE parquet_decimal;" diff --git a/tests/queries/0_stateless/00900_long_parquet_decimal.sh b/tests/queries/0_stateless/00900_long_parquet_decimal.sh index 1b82ae9bbc7..a819dcbcdc3 100755 --- a/tests/queries/0_stateless/00900_long_parquet_decimal.sh +++ b/tests/queries/0_stateless/00900_long_parquet_decimal.sh @@ -6,108 +6,115 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" -# Simple small values -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);" -#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1);" -#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10);" -#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (1, 1, 1);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; + + -- Simple small values + CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory; + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + -- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1); + -- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10); + -- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100); + INSERT INTO decimal (a, b, c) VALUES (1, 1, 1); + INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal0.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump echo diff0: diff "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (0.000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-0.000000000000000001);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');" + CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory; + INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999); + INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999); + INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999); + INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999); + INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999); + INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999); + INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999); + INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999); + INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999); + INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999); + INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001); + INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001); + INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001); + INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001); + INSERT INTO decimal (i) VALUES (0.000000000000000001); + INSERT INTO decimal (i) VALUES (-0.000000000000000001); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0); + INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990'); + INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38'); + INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal1.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump echo diff1: diff "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42);" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);" + CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory; + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42); + INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal2.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump echo diff2: diff "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2; - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory;" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" + CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory; + CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;" # Empty table test # throws No data to insert ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" 2> /dev/null echo nothing: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM decimal2 ORDER BY a, b, c, d; + TRUNCATE TABLE decimal2; -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, Null, Null)" + INSERT INTO decimal VALUES (Null, Null, Null, Null);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" echo nulls: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" -${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM decimal2 ORDER BY a, b, c, d; + TRUNCATE TABLE decimal2; -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (1, Null, Null, Null)" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, 1, Null, Null)" -${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, 1, Null)" + INSERT INTO decimal VALUES (1, Null, Null, Null); + INSERT INTO decimal VALUES (Null, 1, Null, Null); + INSERT INTO decimal VALUES (Null, Null, 1, Null);" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_3.parquet ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" -echo full orig: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d;" -echo full inserted: -${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" +${CLICKHOUSE_CLIENT} -n --query=" + SELECT 'full orig:'; + SELECT * FROM decimal ORDER BY a, b, c, d; + SELECT 'full inserted:'; + SELECT * FROM decimal2 ORDER BY a, b, c, d;" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump ${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump echo diff3: diff "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;" +${CLICKHOUSE_CLIENT} -n --query=" + DROP TABLE IF EXISTS decimal; + DROP TABLE IF EXISTS decimal2;" diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 3a7022ac0cf..0a7f10fe16d 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -57,14 +57,14 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A # COLUMNS=`$CUR_DIR/00900_parquet_create_table_columns.py $JSON` 2>&1 || continue COLUMNS=$(cat "$COLUMNS_FILE") || continue - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" - $CLICKHOUSE_CLIENT <&1 | sed 's/Exception/Ex---tion/' - ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100" - ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_load" + ${CLICKHOUSE_CLIENT} -n --query=" + SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100; + DROP TABLE parquet_load;" done From 69cf54192c3e4a74db645f48afad90f52dd2da1e Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 27 Aug 2024 09:33:45 +0000 Subject: [PATCH 1416/1722] 1) Add tests for the identifier quoting style setting. 2) Update uni-test for table overrides --- src/Parsers/ASTColumnDeclaration.cpp | 2 +- .../ASTDictionaryAttributeDeclaration.cpp | 2 +- src/Parsers/IAST.cpp | 38 +- src/Parsers/IAST.h | 6 +- src/Parsers/tests/gtest_Parser.cpp | 2 +- ..._format_identifier_quoting_style.reference | 40 +++ ...output_format_identifier_quoting_style.sql | 328 ++++++++++++++++++ 7 files changed, 408 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference create mode 100644 tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 7b5ea49b63d..d7728462df3 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -67,7 +67,7 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo frame.need_parens = false; /// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query. - format_settings.writeIdentifier(name, true); + format_settings.quoteIdentifier(name); if (type) { diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp index 070cc6f222a..a600987dc45 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp @@ -35,7 +35,7 @@ void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settin { frame.need_parens = false; - settings.writeIdentifier(name, true); + settings.quoteIdentifier(name); if (type) { diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 858630cef2e..54b11f2888e 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -217,13 +217,13 @@ String IAST::getColumnNameWithoutAlias() const } -void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoting) const +void IAST::FormatSettings::writeIdentifier(const String & name) const { switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible arguments: always_quote_identifiers = true && " "identifier_quoting_style == IdentifierQuotingStyle::None"); @@ -232,7 +232,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::Backticks: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeBackQuotedString(name, ostr); else writeProbablyBackQuotedString(name, ostr); @@ -240,7 +240,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::DoubleQuotes: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeDoubleQuotedString(name, ostr); else writeProbablyDoubleQuotedString(name, ostr); @@ -248,7 +248,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::BackticksMySQL: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeBackQuotedStringMySQL(name, ostr); else writeProbablyBackQuotedStringMySQL(name, ostr); @@ -257,6 +257,34 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } } + +void IAST::FormatSettings::quoteIdentifier(const String & name) const +{ + switch (identifier_quoting_style) + { + case IdentifierQuotingStyle::None: + { + writeBackQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::Backticks: + { + writeBackQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::DoubleQuotes: + { + writeDoubleQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::BackticksMySQL: + { + writeBackQuotedStringMySQL(name, ostr); + break; + } + } +} + void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const { String indent_str(indent, '-'); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c8fb93d3bf6..dfb6a6cbeba 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -237,8 +237,10 @@ public: { } - // If `force_quoting` is true, `name` is always quoted regardless of `always_quote_identifiers` - void writeIdentifier(const String & name, bool force_quoting = false) const; + void writeIdentifier(const String & name) const; + // Quote identifier `name` even when `always_quote_identifiers` is false. + // If `identifier_quoting_style` is `IdentifierQuotingStyle::None`, quote it with `IdentifierQuotingStyle::Backticks` + void quoteIdentifier(const String & name) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index f0abc68f966..98cd9682c9c 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -243,7 +243,7 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, }, { "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX foo `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" + "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX `foo` `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" }, { "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", diff --git a/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference new file mode 100644 index 00000000000..c563617a01c --- /dev/null +++ b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference @@ -0,0 +1,40 @@ +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid` +CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month` +CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE "default"."uk_price_paid"\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX "county_index" "county" TYPE set(10) GRANULARITY 1,\n PROJECTION "town_date_projection"\n (\n SELECT \n "town",\n "date",\n "price"\n ORDER BY \n "town",\n "date"\n ),\n PROJECTION "handy_aggs_projection"\n (\n SELECT \n avg("price"),\n max("price"),\n sum("price")\n GROUP BY "town"\n )\n)\nENGINE = MergeTree\nORDER BY ("postcode1", "postcode2", "date")\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW "default"."prices_by_year_view" TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n "price",\n "date",\n "addr1",\n "addr2",\n "street",\n "town",\n "district",\n "county"\nFROM "default"."uk_price_paid" +CREATE TABLE "default"."uk_prices_aggs_dest"\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY "month"\nORDER BY "month"\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW "default"."uk_prices_aggs_view" TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nAS WITH toStartOfMonth("date") AS "month"\nSELECT\n "month",\n minSimpleState("price") AS "min_price",\n maxSimpleState("price") AS "max_price",\n countState("price") AS "volume",\n avgState("price") AS "avg_price"\nFROM "default"."uk_price_paid"\nGROUP BY "month" +CREATE DICTIONARY "default"."uk_mortgage_rates_dict"\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY "date"\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid` +CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month` +CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) diff --git a/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql new file mode 100644 index 00000000000..c500dd4e4c6 --- /dev/null +++ b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql @@ -0,0 +1,328 @@ +DROP DICTIONARY IF EXISTS uk_mortgage_rates_dict; +DROP TABLE IF EXISTS uk_mortgage_rates; +DROP VIEW IF EXISTS uk_prices_aggs_view; +DROP TABLE IF EXISTS uk_prices_aggs_dest; +DROP VIEW IF EXISTS prices_by_year_view; +DROP TABLE IF EXISTS prices_by_year_dest; +DROP TABLE IF EXISTS uk_price_paid; + +-- Create tables, views, dictionaries + +CREATE TABLE uk_price_paid +( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + is_new UInt8, + duration Enum('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String), + INDEX county_index county TYPE set(10) GRANULARITY 1, + PROJECTION town_date_projection + ( + SELECT + town, + date, + price + ORDER BY + town, + date + ), + PROJECTION handy_aggs_projection + ( + SELECT + avg(price), + max(price), + sum(price) + GROUP BY town + ) +) +ENGINE = MergeTree +ORDER BY (postcode1, postcode2, date); + +CREATE TABLE prices_by_year_dest ( + price UInt32, + date Date, + addr1 String, + addr2 String, + street LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String) +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date); + +CREATE MATERIALIZED VIEW prices_by_year_view +TO prices_by_year_dest +AS + SELECT + price, + date, + addr1, + addr2, + street, + town, + district, + county + FROM uk_price_paid; + +CREATE TABLE uk_prices_aggs_dest ( + month Date, + min_price SimpleAggregateFunction(min, UInt32), + max_price SimpleAggregateFunction(max, UInt32), + volume AggregateFunction(count, UInt32), + avg_price AggregateFunction(avg, UInt32) +) +ENGINE = AggregatingMergeTree +PRIMARY KEY month; + +CREATE MATERIALIZED VIEW uk_prices_aggs_view +TO uk_prices_aggs_dest +AS + WITH + toStartOfMonth(date) AS month + SELECT + month, + minSimpleState(price) AS min_price, + maxSimpleState(price) AS max_price, + countState(price) AS volume, + avgState(price) AS avg_price + FROM uk_price_paid + GROUP BY month; + +CREATE TABLE uk_mortgage_rates ( + date DateTime64, + variable Decimal32(2), + fixed Decimal32(2), + bank Decimal32(2) +) +ENGINE Memory(); + +INSERT INTO uk_mortgage_rates VALUES ('2004-02-29', 5.02, 4.9, 4); +INSERT INTO uk_mortgage_rates VALUES ('2004-03-31', 5.11, 4.91, 4); + +CREATE DICTIONARY uk_mortgage_rates_dict ( + date DateTime64, + variable Decimal32(2), + fixed Decimal32(2), + bank Decimal32(2) +) +PRIMARY KEY date +SOURCE( + CLICKHOUSE(TABLE 'uk_mortgage_rates') +) +LAYOUT(COMPLEX_KEY_HASHED()) +LIFETIME(2628000000); + + +-- Show tables, views, dictionaries with default settings +SHOW CREATE TABLE uk_price_paid; + +SHOW CREATE VIEW prices_by_year_view; + +SHOW CREATE uk_prices_aggs_dest; + +SHOW CREATE VIEW uk_prices_aggs_view; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict; + + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='None' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='Backticks' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='Backticks' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='DoubleQuotes' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='DoubleQuotes' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='BackticksMySQL' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='BackticksMySQL' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +DROP DICTIONARY uk_mortgage_rates_dict; +DROP TABLE uk_mortgage_rates; +DROP VIEW uk_prices_aggs_view; +DROP TABLE uk_prices_aggs_dest; +DROP VIEW prices_by_year_view; +DROP TABLE prices_by_year_dest; +DROP TABLE uk_price_paid; From 1aebcc1cb7b90416c7a85a40d915441efbe1c98a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 27 Aug 2024 11:43:15 +0200 Subject: [PATCH 1417/1722] Fix 01114_database_atomic flakiness --- .../0_stateless/01114_database_atomic.sh | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index fed76727a27..f74036c8462 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -53,10 +53,20 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '$explicit_uuid' (n UIn $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +RANDOM_COMMENT="$RANDOM" +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) -sleep 1 # SELECT and INSERT should start before the following RENAMEs +it=0 +while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do + it=$((it+1)) + if [ $it -ge 50 ]; + then + echo "Failed to wait for first batch of queries" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'" + fi + sleep 0.1 +done $CLICKHOUSE_CLIENT -nm -q " RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp; @@ -79,8 +89,18 @@ INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); SELECT count() FROM test_01114_1.mt " # result: 5 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) -sleep 1 +RANDOM_TUPLE="${RANDOM}_tuple" +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) +it=0 +while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do + it=$((it+1)) + if [ $it -ge 50 ]; + then + echo "Failed to wait for second batch of queries" + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'" + fi + sleep 0.1 +done $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" wait # for INSERT and SELECT From 2d2e8b3a967b5455870d6a9dd873c1deeaf77af3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 27 Aug 2024 11:52:24 +0200 Subject: [PATCH 1418/1722] Make the test parallelizable --- .../01114_database_atomic.reference | 17 ++-- .../0_stateless/01114_database_atomic.sh | 89 +++++++++---------- 2 files changed, 50 insertions(+), 56 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index f42cd099d4e..33080a7a77b 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -1,17 +1,14 @@ 1 -CREATE DATABASE test_01114_1\nENGINE = Atomic -CREATE DATABASE test_01114_2\nENGINE = Atomic -CREATE DATABASE test_01114_3\nENGINE = Ordinary -test_01114_1 Atomic store 00001114-1000-4000-8000-000000000001 1 -test_01114_2 Atomic store 00001114-1000-4000-8000-000000000002 1 -test_01114_3 Ordinary test_01114_3 test_01114_3 1 +CREATE DATABASE default_1\nENGINE = Atomic +CREATE DATABASE default_2\nENGINE = Atomic +CREATE DATABASE default_3\nENGINE = Ordinary 110 100 -CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt (`n` UInt64) ENGINE = MergeTree PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000002 CREATE TABLE default_2.mt (`n` UInt64) ENGINE = MergeTree PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 110 -CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 5 dropped 110 5995 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index f74036c8462..412f5336224 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -9,25 +9,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +DATABASE_1="${CLICKHOUSE_DATABASE}_1" +DATABASE_2="${CLICKHOUSE_DATABASE}_2" +DATABASE_3="${CLICKHOUSE_DATABASE}_3" -$CLICKHOUSE_CLIENT -nm -q " -DROP DATABASE IF EXISTS test_01114_1; -DROP DATABASE IF EXISTS test_01114_2; -DROP DATABASE IF EXISTS test_01114_3; -" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE ${DATABASE_1} ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" -$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE test_01114_1 ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${DATABASE_1} ENGINE=Atomic" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${DATABASE_2}" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE ${DATABASE_3} ENGINE=Ordinary" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_2" -$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE test_01114_3 ENGINE=Ordinary" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE ${DATABASE_1}" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE ${DATABASE_2}" +$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE ${DATABASE_3}" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_1" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_2" -$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_3" - -uuid_db_1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='test_01114_1'"` -uuid_db_2=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='test_01114_2'"` +uuid_db_1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='${DATABASE_1}'"` +uuid_db_2=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='${DATABASE_2}'"` $CLICKHOUSE_CLIENT -q "SELECT name, engine, splitByChar('/', data_path)[-2], @@ -35,27 +32,27 @@ $CLICKHOUSE_CLIENT -q "SELECT name, FROM system.databases WHERE name LIKE 'test_01114_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -nm -q " -CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); -INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100); -CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5); -INSERT INTO test_01114_3.mt SELECT * FROM numbers(110); +CREATE TABLE ${DATABASE_1}.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO ${DATABASE_1}.mt_tmp SELECT * FROM numbers(100); +CREATE TABLE ${DATABASE_3}.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5); +INSERT INTO ${DATABASE_3}.mt SELECT * FROM numbers(110); -RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp; /* move from Atomic to Ordinary */ -RENAME TABLE test_01114_3.mt TO test_01114_1.mt; /* move from Ordinary to Atomic */ -SELECT count() FROM test_01114_1.mt; -SELECT count() FROM test_01114_3.mt_tmp; +RENAME TABLE ${DATABASE_1}.mt_tmp TO ${DATABASE_3}.mt_tmp; /* move from Atomic to Ordinary */ +RENAME TABLE ${DATABASE_3}.mt TO ${DATABASE_1}.mt; /* move from Ordinary to Atomic */ +SELECT count() FROM ${DATABASE_1}.mt; +SELECT count() FROM ${DATABASE_3}.mt_tmp; -DROP DATABASE test_01114_3; +DROP DATABASE ${DATABASE_3}; " explicit_uuid=$($CLICKHOUSE_CLIENT -q "SELECT generateUUIDv4()") -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${DATABASE_2}.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_2}.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='${DATABASE_2}'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" RANDOM_COMMENT="$RANDOM" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do @@ -69,28 +66,28 @@ while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE que done $CLICKHOUSE_CLIENT -nm -q " -RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp; -RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp; -EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp; -RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt; -EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt; +RENAME TABLE ${DATABASE_1}.mt TO ${DATABASE_1}.mt_tmp; +RENAME TABLE ${DATABASE_1}.mt_tmp TO ${DATABASE_2}.mt_tmp; +EXCHANGE TABLES ${DATABASE_2}.mt AND ${DATABASE_2}.mt_tmp; +RENAME TABLE ${DATABASE_2}.mt_tmp TO ${DATABASE_1}.mt; +EXCHANGE TABLES ${DATABASE_1}.mt AND ${DATABASE_2}.mt; " # Check that nothing changed -$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" -uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='test_01114_1' AND name='mt'") -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_1.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" -$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" +$CLICKHOUSE_CLIENT -q "SELECT count() FROM ${DATABASE_1}.mt" +uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='${DATABASE_1}' AND name='mt'") +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_1}.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" +$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_2}.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -nm -q " -DROP TABLE test_01114_1.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0; -CREATE TABLE test_01114_1.mt (s String) ENGINE=Log(); -INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); -SELECT count() FROM test_01114_1.mt +DROP TABLE ${DATABASE_1}.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0; +CREATE TABLE ${DATABASE_1}.mt (s String) ENGINE=Log(); +INSERT INTO ${DATABASE_1}.mt SELECT 's' || toString(number) FROM numbers(5); +SELECT count() FROM ${DATABASE_1}.mt " # result: 5 RANDOM_TUPLE="${RANDOM}_tuple" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do it=$((it+1)) @@ -101,9 +98,9 @@ while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE que fi sleep 0.1 done -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${DATABASE_1}" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" wait # for INSERT and SELECT -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01114_2.mt" # result: 30, 435 -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_2" --database_atomic_wait_for_drop_and_detach_synchronously=0 +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${DATABASE_2}.mt" # result: 30, 435 +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${DATABASE_2}" --database_atomic_wait_for_drop_and_detach_synchronously=0 From 94f168e5ef6fb1e7121197cef9bd32de0ff391d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 27 Aug 2024 12:12:34 +0200 Subject: [PATCH 1419/1722] CI: Fix job rerun check --- tests/ci/ci_utils.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index e8d9e7dc254..cd7a271f892 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -286,4 +286,6 @@ class Utils: @staticmethod def is_job_triggered_manually(): - return "robot" not in Envs.GITHUB_ACTOR + return ( + "robot" not in Envs.GITHUB_ACTOR and "clickhouse-ci" not in Envs.GITHUB_ACTOR + ) From 0ad66778da65f9e6c2659044343763e47f3f4878 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 27 Aug 2024 10:19:19 +0000 Subject: [PATCH 1420/1722] Automatic style fix --- tests/ci/ci_utils.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index cd7a271f892..8b60f61b006 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -287,5 +287,6 @@ class Utils: @staticmethod def is_job_triggered_manually(): return ( - "robot" not in Envs.GITHUB_ACTOR and "clickhouse-ci" not in Envs.GITHUB_ACTOR + "robot" not in Envs.GITHUB_ACTOR + and "clickhouse-ci" not in Envs.GITHUB_ACTOR ) From 1523df6ec310ae7a431b32ef084c50e4377d11ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Aug 2024 10:31:11 +0000 Subject: [PATCH 1421/1722] Ignore `MODIFY_COLUMN` commands without column type when parsing mutation commands --- src/Storages/MutationCommands.cpp | 5 ++++- ...xed_mutations_and_remove_materialized.reference | 8 ++++++++ ...ith_mixed_mutations_and_remove_materialized.sql | 14 ++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference create mode 100644 tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f736c863eee..f5ccc80f1d8 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -115,7 +115,10 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.column_name = getIdentifierName(command->column); return res; } - else if (parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN) + /// MODIFY COLUMN x REMOVE MATERIALIZED is a valid alter command, but doesn't have any specified column type, thus no mutation is needed + else if ( + parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->col_decl + && command->col_decl->as().type) { MutationCommand res; res.ast = command->ptr(); diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference new file mode 100644 index 00000000000..9166a82f472 --- /dev/null +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference @@ -0,0 +1,8 @@ +BEFORE a_r1 x String +BEFORE a_r1 y String MATERIALIZED \'str\' +BEFORE a_r2 x String +BEFORE a_r2 y String MATERIALIZED \'str\' +AFTER a_r1 x String +AFTER a_r1 y String +AFTER a_r2 x String +AFTER a_r2 y String diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql new file mode 100644 index 00000000000..3c43b9a8521 --- /dev/null +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS a_r1 SYNC; +DROP TABLE IF EXISTS a_r2 SYNC; +CREATE TABLE a_r1 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x; +CREATE TABLE a_r2 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r2') ORDER BY x; + +INSERT INTO a_r1 SELECT toString(number) FROM numbers(100); +SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; + +ALTER TABLE a_r1 + ADD INDEX IF NOT EXISTS some_index x TYPE set(16) GRANULARITY 1, + MODIFY COLUMN y REMOVE MATERIALIZED +SETTINGS alter_sync = 2, mutations_sync = 2; + +SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; From 793b549291db336be2596c2fe7382d7b7943b547 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Aug 2024 10:31:22 +0000 Subject: [PATCH 1422/1722] Fix typo in docs --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7bd36ccd00f..816d1caa632 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3226,7 +3226,7 @@ Default value: `0`. ## lightweight_deletes_sync {#lightweight_deletes_sync} -The same as 'mutation_sync', but controls only execution of lightweight deletes. +The same as [`mutations_sync`](#mutations_sync), but controls only execution of lightweight deletes. Possible values: From 43482c4ed0cec53680314357ba2a91891b27dd73 Mon Sep 17 00:00:00 2001 From: Pedro Ferreira Date: Tue, 27 Aug 2024 10:42:47 +0000 Subject: [PATCH 1423/1722] Recreate database at each run --- .../test_distributed.py | 37 +++++++++---------- .../test_truncate_database/test_replicated.py | 5 ++- 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_truncate_database/test_distributed.py b/tests/integration/test_truncate_database/test_distributed.py index fed3d16d190..67dea70b1f4 100644 --- a/tests/integration/test_truncate_database/test_distributed.py +++ b/tests/integration/test_truncate_database/test_distributed.py @@ -1,40 +1,35 @@ import pytest - from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/distributed_servers.xml"], with_zookeeper=True + "node1", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] ) node2 = cluster.add_instance( "node2", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] ) +def fill_nodes(nodes): + for node in nodes: + node.query( + """ + DROP DATABASE IF EXISTS test; + CREATE DATABASE test; + CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id; + CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table); + INSERT INTO test.local_table VALUES ({pos}, '{replica}'); + """.format( + pos=node.name[4:], replica=node.name + ) + ) + @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - for node in (node1, node2): - node.query( - """ - CREATE DATABASE test; - CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id - """ - ) - - node1.query("INSERT INTO test.local_table VALUES (1, 'node1')") - node2.query("INSERT INTO test.local_table VALUES (2, 'node2')") - - node1.query( - "CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)" - ) - node2.query( - "CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)" - ) - yield cluster finally: @@ -42,6 +37,8 @@ def started_cluster(): def test_truncate_database_distributed(started_cluster): + fill_nodes([node1, node2]) + query1 = "SELECT count() FROM test.distributed WHERE (id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))" query2 = "SELECT sum((id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))) FROM test.distributed" assert node1.query(query1) == "1\n" diff --git a/tests/integration/test_truncate_database/test_replicated.py b/tests/integration/test_truncate_database/test_replicated.py index d55a0dc7557..edcc0446da3 100644 --- a/tests/integration/test_truncate_database/test_replicated.py +++ b/tests/integration/test_truncate_database/test_replicated.py @@ -6,6 +6,7 @@ def fill_nodes(nodes, shard): for node in nodes: node.query( """ + DROP DATABASE IF EXISTS test; CREATE DATABASE test; CREATE TABLE test.test_table(date Date, id UInt32) @@ -33,8 +34,6 @@ def start_cluster(): try: cluster.start() - fill_nodes([node1, node2, node3], 1) - yield cluster except Exception as ex: @@ -45,6 +44,8 @@ def start_cluster(): def test_truncate_database_replicated(start_cluster): + fill_nodes([node1, node2, node3], 1) + node1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" ) From c44cc71310f982bd9d4101ee02001f5590971f17 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Aug 2024 11:32:05 +0000 Subject: [PATCH 1424/1722] forgot several files --- tests/integration/helpers/postgres_utility.py | 14 +++++++------- .../test.py | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 468c3b3bb63..4bb68284314 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -245,9 +245,9 @@ class PostgresManager: ): postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}`") - create_query = f"CREATE DATABASE {materialized_database} ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" + create_query = f"CREATE DATABASE `{materialized_database}` ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" if len(settings) > 0: create_query += " SETTINGS " for i in range(len(settings)): @@ -259,7 +259,7 @@ class PostgresManager: assert materialized_database in self.instance.query("SHOW DATABASES") def drop_materialized_db(self, materialized_database="test_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}` SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) @@ -329,11 +329,11 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") assert table in database_tables @@ -366,9 +366,9 @@ def check_tables_are_synchronized( table_path = "" if len(schema_name) == 0: - table_path = f"{materialized_database}.{table_name}" + table_path = f"`{materialized_database}`.`{table_name}`" else: - table_path = f"{materialized_database}.`{schema_name}.{table_name}`" + table_path = f"`{materialized_database}`.`{schema_name}.{table_name}`" print(f"Checking table is synchronized: {table_path}") result_query = f"select * from {table_path} order by {order_by};" diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 5599786026f..280985b55b3 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -356,7 +356,7 @@ def test_change_queries_frequency(): max_values_size=1000, array_size_range=[10, 50], ) - + node.query("SYSTEM FLUSH LOGS") select_log_query = f"SELECT countIf(timeout_milliseconds - {min_ms} < 25) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) assert int(res) >= 5 @@ -364,12 +364,12 @@ def test_change_queries_frequency(): _insert_queries_in_parallel( table_name, settings, - thread_num=20, - tasks=2000, + thread_num=10, + tasks=1000, max_values_size=1000, array_size_range=[10, 15], ) - + node.query("SYSTEM FLUSH LOGS") select_log_query = f"SELECT countIf({max_ms} - timeout_milliseconds < 100) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) assert int(res) >= 5 From c1a83f47341032ae90f43ba84a54c0ec0e60810c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 27 Aug 2024 13:44:17 +0200 Subject: [PATCH 1425/1722] Fix possible wrong result during anyHeavy state merge --- src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp | 3 +++ tests/queries/0_stateless/03230_anyHeavy_merge.reference | 1 + tests/queries/0_stateless/03230_anyHeavy_merge.sql | 4 ++++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03230_anyHeavy_merge.reference create mode 100644 tests/queries/0_stateless/03230_anyHeavy_merge.sql diff --git a/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp b/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp index ffddd46f2e3..dbc5f9be72f 100644 --- a/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp @@ -68,7 +68,10 @@ public: if (data().isEqualTo(to.data())) counter += to.counter; else if (!data().has() || counter < to.counter) + { data().set(to.data(), arena); + counter = to.counter - counter; + } else counter -= to.counter; } diff --git a/tests/queries/0_stateless/03230_anyHeavy_merge.reference b/tests/queries/0_stateless/03230_anyHeavy_merge.reference new file mode 100644 index 00000000000..78981922613 --- /dev/null +++ b/tests/queries/0_stateless/03230_anyHeavy_merge.reference @@ -0,0 +1 @@ +a diff --git a/tests/queries/0_stateless/03230_anyHeavy_merge.sql b/tests/queries/0_stateless/03230_anyHeavy_merge.sql new file mode 100644 index 00000000000..5d4c0e55d0f --- /dev/null +++ b/tests/queries/0_stateless/03230_anyHeavy_merge.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (letter String) ENGINE=MergeTree order by () partition by letter; +INSERT INTO t VALUES ('a'), ('a'), ('a'), ('a'), ('b'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('c'); +SELECT anyHeavy(if(letter != 'b', letter, NULL)) FROM t; From c7d0d790e2b37bcd91f5e147d775e656bf3d22a7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 27 Aug 2024 12:38:37 +0000 Subject: [PATCH 1426/1722] fix materialized views with optimize_functions_to_subcolumns --- .../Passes/FunctionToSubcolumnsPass.cpp | 10 +++-- .../0_stateless/03230_subcolumns_mv.reference | 1 + .../0_stateless/03230_subcolumns_mv.sql | 37 +++++++++++++++++++ 3 files changed, 45 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03230_subcolumns_mv.reference create mode 100644 tests/queries/0_stateless/03230_subcolumns_mv.sql diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 1fc3eec6833..6caf69e3a2c 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -209,7 +209,7 @@ std::map, NodeToSubcolumnTransformer> node_transfor }, }; -std::tuple getTypedNodesForOptimization(const QueryTreeNodePtr & node) +std::tuple getTypedNodesForOptimization(const QueryTreeNodePtr & node, const ContextPtr & context) { auto * function_node = node->as(); if (!function_node) @@ -232,6 +232,10 @@ std::tuple getTypedNodesForOptimizati const auto & storage_snapshot = table_node->getStorageSnapshot(); auto column = first_argument_column_node->getColumn(); + auto view_source = context->getViewSource(); + if (view_source && view_source->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted()) + return {}; + if (!storage->supportsOptimizationToSubcolumns() || storage->isVirtualColumn(column.name, storage_snapshot->metadata)) return {}; @@ -266,7 +270,7 @@ public: return; } - auto [function_node, first_argument_node, table_node] = getTypedNodesForOptimization(node); + auto [function_node, first_argument_node, table_node] = getTypedNodesForOptimization(node, getContext()); if (function_node && first_argument_node && table_node) { enterImpl(*function_node, *first_argument_node, *table_node); @@ -416,7 +420,7 @@ public: if (!getSettings().optimize_functions_to_subcolumns) return; - auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node); + auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node, getContext()); if (!function_node || !first_argument_column_node || !table_node) return; diff --git a/tests/queries/0_stateless/03230_subcolumns_mv.reference b/tests/queries/0_stateless/03230_subcolumns_mv.reference new file mode 100644 index 00000000000..03528148b49 --- /dev/null +++ b/tests/queries/0_stateless/03230_subcolumns_mv.reference @@ -0,0 +1 @@ +['key1','key2'] ['value1','value2'] diff --git a/tests/queries/0_stateless/03230_subcolumns_mv.sql b/tests/queries/0_stateless/03230_subcolumns_mv.sql new file mode 100644 index 00000000000..e2e577f54c1 --- /dev/null +++ b/tests/queries/0_stateless/03230_subcolumns_mv.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS rawtable; +DROP TABLE IF EXISTS raw_to_attributes_mv; +DROP TABLE IF EXISTS attributes; + +SET optimize_functions_to_subcolumns = 1; + +CREATE TABLE rawtable +( + `Attributes` Map(String, String), +) +ENGINE = MergeTree +ORDER BY tuple(); + +CREATE MATERIALIZED VIEW raw_to_attributes_mv TO attributes +( + `AttributeKeys` Array(String), + `AttributeValues` Array(String) +) +AS SELECT + mapKeys(Attributes) AS AttributeKeys, + mapValues(Attributes) AS AttributeValues +FROM rawtable; + +CREATE TABLE attributes +( + `AttributeKeys` Array(String), + `AttributeValues` Array(String) +) +ENGINE = ReplacingMergeTree +ORDER BY tuple(); + +INSERT INTO rawtable VALUES ({'key1': 'value1', 'key2': 'value2'}); +SELECT * FROM raw_to_attributes_mv ORDER BY AttributeKeys; + +DROP TABLE IF EXISTS rawtable; +DROP TABLE IF EXISTS raw_to_attributes_mv; +DROP TABLE IF EXISTS attributes; From 6ee197ea59d9f7f5d3e8dc24b8eded763c65d024 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Aug 2024 13:01:37 +0000 Subject: [PATCH 1427/1722] Don't use serializations cache in const Dynamic column methods --- src/Columns/ColumnDynamic.cpp | 11 +++++------ src/Columns/ColumnDynamic.h | 6 +++--- src/DataTypes/DataTypeDynamic.cpp | 2 +- src/DataTypes/Serializations/SerializationDynamic.cpp | 5 ++--- ...namic_serializations_uninitialized_value.reference | 4 ++++ ...228_dynamic_serializations_uninitialized_value.sql | 4 ++++ 6 files changed, 19 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference create mode 100644 tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..a66db43d6d4 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -300,7 +300,7 @@ void ColumnDynamic::get(size_t n, Field & res) const auto value_data = shared_variant.getDataAt(variant_col.offsetAt(n)); ReadBufferFromMemory buf(value_data.data, value_data.size); auto type = decodeDataType(buf); - getVariantSerialization(type)->deserializeBinary(res, buf, getFormatSettings()); + type->getDefaultSerialization()->deserializeBinary(res, buf, getFormatSettings()); } @@ -736,8 +736,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const { const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); encodeDataType(variant_type, buf); - getVariantSerialization(variant_type, variant_info.variant_names[discr]) - ->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings()); + variant_type->getDefaultSerialization()->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings()); type_and_value = buf.str(); } @@ -870,7 +869,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// We have both values serialized in binary format, so we need to /// create temporary column, insert both values into it and compare. auto tmp_column = left_data_type->createColumn(); - const auto & serialization = getVariantSerialization(left_data_type, left_data_type_name); + const auto & serialization = left_data_type->getDefaultSerialization(); serialization->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); serialization->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); return tmp_column->compareAt(0, 1, *tmp_column, nan_direction_hint); @@ -892,7 +891,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// We have left value serialized in binary format, we need to /// create temporary column, insert the value into it and compare. auto tmp_column = left_data_type->createColumn(); - getVariantSerialization(left_data_type, left_data_type_name)->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); + left_data_type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf_left, getFormatSettings()); return tmp_column->compareAt(0, right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint); } /// Check if only right value is in shared data. @@ -912,7 +911,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_ /// We have right value serialized in binary format, we need to /// create temporary column, insert the value into it and compare. auto tmp_column = right_data_type->createColumn(); - getVariantSerialization(right_data_type, right_data_type_name)->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); + right_data_type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf_right, getFormatSettings()); return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), 0, *tmp_column, nan_direction_hint); } /// Otherwise both values are regular variants. diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 2ae862de3af..72542a15530 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -414,7 +414,7 @@ public: /// Insert value into shared variant. Also updates Variant discriminators and offsets. void insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n); - const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) const + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) { /// Get serialization for provided data type. /// To avoid calling type->getDefaultSerialization() every time we use simple cache with max size. @@ -428,7 +428,7 @@ public: return serialization_cache.emplace(variant_name, variant_type->getDefaultSerialization()).first->second; } - const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } + const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) { return getVariantSerialization(variant_type, variant_type->getName()); } private: void createVariantInfo(const DataTypePtr & variant_type); @@ -473,7 +473,7 @@ private: /// We can use serializations of different data types to serialize values into shared variant. /// To avoid creating the same serialization multiple times, use simple cache. static const size_t SERIALIZATION_CACHE_MAX_SIZE = 256; - mutable std::unordered_map serialization_cache; + std::unordered_map serialization_cache; }; void extendVariantColumn( diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index c35f7526a18..fb938f5fbd8 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -185,7 +185,7 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa auto type = decodeDataType(buf); if (type->getName() == subcolumn_type_name) { - dynamic_column.getVariantSerialization(subcolumn_type, subcolumn_type_name)->deserializeBinary(*subcolumn, buf, format_settings); + subcolumn_type->getDefaultSerialization()->deserializeBinary(*subcolumn, buf, format_settings); null_map.push_back(0); } else diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index b921a3bc897..32964e17bce 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -489,9 +489,8 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu } const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(global_discr); - const auto & variant_type_name = variant_info.variant_names[global_discr]; encodeDataType(variant_type, ostr); - dynamic_column.getVariantSerialization(variant_type, variant_type_name)->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); + variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); } template @@ -629,7 +628,7 @@ static void serializeTextImpl( ReadBufferFromMemory buf(value.data, value.size); auto variant_type = decodeDataType(buf); auto tmp_variant_column = variant_type->createColumn(); - auto variant_serialization = dynamic_column.getVariantSerialization(variant_type); + auto variant_serialization = variant_type->getDefaultSerialization(); variant_serialization->deserializeBinary(*tmp_variant_column, buf, settings); nested_serialize(*variant_serialization, *tmp_variant_column, 0, ostr); } diff --git a/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference new file mode 100644 index 00000000000..64c193a8499 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.reference @@ -0,0 +1,4 @@ +str 3 \N +str 3 \N +str 3 \N +str 3 \N diff --git a/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql new file mode 100644 index 00000000000..8a565fe36b9 --- /dev/null +++ b/tests/queries/0_stateless/03228_dynamic_serializations_uninitialized_value.sql @@ -0,0 +1,4 @@ +set allow_experimental_dynamic_type=1; +set cast_keep_nullable=1; +SELECT toFixedString('str', 3), 3, CAST(if(1 = 0, toInt8(3), NULL), 'Int32') AS x from numbers(10) GROUP BY GROUPING SETS ((CAST(toInt32(1), 'Int32')), ('str', 3), (CAST(toFixedString('str', 3), 'Dynamic')), (CAST(toFixedString(toFixedString('str', 3), 3), 'Dynamic'))); + From cc701c92b4d870a4a2028cccbc46c2cfda8c405d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Aug 2024 15:05:05 +0200 Subject: [PATCH 1428/1722] Add plan_step_name and plan_step_description columns to system.processors_profile_log --- src/Interpreters/ProcessorsProfileLog.cpp | 4 ++++ src/Interpreters/ProcessorsProfileLog.h | 2 ++ src/Processors/IProcessor.cpp | 12 ++++++++++++ src/Processors/IProcessor.h | 10 +++++----- 4 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ProcessorsProfileLog.cpp b/src/Interpreters/ProcessorsProfileLog.cpp index 7dec2a3163a..8a646b5d0e7 100644 --- a/src/Interpreters/ProcessorsProfileLog.cpp +++ b/src/Interpreters/ProcessorsProfileLog.cpp @@ -30,6 +30,8 @@ ColumnsDescription ProcessorProfileLogElement::getColumnsDescription() {"id", std::make_shared(), "ID of processor."}, {"parent_ids", std::make_shared(std::make_shared()), "Parent processors IDs."}, {"plan_step", std::make_shared(), "ID of the query plan step which created this processor. The value is zero if the processor was not added from any step."}, + {"plan_step_name", std::make_shared(), "Name of the query plan step which created this processor. The value is empty if the processor was not added from any step."}, + {"plan_step_description", std::make_shared(), "Description of the query plan step which created this processor. The value is empty if the processor was not added from any step."}, {"plan_group", std::make_shared(), "Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result."}, {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, @@ -64,6 +66,8 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const } columns[i++]->insert(plan_step); + columns[i++]->insert(plan_step_name); + columns[i++]->insert(plan_step_description); columns[i++]->insert(plan_group); columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size()); diff --git a/src/Interpreters/ProcessorsProfileLog.h b/src/Interpreters/ProcessorsProfileLog.h index abece2604f2..fbf52f45f56 100644 --- a/src/Interpreters/ProcessorsProfileLog.h +++ b/src/Interpreters/ProcessorsProfileLog.h @@ -19,6 +19,8 @@ struct ProcessorProfileLogElement UInt64 plan_step{}; UInt64 plan_group{}; + String plan_step_name; + String plan_step_description; String initial_query_id; String query_id; diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index fc595a7b565..d9bd5074c09 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -9,6 +10,17 @@ namespace DB { +void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group) +{ + query_plan_step = step; + query_plan_step_group = group; + if (step) + { + plan_step_name = step->getName(); + plan_step_description = step->getStepDescription(); + } +} + void IProcessor::cancel() noexcept { diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 02b8a3daa28..fd75eb530aa 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -311,14 +311,12 @@ public: constexpr static size_t NO_STREAM = std::numeric_limits::max(); /// Step of QueryPlan from which processor was created. - void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0) - { - query_plan_step = step; - query_plan_step_group = group; - } + void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0); IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; } size_t getQueryPlanStepGroup() const { return query_plan_step_group; } + const String & getPlanStepName() const { return plan_step_name; } + const String & getPlanStepDescription() const { return plan_step_description; } uint64_t getElapsedNs() const { return elapsed_ns; } uint64_t getInputWaitElapsedNs() const { return input_wait_elapsed_ns; } @@ -410,6 +408,8 @@ private: IQueryPlanStep * query_plan_step = nullptr; size_t query_plan_step_group = 0; + String plan_step_name; + String plan_step_description; }; From 88b72238c860b654657bfe56a10f4b864571bfb2 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Aug 2024 13:09:19 +0000 Subject: [PATCH 1429/1722] Fix JSON type parsing with nulls as defaults --- src/Formats/JSONExtractTree.cpp | 20 +++++++++++++++++++ ...9_json_null_as_default_for_tuple.reference | 1 + .../03229_json_null_as_default_for_tuple.sql | 4 ++++ 3 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference create mode 100644 tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 122224535a7..9ea335ee7fe 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1179,6 +1179,12 @@ public: const FormatSettings & format_settings, String & error) const override { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + auto & tuple = assert_cast(column); size_t old_size = column.size(); bool were_valid_elements = false; @@ -1298,6 +1304,12 @@ public: const FormatSettings & format_settings, String & error) const override { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + if (!element.isObject()) { error = fmt::format("cannot read Map value from JSON element: {}", jsonElementToString(element, format_settings)); @@ -1362,6 +1374,14 @@ public: String & error) const override { auto & column_variant = assert_cast(column); + + /// Check if element is NULL. + if (element.isNull()) + { + column_variant.insertDefault(); + return true; + } + for (size_t i : order) { auto & variant = column_variant.getVariantByGlobalDiscriminator(i); diff --git a/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference new file mode 100644 index 00000000000..fbbba67a2b8 --- /dev/null +++ b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.reference @@ -0,0 +1 @@ +{"a":[["1",{}],[null,{}]]} [(1,'{}'),(NULL,'{}')] Array(Tuple(Nullable(Int64), JSON(max_dynamic_types=16, max_dynamic_paths=256))) diff --git a/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql new file mode 100644 index 00000000000..d405f6232e4 --- /dev/null +++ b/tests/queries/0_stateless/03229_json_null_as_default_for_tuple.sql @@ -0,0 +1,4 @@ +set allow_experimental_json_type=1; + +select materialize('{"a" : [[1, {}], null]}')::JSON as json, getSubcolumn(json, 'a'), dynamicType(getSubcolumn(json, 'a')); + From 2a68ad8704d62b179a123a8fe3355d58450f2b99 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Aug 2024 15:12:01 +0200 Subject: [PATCH 1430/1722] Fill plan_step_name and plan_step_description --- src/Interpreters/executeQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index decc16a3704..d8ceae77d13 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -471,6 +471,8 @@ void logQueryFinish( processor_elem.parent_ids = std::move(parents); processor_elem.plan_step = reinterpret_cast(processor->getQueryPlanStep()); + processor_elem.plan_step_name = processor->getPlanStepName(); + processor_elem.plan_step_description = processor->getPlanStepDescription(); processor_elem.plan_group = processor->getQueryPlanStepGroup(); processor_elem.processor_name = processor->getName(); From 35adf8d12b7ff906a1b846f9547aea792041d476 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 27 Aug 2024 15:22:54 +0200 Subject: [PATCH 1431/1722] add toBool to docs --- .../functions/type-conversion-functions.md | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index edd04580f27..f7fd2d68cf7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -49,6 +49,55 @@ SETTINGS cast_keep_nullable = 1 └──────────────────┴─────────────────────┴──────────────────┘ ``` +## toBool + +Converts an input value to a value of type [`Bool`](../data-types/boolean.md). Throws an exception in case of an error. + +**Syntax** + +```sql +toBool(expr) +``` + +**Arguments** + +- `expr` — Expression returning a number or a string. [Expression](../syntax.md/#syntax-expressions). + +Supported arguments: +- Values of type (U)Int8/16/32/64/128/256. +- Values of type Float32/64. +- Strings `true` or `false` (case-insensitive). + +**Returned value** + +- Returns `true` or `false` based on evaluation of the argument. [Bool](../data-types/boolean.md). + +**Example** + +Query: + +```sql +SELECT + toBool(toUInt8(1)), + toBool(toInt8(-1)), + toBool(toFloat32(1.01)), + toBool('true'), + toBool('false'), + toBool('FALSE') +FORMAT Vertical +``` + +Result: + +```response +toBool(toUInt8(1)): true +toBool(toInt8(-1)): true +toBool(toFloat32(1.01)): true +toBool('true'): true +toBool('false'): false +toBool('FALSE'): false +``` + ## toInt8 Converts an input value to a value of type [`Int8`](../data-types/int-uint.md). Throws an exception in case of an error. From c884a2fb9d15604cbb82f4b41636fa4d15ebb175 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Aug 2024 13:26:20 +0000 Subject: [PATCH 1432/1722] perf tests set cgroups_memory_usage_observer_wait_time to zero --- docker/test/performance-comparison/run.sh | 5 +++-- .../config.d/cgroups_memory_usage_observer_wait_time.xml | 3 +++ tests/performance/scripts/entrypoint.sh | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) create mode 100644 tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml diff --git a/docker/test/performance-comparison/run.sh b/docker/test/performance-comparison/run.sh index 6ef781fa4c8..e40f643e08c 100644 --- a/docker/test/performance-comparison/run.sh +++ b/docker/test/performance-comparison/run.sh @@ -13,7 +13,8 @@ entry="/usr/share/clickhouse-test/performance/scripts/entrypoint.sh" # https://www.kernel.org/doc/Documentation/filesystems/tmpfs.txt # Double-escaped backslashes are a tribute to the engineering wonder of docker -- # it gives '/bin/sh: 1: [bash,: not found' otherwise. -numactl --hardware +echo > compare.log +numactl --hardware | tee -a compare.log node=$(( RANDOM % $(numactl --hardware | sed -n 's/^.*available:\(.*\)nodes.*$/\1/p') )); -echo Will bind to NUMA node $node; +echo Will bind to NUMA node $node | tee -a compare.log numactl --cpunodebind=$node --membind=$node $entry diff --git a/tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml b/tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml new file mode 100644 index 00000000000..a7868b11212 --- /dev/null +++ b/tests/performance/scripts/config/config.d/cgroups_memory_usage_observer_wait_time.xml @@ -0,0 +1,3 @@ + + 0 + diff --git a/tests/performance/scripts/entrypoint.sh b/tests/performance/scripts/entrypoint.sh index db7d96ad150..d392ba7c410 100755 --- a/tests/performance/scripts/entrypoint.sh +++ b/tests/performance/scripts/entrypoint.sh @@ -152,7 +152,7 @@ cat /proc/sys/kernel/core_pattern { time $SCRIPT_DIR/download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \ time stage=configure "$script_path"/compare.sh ; \ -} 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee compare.log +} 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee -a compare.log # Stop the servers to free memory. Normally they are restarted before getting # the profile info, so they shouldn't use much, but if the comparison script From 53bfab4703cd6fd2369c0ad6aa8fe50e1fd77e78 Mon Sep 17 00:00:00 2001 From: Pedro Ferreira Date: Tue, 27 Aug 2024 14:20:31 +0000 Subject: [PATCH 1433/1722] Missing newline --- tests/integration/test_truncate_database/test_distributed.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_truncate_database/test_distributed.py b/tests/integration/test_truncate_database/test_distributed.py index 67dea70b1f4..6b822077a3c 100644 --- a/tests/integration/test_truncate_database/test_distributed.py +++ b/tests/integration/test_truncate_database/test_distributed.py @@ -10,6 +10,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"] ) + def fill_nodes(nodes): for node in nodes: node.query( From b8d5a82f975e4da9be8dff890b2d0caf3832a1b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Aug 2024 14:31:37 +0000 Subject: [PATCH 1434/1722] Fix test --- ..._mutations_and_remove_materialized.reference | 12 ++++-------- ..._mixed_mutations_and_remove_materialized.sql | 17 ++++++++--------- 2 files changed, 12 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference index 9166a82f472..66cf66367c1 100644 --- a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference @@ -1,8 +1,4 @@ -BEFORE a_r1 x String -BEFORE a_r1 y String MATERIALIZED \'str\' -BEFORE a_r2 x String -BEFORE a_r2 y String MATERIALIZED \'str\' -AFTER a_r1 x String -AFTER a_r1 y String -AFTER a_r2 x String -AFTER a_r2 y String +BEFORE a x String +BEFORE a y String MATERIALIZED \'str\' +AFTER a x String +AFTER a y String diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql index 3c43b9a8521..d8ac3280792 100644 --- a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql @@ -1,14 +1,13 @@ -DROP TABLE IF EXISTS a_r1 SYNC; -DROP TABLE IF EXISTS a_r2 SYNC; -CREATE TABLE a_r1 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x; -CREATE TABLE a_r2 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r2') ORDER BY x; +DROP TABLE IF EXISTS a SYNC; +CREATE TABLE a (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x; -INSERT INTO a_r1 SELECT toString(number) FROM numbers(100); -SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; +INSERT INTO a SELECT toString(number) FROM numbers(100); +SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table = 'a' ORDER BY table, name; -ALTER TABLE a_r1 - ADD INDEX IF NOT EXISTS some_index x TYPE set(16) GRANULARITY 1, +-- DROP INDEX is important to make the mutation not a pure metadata mutation +ALTER TABLE a + DROP INDEX IF EXISTS some_index, MODIFY COLUMN y REMOVE MATERIALIZED SETTINGS alter_sync = 2, mutations_sync = 2; -SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; +SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table = 'a' ORDER BY table, name; From 9ae0d5161349bd89a9a185a8f60aaab0e9357f34 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 27 Aug 2024 15:02:02 +0000 Subject: [PATCH 1435/1722] Reformat the code, pass variable names in comments when calling --- src/Core/ExternalTable.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 1f8e63db72e..7fe23f844b6 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -18,11 +18,11 @@ #include #include #include +#include #include #include #include #include -#include "Parsers/IdentifierQuotingStyle.h" namespace DB @@ -88,7 +88,13 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg if (column) structure.emplace_back( column->name, - column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks)); + column->type->formatWithPossiblyHidingSensitiveData( + /*max_length=*/0, + /*one_line=*/true, + /*show_secrets=*/true, + /*print_pretty_type_names=*/false, + /*always_quote_identifiers=*/false, + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -108,7 +114,12 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen structure.emplace_back( "_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData( - 0, true, true, false, false, IdentifierQuotingStyle::Backticks)); + /*max_length=*/0, + /*one_line=*/true, + /*show_secrets=*/true, + /*print_pretty_type_names=*/false, + /*always_quote_identifiers=*/false, + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks)); } void BaseExternalTable::initSampleBlock() From 90cc6199664705c0c0214f60b4cbb246480d372d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 15:06:43 +0000 Subject: [PATCH 1436/1722] Better care about grouping keys order for GROUPING SETS. --- src/Interpreters/Aggregator.h | 12 + src/Interpreters/InterpreterSelectQuery.cpp | 51 ++-- src/Planner/Planner.cpp | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 101 ++++--- src/Processors/QueryPlan/AggregatingStep.h | 19 +- .../QueryPlan/MergingAggregatedStep.cpp | 14 +- .../QueryPlan/MergingAggregatedStep.h | 2 + .../Transforms/MergingAggregatedTransform.cpp | 196 ++++++++++--- .../Transforms/MergingAggregatedTransform.h | 26 +- .../02165_replicated_grouping_sets.reference | 266 ++++++++++++++---- .../02165_replicated_grouping_sets.sql | 23 +- 11 files changed, 517 insertions(+), 194 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index f4f1e9a1df3..2cb04fc7c51 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -59,6 +59,18 @@ class CompiledAggregateFunctionsHolder; class NativeWriter; struct OutputBlockColumns; +struct GroupingSetsParams +{ + GroupingSetsParams() = default; + + GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { } + + Names used_keys; + Names missing_keys; +}; + +using GroupingSetsParamsList = std::vector; + /** How are "total" values calculated with WITH TOTALS? * (For more details, see TotalsHavingTransform.) * diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9e5fffac6e4..ca0e84a5267 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -347,6 +347,27 @@ bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) return false; } +GroupingSetsParamsList getAggregatorGroupingSetsParams(const NamesAndTypesLists & aggregation_keys_list, const Names & all_keys) +{ + GroupingSetsParamsList result; + + for (const auto & aggregation_keys : aggregation_keys_list) + { + NameSet keys; + for (const auto & key : aggregation_keys) + keys.insert(key.name); + + Names missing_keys; + for (const auto & key : all_keys) + if (!keys.contains(key)) + missing_keys.push_back(key); + + result.emplace_back(aggregation_keys.getNames(), std::move(missing_keys)); + } + + return result; +} + } InterpreterSelectQuery::InterpreterSelectQuery( @@ -2005,6 +2026,7 @@ static void executeMergeAggregatedImpl( bool has_grouping_sets, const Settings & settings, const NamesAndTypesList & aggregation_keys, + const NamesAndTypesLists & aggregation_keys_list, const AggregateDescriptions & aggregates, bool should_produce_results_in_order_of_bucket_number, SortDescription group_by_sort_description) @@ -2027,10 +2049,12 @@ static void executeMergeAggregatedImpl( */ Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size, settings.min_hit_rate_to_use_consecutive_keys_optimization); + auto grouping_sets_params = getAggregatorGroupingSetsParams(aggregation_keys_list, keys); auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), params, + grouping_sets_params, final, /// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989) settings.distributed_aggregation_memory_efficient && is_remote_storage && !has_grouping_sets, @@ -2651,30 +2675,6 @@ static Aggregator::Params getAggregatorParams( }; } -static GroupingSetsParamsList getAggregatorGroupingSetsParams(const SelectQueryExpressionAnalyzer & query_analyzer, const Names & all_keys) -{ - GroupingSetsParamsList result; - if (query_analyzer.useGroupingSetKey()) - { - auto const & aggregation_keys_list = query_analyzer.aggregationKeysList(); - - for (const auto & aggregation_keys : aggregation_keys_list) - { - NameSet keys; - for (const auto & key : aggregation_keys) - keys.insert(key.name); - - Names missing_keys; - for (const auto & key : all_keys) - if (!keys.contains(key)) - missing_keys.push_back(key); - - result.emplace_back(aggregation_keys.getNames(), std::move(missing_keys)); - } - } - return result; -} - void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { executeExpression(query_plan, expression, "Before GROUP BY"); @@ -2694,7 +2694,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac settings.group_by_two_level_threshold, settings.group_by_two_level_threshold_bytes); - auto grouping_sets_params = getAggregatorGroupingSetsParams(*query_analyzer, keys); + auto grouping_sets_params = getAggregatorGroupingSetsParams(query_analyzer->aggregationKeysList(), keys); SortDescription group_by_sort_description; SortDescription sort_description_for_merging; @@ -2762,6 +2762,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool has_grouping_sets, context->getSettingsRef(), query_analyzer->aggregationKeys(), + query_analyzer->aggregationKeysList(), query_analyzer->aggregates(), should_produce_results_in_order_of_bucket_number, std::move(group_by_sort_description)); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c0efed8550f..7b5101c5c7d 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -528,6 +528,7 @@ void addMergingAggregatedStep(QueryPlan & query_plan, auto merging_aggregated = std::make_unique( query_plan.getCurrentDataStream(), params, + aggregation_analysis_result.grouping_sets_parameters_list, query_analysis_result.aggregate_final, /// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989) settings.distributed_aggregation_memory_efficient && (is_remote_storage || parallel_replicas_from_merge_tree) && !query_analysis_result.aggregation_with_rollup_or_cube_or_grouping_sets, diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 8a5ed7fde65..a4d707704b1 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -151,6 +151,61 @@ void AggregatingStep::applyOrder(SortDescription sort_description_for_merging_, explicit_sorting_required_for_aggregation_in_order = false; } +ActionsDAG AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG( + const Block & in_header, + const Block & out_header, + const GroupingSetsParamsList & grouping_sets_params, + UInt64 group, + bool group_by_use_nulls) +{ + /// Here we create a DAG which fills missing keys and adds `__grouping_set` column + ActionsDAG dag(in_header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs outputs; + outputs.reserve(out_header.columns() + 1); + + auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, group), 0); + const auto * grouping_node = &dag.addColumn( + {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); + + grouping_node = &dag.materializeNode(*grouping_node); + outputs.push_back(grouping_node); + + const auto & missing_columns = grouping_sets_params[group].missing_keys; + const auto & used_keys = grouping_sets_params[group].used_keys; + + auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr); + for (size_t i = 0; i < out_header.columns(); ++i) + { + const auto & col = out_header.getByPosition(i); + const auto missing_it = std::find_if( + missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; }); + const auto used_it = std::find_if( + used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; }); + if (missing_it != missing_columns.end()) + { + auto column_with_default = col.column->cloneEmpty(); + col.type->insertDefaultInto(*column_with_default); + column_with_default->finalize(); + + auto column = ColumnConst::create(std::move(column_with_default), 0); + const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name}); + node = &dag.materializeNode(*node); + outputs.push_back(node); + } + else + { + const auto * column_node = dag.getOutputs()[in_header.getPositionByName(col.name)]; + if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) + outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name)); + else + outputs.push_back(column_node); + } + } + + dag.getOutputs().swap(outputs); + return dag; +} + void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { QueryPipelineProcessorsCollector collector(pipeline, this); @@ -300,51 +355,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B { const auto & header = ports[set_counter]->getHeader(); - /// Here we create a DAG which fills missing keys and adds `__grouping_set` column - ActionsDAG dag(header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs outputs; - outputs.reserve(output_header.columns() + 1); - - auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0); - const auto * grouping_node = &dag.addColumn( - {ColumnPtr(std::move(grouping_col)), std::make_shared(), "__grouping_set"}); - - grouping_node = &dag.materializeNode(*grouping_node); - outputs.push_back(grouping_node); - - const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; - const auto & used_keys = grouping_sets_params[set_counter].used_keys; - - auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr); - for (size_t i = 0; i < output_header.columns(); ++i) - { - auto & col = output_header.getByPosition(i); - const auto missing_it = std::find_if( - missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; }); - const auto used_it = std::find_if( - used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; }); - if (missing_it != missing_columns.end()) - { - auto column_with_default = col.column->cloneEmpty(); - col.type->insertDefaultInto(*column_with_default); - column_with_default->finalize(); - - auto column = ColumnConst::create(std::move(column_with_default), 0); - const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name}); - node = &dag.materializeNode(*node); - outputs.push_back(node); - } - else - { - const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)]; - if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) - outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name)); - else - outputs.push_back(column_node); - } - } - - dag.getOutputs().swap(outputs); + auto dag = makeCreatingMissingKeysForGroupingSetDAG(header, output_header, grouping_sets_params, set_counter, group_by_use_nulls); auto expression = std::make_shared(std::move(dag), settings.getActionsSettings()); auto transform = std::make_shared(header, expression); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index ae43295024a..4e4078047f1 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -7,18 +7,6 @@ namespace DB { -struct GroupingSetsParams -{ - GroupingSetsParams() = default; - - GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { } - - Names used_keys; - Names missing_keys; -}; - -using GroupingSetsParamsList = std::vector; - Block appendGroupingSetColumn(Block header); Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls); @@ -77,6 +65,13 @@ public: /// Argument input_stream would be the second input (from projection). std::unique_ptr convertToAggregatingProjection(const DataStream & input_stream) const; + static ActionsDAG makeCreatingMissingKeysForGroupingSetDAG( + const Block & in_header, + const Block & out_header, + const GroupingSetsParamsList & grouping_sets_params, + UInt64 group, + bool group_by_use_nulls); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 7207b5e6c7f..f3eb352faac 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -42,6 +42,7 @@ static ITransformingStep::Traits getTraits(bool should_produce_results_in_order_ MergingAggregatedStep::MergingAggregatedStep( const DataStream & input_stream_, Aggregator::Params params_, + GroupingSetsParamsList grouping_sets_params_, bool final_, bool memory_efficient_aggregation_, size_t max_threads_, @@ -56,6 +57,7 @@ MergingAggregatedStep::MergingAggregatedStep( MergingAggregatedTransform::appendGroupingIfNeeded(input_stream_.header, params_.getHeader(input_stream_.header, final_)), getTraits(should_produce_results_in_order_of_bucket_number_)) , params(std::move(params_)) + , grouping_sets_params(std::move(grouping_sets_params_)) , final(final_) , memory_efficient_aggregation(memory_efficient_aggregation_) , max_threads(max_threads_) @@ -94,14 +96,13 @@ void MergingAggregatedStep::applyOrder(SortDescription sort_description, DataStr void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto transform_params = std::make_shared(pipeline.getHeader(), std::move(params), final); - if (memoryBoundMergingWillBeUsed()) { - if (input_streams.front().header.has("__grouping_set")) + if (input_streams.front().header.has("__grouping_set") || !grouping_sets_params.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Memory bound merging of aggregated results is not supported for grouping sets."); + auto transform_params = std::make_shared(pipeline.getHeader(), std::move(params), final); auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), @@ -136,18 +137,19 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c pipeline.resize(1); /// Now merge the aggregated blocks - pipeline.addSimpleTransform([&](const Block & header) - { return std::make_shared(header, transform_params, max_threads); }); + auto transform = std::make_shared(pipeline.getHeader(), params, final, grouping_sets_params, max_threads); + pipeline.addTransform(std::move(transform)); } else { - if (input_streams.front().header.has("__grouping_set")) + if (input_streams.front().header.has("__grouping_set") || !grouping_sets_params.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Memory efficient merging of aggregated results is not supported for grouping sets."); auto num_merge_threads = memory_efficient_merge_threads ? memory_efficient_merge_threads : max_threads; + auto transform_params = std::make_shared(pipeline.getHeader(), std::move(params), final); pipeline.addMergingAggregatedMemoryEfficientTransform(transform_params, num_merge_threads); } diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 654f794d5f5..5c3842a6c33 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -16,6 +16,7 @@ public: MergingAggregatedStep( const DataStream & input_stream_, Aggregator::Params params_, + GroupingSetsParamsList grouping_sets_params_, bool final_, bool memory_efficient_aggregation_, size_t max_threads_, @@ -43,6 +44,7 @@ private: Aggregator::Params params; + GroupingSetsParamsList grouping_sets_params; bool final; bool memory_efficient_aggregation; size_t max_threads; diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 78fb2f340bf..cf383cfcf9d 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -1,7 +1,9 @@ #include #include #include +#include #include +#include #include namespace DB @@ -23,19 +25,93 @@ Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header return out_header; } +MergingAggregatedTransform::~MergingAggregatedTransform() = default; + MergingAggregatedTransform::MergingAggregatedTransform( - Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_) - : IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params_->getHeader())) - , params(std::move(params_)), max_threads(max_threads_), has_grouping_sets(header_.has("__grouping_set")) + Block header_, + Aggregator::Params params, + bool final, + GroupingSetsParamsList grouping_sets_params, + size_t max_threads_) + : IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params.getHeader(header_, final))) + , max_threads(max_threads_) { + if (!grouping_sets_params.empty()) + { + if (!header_.has("__grouping_set")) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot find __grouping_set column in header of MergingAggregatedTransform with grouping sets." + "Header {}", header_.dumpStructure()); + + auto in_header = header_; + in_header.erase(header_.getPositionByName("__grouping_set")); + auto out_header = params.getHeader(header_, final); + + grouping_sets.reserve(grouping_sets_params.size()); + for (const auto & grouping_set_params : grouping_sets_params) + { + size_t group = grouping_sets.size(); + + ActionsDAG reordering(in_header.getColumnsWithTypeAndName()); + auto & outputs = reordering.getOutputs(); + ActionsDAG::NodeRawConstPtrs new_outputs; + new_outputs.reserve(in_header.columns() + grouping_set_params.used_keys.size() - grouping_set_params.used_keys.size()); + + std::unordered_map index; + for (size_t pos = 0; pos < outputs.size(); ++pos) + index.emplace(outputs[pos]->result_name, pos); + + for (const auto & used_name : grouping_set_params.used_keys) + { + auto & idx = index[used_name]; + new_outputs.push_back(outputs[idx]); + } + + for (const auto & used_name : grouping_set_params.used_keys) + index[used_name] = outputs.size(); + for (const auto & missing_name : grouping_set_params.missing_keys) + index[missing_name] = outputs.size(); + + for (const auto * output : outputs) + { + if (index[output->result_name] != outputs.size()) + new_outputs.push_back(output); + } + + outputs.swap(new_outputs); + + Aggregator::Params set_params(grouping_set_params.used_keys, + params.aggregates, + params.overflow_row, + params.max_threads, + params.max_block_size, + params.min_hit_rate_to_use_consecutive_keys_optimization); + + auto transform_params = std::make_shared(reordering.updateHeader(in_header), std::move(set_params), final); + + auto creating = AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG( + transform_params->getHeader(), + out_header, + grouping_sets_params, group, false); + + auto & groupiung_set = grouping_sets.emplace_back(); + groupiung_set.reordering_key_columns_actions = std::make_shared(std::move(reordering)); + groupiung_set.creating_missing_keys_actions = std::make_shared(std::move(creating)); + groupiung_set.params = std::move(transform_params); + } + } + else + { + auto & groupiung_set = grouping_sets.emplace_back(); + groupiung_set.params = std::make_shared(header_, std::move(params), final); + } } void MergingAggregatedTransform::addBlock(Block block) { - if (!has_grouping_sets) + if (grouping_sets.size() == 1) { - auto & bucket_to_blocks = grouping_sets[0]; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + grouping_sets[0].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); return; } @@ -49,13 +125,12 @@ void MergingAggregatedTransform::addBlock(Block block) if (!grouping_column_typed) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName()); - /// Enumerate groups and fill the selector. - std::map enumerated_groups; IColumn::Selector selector; const auto & grouping_data = grouping_column_typed->getData(); size_t num_rows = grouping_data.size(); UInt64 last_group = grouping_data[0]; + UInt64 max_group = last_group; for (size_t row = 1; row < num_rows; ++row) { auto group = grouping_data[row]; @@ -65,32 +140,32 @@ void MergingAggregatedTransform::addBlock(Block block) continue; /// Optimization for single group. - if (enumerated_groups.empty()) - { + if (selector.empty()) selector.reserve(num_rows); - enumerated_groups.emplace(last_group, enumerated_groups.size()); - } /// Fill the last equal range. - selector.resize_fill(row, enumerated_groups[last_group]); - /// Enumerate new group if did not see it before. - enumerated_groups.emplace(group, enumerated_groups.size()); - + selector.resize_fill(row, last_group); last_group = group; + max_group = std::max(last_group, max_group); } + if (max_group >= grouping_sets.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid group number {}. Number of groups {}.", last_group, grouping_sets.size()); + /// Optimization for single group. - if (enumerated_groups.empty()) + if (selector.empty()) { - auto & bucket_to_blocks = grouping_sets[last_group]; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + auto bucket = block.info.bucket_num; + grouping_sets[last_group].reordering_key_columns_actions->execute(block); + grouping_sets[last_group].bucket_to_blocks[bucket].emplace_back(std::move(block)); return; } /// Fill the last equal range. - selector.resize_fill(num_rows, enumerated_groups[last_group]); + selector.resize_fill(num_rows, last_group); - const size_t num_groups = enumerated_groups.size(); + const size_t num_groups = max_group + 1; Blocks splitted_blocks(num_groups); for (size_t group_id = 0; group_id < num_groups; ++group_id) @@ -104,28 +179,28 @@ void MergingAggregatedTransform::addBlock(Block block) splitted_blocks[group_id].getByPosition(col_idx_in_block).column = std::move(splitted_columns[group_id]); } - for (auto [group, group_id] : enumerated_groups) + for (size_t group = 0; group < num_groups; ++group) { - auto & bucket_to_blocks = grouping_sets[group]; - auto & splitted_block = splitted_blocks[group_id]; + auto & splitted_block = splitted_blocks[group]; splitted_block.info = block.info; - bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block)); + grouping_sets[group].reordering_key_columns_actions->execute(splitted_block); + grouping_sets[group].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block)); } } -void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) -{ - auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); - for (auto & block : block_list) - { - auto num_rows = block.rows(); - ColumnWithTypeAndName col; - col.type = std::make_shared(); - col.name = "__grouping_set"; - col.column = ColumnUInt64::create(num_rows, group); - block.insert(grouping_position, std::move(col)); - } -} +// void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) +// { +// auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); +// for (auto & block : block_list) +// { +// auto num_rows = block.rows(); +// ColumnWithTypeAndName col; +// col.type = std::make_shared(); +// col.name = "__grouping_set"; +// col.column = ColumnUInt64::create(num_rows, group); +// block.insert(grouping_position, std::move(col)); +// } +// } void MergingAggregatedTransform::consume(Chunk chunk) { @@ -170,6 +245,25 @@ void MergingAggregatedTransform::consume(Chunk chunk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform."); } +// static void debugBlock(const Block & block) +// { +// std::cerr << block.dumpStructure() << std::endl; +// size_t rows = block.rows(); +// for (size_t row = 0; row < rows; ++row) +// { +// for (size_t col = 0; col < block.columns(); ++col) +// { +// const auto & c = block.getByPosition(col); +// if (c.column->isNumeric()) +// std::cerr << c.column->getUInt(row) << ' '; +// else +// std::cerr << c.column->getDataAt(row).toString() << ' '; + +// } +// std::cerr << std::endl; +// } +// } + Chunk MergingAggregatedTransform::generate() { if (!generate_started) @@ -180,15 +274,31 @@ Chunk MergingAggregatedTransform::generate() /// Exception safety. Make iterator valid in case any method below throws. next_block = blocks.begin(); - for (auto & [group, group_blocks] : grouping_sets) + for (auto & grouping_set : grouping_sets) { - /// TODO: this operation can be made async. Add async for IAccumulatingTransform. + auto & params = grouping_set.params; + auto & bucket_to_blocks = grouping_set.bucket_to_blocks; AggregatedDataVariants data_variants; - params->aggregator.mergeBlocks(std::move(group_blocks), data_variants, max_threads, is_cancelled); + + // std::cerr << "== Group " << group << std::endl; + // for (const auto & [buk, lst] : bucket_to_blocks) + // { + // std::cerr << ".. buk " << buk << std::endl; + // for (const auto & b : lst) + // debugBlock(b); + // } + + /// TODO: this operation can be made async. Add async for IAccumulatingTransform. + params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled); auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); - if (has_grouping_sets) - appendGroupingColumn(group, merged_blocks); + if (grouping_set.creating_missing_keys_actions) + for (auto & block : merged_blocks) + grouping_set.creating_missing_keys_actions->execute(block); + + // std::cerr << "== Merged " << group << std::endl; + // for (const auto & b : merged_blocks) + // debugBlock(b); blocks.splice(blocks.end(), std::move(merged_blocks)); } diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index 1d801f7a94d..3a043ad74b8 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -6,13 +6,24 @@ namespace DB { +class ExpressionActions; +using ExpressionActionsPtr = std::shared_ptr; + /** A pre-aggregate stream of blocks in which each block is already aggregated. * Aggregate functions in blocks should not be finalized so that their states can be merged. */ class MergingAggregatedTransform : public IAccumulatingTransform { public: - MergingAggregatedTransform(Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_); + MergingAggregatedTransform( + Block header_, + Aggregator::Params params_, + bool final_, + GroupingSetsParamsList grouping_sets_params, + size_t max_threads_); + + ~MergingAggregatedTransform() override; + String getName() const override { return "MergingAggregatedTransform"; } static Block appendGroupingIfNeeded(const Block & in_header, Block out_header); @@ -22,13 +33,19 @@ protected: Chunk generate() override; private: - AggregatingTransformParamsPtr params; LoggerPtr log = getLogger("MergingAggregatedTransform"); size_t max_threads; - using GroupingSets = std::unordered_map; + struct GroupingSet + { + Aggregator::BucketToBlocks bucket_to_blocks; + ExpressionActionsPtr reordering_key_columns_actions; + ExpressionActionsPtr creating_missing_keys_actions; + AggregatingTransformParamsPtr params; + }; + + using GroupingSets = std::vector; GroupingSets grouping_sets; - const bool has_grouping_sets; UInt64 total_input_rows = 0; UInt64 total_input_blocks = 0; @@ -40,7 +57,6 @@ private: bool generate_started = false; void addBlock(Block block); - void appendGroupingColumn(UInt64 group, BlocksList & block_list); }; } diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference index 4589dc7d7a5..31cbf2ad670 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.reference +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.reference @@ -11,57 +11,215 @@ 0 6 4 1 10 4 2 14 4 -['.'] -['.','.'] -['.','.','.'] -['.','.','.','.'] -['.','.','.','.','.'] -['.','.','.','.','.','.'] -['.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.','.'] -['.'] -['.'] -['.','.'] -['.','.'] -['.','.','.'] -['.','.','.'] -['.','.','.','.'] -['.','.','.','.'] -['.','.','.','.','.'] -['.','.','.','.','.'] -['.','.','.','.','.','.'] -['.','.','.','.','.','.'] -['.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.','.'] -['.','.','.','.','.','.','.','.','.'] -1 -2 -3 -4 -5 -6 -7 -8 -9 -1 -1 -2 -2 -3 -3 -4 -4 -5 -5 -6 -6 -7 -7 -8 -8 -9 -9 +-- { echo On } + +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 ['.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 ['.'] +1 ['.'] +2 ['.','.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +2 2 +2 3 +2 4 +2 5 +2 6 +2 7 +2 8 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +1 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +1 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 +2 3 +2 3 +2 3 +2 4 +2 4 +2 4 +2 5 +2 5 +2 5 +2 6 +2 6 +2 6 +2 7 +2 7 +2 7 +2 8 +2 8 +2 8 +2 9 +2 9 +2 9 +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 ['.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 ['.'] +2 ['.'] +2 ['.','.'] +2 ['.','.'] +2 ['.','.','.'] +2 ['.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +2 ['.','.','.','.','.','.','.','.','.'] +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 2 +2 3 +2 4 +2 5 +2 6 +2 7 +2 8 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 1 +2 2 +2 2 +2 3 +2 3 +2 4 +2 4 +2 5 +2 5 +2 6 +2 6 +2 7 +2 7 +2 8 +2 8 +2 9 +2 9 +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +2 1 +2 1 +2 1 +2 2 +2 2 +2 2 +2 3 +2 3 +2 3 +2 4 +2 4 +2 4 +2 5 +2 5 +2 5 +2 6 +2 6 +2 6 +2 7 +2 7 +2 7 +2 8 +2 8 +2 8 +2 9 +2 9 +2 9 diff --git a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql index 333dab79575..47d4446f348 100644 --- a/tests/queries/0_stateless/02165_replicated_grouping_sets.sql +++ b/tests/queries/0_stateless/02165_replicated_grouping_sets.sql @@ -44,7 +44,22 @@ ORDER BY sum_value ASC, count_value ASC; -SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; -SELECT arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; -SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; -SELECT toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +set prefer_localhost_replica = 1; + +-- { echo On } + +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; + +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; + +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; + +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; +SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000; From 2b495e22cdeea4c769c73b7e21f448be2c5ffcae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 15:38:52 +0000 Subject: [PATCH 1437/1722] Fixing a test. --- src/Processors/Transforms/MergingAggregatedTransform.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index cf383cfcf9d..dd97364f879 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -111,7 +111,10 @@ void MergingAggregatedTransform::addBlock(Block block) { if (grouping_sets.size() == 1) { - grouping_sets[0].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block)); + auto bucket = block.info.bucket_num; + if (grouping_sets[0].reordering_key_columns_actions) + grouping_sets[0].reordering_key_columns_actions->execute(block); + grouping_sets[0].bucket_to_blocks[bucket].emplace_back(std::move(block)); return; } From aed953bf943eb57feef7ae5a0f2ebf9336543f88 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Aug 2024 16:00:33 +0000 Subject: [PATCH 1438/1722] black --- tests/integration/helpers/postgres_utility.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 4bb68284314..c61c535bd62 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -329,11 +329,15 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") + database_tables = instance.query( + f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" + ) while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") + database_tables = instance.query( + f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" + ) assert table in database_tables From ac91471042ebac5fc5467aef9efe806124293f1a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 27 Aug 2024 16:06:12 +0000 Subject: [PATCH 1439/1722] add comment --- src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 6caf69e3a2c..6f1c3937880 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -232,6 +232,8 @@ std::tuple getTypedNodesForOptimizati const auto & storage_snapshot = table_node->getStorageSnapshot(); auto column = first_argument_column_node->getColumn(); + /// If view source is set we cannot optimize because it doesn't support moving functions to subcolumns. + /// The storage is replaced to the view source but it happens only after building a query tree and applying passes. auto view_source = context->getViewSource(); if (view_source && view_source->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted()) return {}; From a7584bbb80c68917702a31492a5024faf7c2aaf4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 16:07:31 +0000 Subject: [PATCH 1440/1722] Remove comments. --- .../Transforms/MergingAggregatedTransform.cpp | 45 ------------------- 1 file changed, 45 deletions(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index dd97364f879..edd544fb6af 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -191,20 +191,6 @@ void MergingAggregatedTransform::addBlock(Block block) } } -// void MergingAggregatedTransform::appendGroupingColumn(UInt64 group, BlocksList & block_list) -// { -// auto grouping_position = getOutputPort().getHeader().getPositionByName("__grouping_set"); -// for (auto & block : block_list) -// { -// auto num_rows = block.rows(); -// ColumnWithTypeAndName col; -// col.type = std::make_shared(); -// col.name = "__grouping_set"; -// col.column = ColumnUInt64::create(num_rows, group); -// block.insert(grouping_position, std::move(col)); -// } -// } - void MergingAggregatedTransform::consume(Chunk chunk) { if (!consume_started) @@ -248,25 +234,6 @@ void MergingAggregatedTransform::consume(Chunk chunk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform."); } -// static void debugBlock(const Block & block) -// { -// std::cerr << block.dumpStructure() << std::endl; -// size_t rows = block.rows(); -// for (size_t row = 0; row < rows; ++row) -// { -// for (size_t col = 0; col < block.columns(); ++col) -// { -// const auto & c = block.getByPosition(col); -// if (c.column->isNumeric()) -// std::cerr << c.column->getUInt(row) << ' '; -// else -// std::cerr << c.column->getDataAt(row).toString() << ' '; - -// } -// std::cerr << std::endl; -// } -// } - Chunk MergingAggregatedTransform::generate() { if (!generate_started) @@ -283,14 +250,6 @@ Chunk MergingAggregatedTransform::generate() auto & bucket_to_blocks = grouping_set.bucket_to_blocks; AggregatedDataVariants data_variants; - // std::cerr << "== Group " << group << std::endl; - // for (const auto & [buk, lst] : bucket_to_blocks) - // { - // std::cerr << ".. buk " << buk << std::endl; - // for (const auto & b : lst) - // debugBlock(b); - // } - /// TODO: this operation can be made async. Add async for IAccumulatingTransform. params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled); auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads); @@ -299,10 +258,6 @@ Chunk MergingAggregatedTransform::generate() for (auto & block : merged_blocks) grouping_set.creating_missing_keys_actions->execute(block); - // std::cerr << "== Merged " << group << std::endl; - // for (const auto & b : merged_blocks) - // debugBlock(b); - blocks.splice(blocks.end(), std::move(merged_blocks)); } From a1517cb9d6598c6ae7cfef5d574702966ea244a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Aug 2024 16:56:03 +0000 Subject: [PATCH 1441/1722] Refactor a bit and add a comment. --- .../Transforms/MergingAggregatedTransform.cpp | 64 +++++++++++-------- 1 file changed, 37 insertions(+), 27 deletions(-) diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index edd544fb6af..9b76acb8081 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -25,6 +25,42 @@ Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header return out_header; } +/// We should keep the order for GROUPING SET keys. +/// Initiator creates a separate Aggregator for every group, so should we do here. +/// Otherwise, two-level aggregation will split the data into different buckets, +/// and the result may have duplicating rows. +static ActionsDAG makeReorderingActions(const Block & in_header, const GroupingSetsParams & params) +{ + ActionsDAG reordering(in_header.getColumnsWithTypeAndName()); + auto & outputs = reordering.getOutputs(); + ActionsDAG::NodeRawConstPtrs new_outputs; + new_outputs.reserve(in_header.columns() + params.used_keys.size() - params.used_keys.size()); + + std::unordered_map index; + for (size_t pos = 0; pos < outputs.size(); ++pos) + index.emplace(outputs[pos]->result_name, pos); + + for (const auto & used_name : params.used_keys) + { + auto & idx = index[used_name]; + new_outputs.push_back(outputs[idx]); + } + + for (const auto & used_name : params.used_keys) + index[used_name] = outputs.size(); + for (const auto & missing_name : params.missing_keys) + index[missing_name] = outputs.size(); + + for (const auto * output : outputs) + { + if (index[output->result_name] != outputs.size()) + new_outputs.push_back(output); + } + + outputs.swap(new_outputs); + return reordering; +} + MergingAggregatedTransform::~MergingAggregatedTransform() = default; MergingAggregatedTransform::MergingAggregatedTransform( @@ -52,33 +88,7 @@ MergingAggregatedTransform::MergingAggregatedTransform( { size_t group = grouping_sets.size(); - ActionsDAG reordering(in_header.getColumnsWithTypeAndName()); - auto & outputs = reordering.getOutputs(); - ActionsDAG::NodeRawConstPtrs new_outputs; - new_outputs.reserve(in_header.columns() + grouping_set_params.used_keys.size() - grouping_set_params.used_keys.size()); - - std::unordered_map index; - for (size_t pos = 0; pos < outputs.size(); ++pos) - index.emplace(outputs[pos]->result_name, pos); - - for (const auto & used_name : grouping_set_params.used_keys) - { - auto & idx = index[used_name]; - new_outputs.push_back(outputs[idx]); - } - - for (const auto & used_name : grouping_set_params.used_keys) - index[used_name] = outputs.size(); - for (const auto & missing_name : grouping_set_params.missing_keys) - index[missing_name] = outputs.size(); - - for (const auto * output : outputs) - { - if (index[output->result_name] != outputs.size()) - new_outputs.push_back(output); - } - - outputs.swap(new_outputs); + auto reordering = makeReorderingActions(in_header, grouping_set_params); Aggregator::Params set_params(grouping_set_params.used_keys, params.aggregates, From 26e1338c4eaa47cc9d6e0aadb473a184d425f780 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 19:05:10 +0200 Subject: [PATCH 1442/1722] Update runner --- tests/integration/runner | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index cdd75438b87..83c28cff7c7 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -153,7 +153,7 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( """ - iptables is in 'legacy' mode. This is not supported. + iptables on your host machine is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. Or, use --ignore-iptables-legacy-check. @@ -170,7 +170,7 @@ def check_iptables_legacy(): if "legacy" in file_info_str: print( """ - ip6tables is in 'legacy' mode. This is not supported. + ip6tables on your host machine is in 'legacy' mode. This is not supported. Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual. Or, use --ignore-iptables-legacy-check. From ea1289a2b36e80c8662b2ab59b6c13cfbcb40526 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 27 Aug 2024 22:42:14 +0000 Subject: [PATCH 1443/1722] Actually fix false "Killed by signal (output files)" in stress_tests.lib --- tests/docker_scripts/stress_tests.lib | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index 4f3e6eeb2f4..cc4c290afef 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -272,8 +272,10 @@ function check_logs_for_critical_errors() # Remove file no_such_key_errors.txt if it's empty [ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt - # Crash - rg -Fa "###################""#####################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ + # Crash. This must have fewer '#'s than the command below, otherwise the command below will match + # the echo of this command (if set -x is enabled, and this script's stdout is sent + # to /test_output/run.log). + rg -Fa "#######################################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \ && echo -e "Killed by signal (in clickhouse-server.log)$FAIL" >> /test_output/test_results.tsv \ || echo -e "Not crashed$OK" >> /test_output/test_results.tsv @@ -285,7 +287,7 @@ function check_logs_for_critical_errors() # Remove file fatal_messages.txt if it's empty [ -s /test_output/fatal_messages.txt ] || rm /test_output/fatal_messages.txt - rg -Faz "####################""####################" /test_output/* > /dev/null \ + rg -Faz "########################################" /test_output/* > /dev/null \ && echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv function get_gdb_log_context() From ea32862187ade24499f4b441d133952d09dd49aa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 27 Aug 2024 23:10:48 +0000 Subject: [PATCH 1444/1722] no-tsan in 02735_parquet_encoder --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 9320d0e57c3..26d0fefed1b 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel +-- Tags: no-fasttest, no-parallel, no-tsan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 61599108c7b91eaf558c0faaeac33c0928932a6f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Aug 2024 01:37:38 +0200 Subject: [PATCH 1445/1722] Make 01114_database_atomic timings consistent --- tests/queries/0_stateless/01114_database_atomic.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 412f5336224..4f635280666 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -51,8 +51,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='${DATABASE_2}'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" RANDOM_COMMENT="$RANDOM" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition [Using 5 threads in parallel]), result: 110, 5995 +$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do @@ -87,7 +87,7 @@ SELECT count() FROM ${DATABASE_1}.mt " # result: 5 RANDOM_TUPLE="${RANDOM}_tuple" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows) it=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do it=$((it+1)) From e0b9c0f14f6a7e817fddc6eedc5f6bd69b6d9f83 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 28 Aug 2024 03:47:15 +0000 Subject: [PATCH 1446/1722] Remove settings update in ASTTableOverride::formatImpl --- src/Parsers/ASTTableOverrides.cpp | 4 +- src/Parsers/tests/gtest_Parser.cpp | 98 +++++++++++++++--------------- 2 files changed, 50 insertions(+), 52 deletions(-) diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index ccb485f6c69..8352e68b156 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -22,10 +22,8 @@ ASTPtr ASTTableOverride::clone() const return res; } -void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState & state, FormatStateStacked frame) const +void ASTTableOverride::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - FormatSettings settings = settings_; - settings.always_quote_identifiers = true; String nl_or_nothing = settings.one_line ? "" : "\n"; String nl_or_ws = settings.one_line ? " " : "\n"; String hl_keyword = settings.hilite ? hilite_keyword : ""; diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 98cd9682c9c..1b50d4480d7 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -225,55 +225,55 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", - "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE `tbl`\n(\n\n),\nTABLE OVERRIDE `a`\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE `b`\n(\n PARTITION BY rand()\n)" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX `foo` `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", - "CREATE DATABASE db\nTABLE OVERRIDE `t1`\n(\n TTL `inserted` + toIntervalMonth(1)\n),\nTABLE OVERRIDE `t2`\n(\n TTL `inserted` + toIntervalMonth(2)\n)" - }, - { - "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE `tab3`\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n),\nTABLE OVERRIDE `tab5`\n(\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)\nCOMMENT 'db comment'" - } -}))); + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", + "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", + "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" + }, + { + "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" + } + }))); INSTANTIATE_TEST_SUITE_P(ParserCreateUserQuery, ParserTest, ::testing::Combine( From 1f10ba2901ef62f8578e75b9aa982ee94e358141 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Aug 2024 09:16:08 +0200 Subject: [PATCH 1447/1722] Update CHANGELOG.md Remove the server setting just in case, because its applicability was not clarified, and its name is not proper. --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 93dcfa18999..854a95820ab 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -30,7 +30,6 @@ * Support more variants of JOIN strictness (`LEFT/RIGHT SEMI/ANTI/ANY JOIN`) with inequality conditions which involve columns from both left and right table. e.g. `t1.y < t2.y` (see the setting `allow_experimental_join_condition`). [#64281](https://github.com/ClickHouse/ClickHouse/pull/64281) ([lgbo](https://github.com/lgbo-ustc)). * Intrpret Hive-style partitioning for different engines (`File`, `URL`, `S3`, `AzureBlobStorage`, `HDFS`). Hive-style partitioning organizes data into partitioned sub-directories, making it efficient to query and manage large datasets. Currently, it only creates virtual columns with the appropriate name and data. The follow-up PR will introduce the appropriate data filtering (performance speedup). [#65997](https://github.com/ClickHouse/ClickHouse/pull/65997) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Add function `printf` for Spark compatiability (but you can use the existing `format` function). [#66257](https://github.com/ClickHouse/ClickHouse/pull/66257) ([æŽæ‰¬](https://github.com/taiyang-li)). -* Added a new server setting, `disable_insertion_and_mutation`. If it is enabled, the server will deny all insertions and mutations. This includes asynchronous INSERTs. This setting can be used to create read-only replicas. [#66519](https://github.com/ClickHouse/ClickHouse/pull/66519) ([Xu Jia](https://github.com/XuJia0210)). * Add options `restore_replace_external_engines_to_null` and `restore_replace_external_table_functions_to_null` to replace external engines and table_engines to `Null` engine that can be useful for testing. It should work for RESTORE and explicit table creation. [#66536](https://github.com/ClickHouse/ClickHouse/pull/66536) ([Ilya Yatsishin](https://github.com/qoega)). * Added support for reading `MULTILINESTRING` geometry in `WKT` format using function `readWKTLineString`. [#67647](https://github.com/ClickHouse/ClickHouse/pull/67647) ([Jacob Reckhard](https://github.com/jacobrec)). * Add a new table function `fuzzQuery`. This function allows the modification of a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1') LIMIT 5;`. [#67655](https://github.com/ClickHouse/ClickHouse/pull/67655) ([pufit](https://github.com/pufit)). From 8a923d2b21e9becf39656da161e8778411d4dccd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 28 Aug 2024 09:58:50 +0200 Subject: [PATCH 1448/1722] Update 02735_parquet_encoder.sql MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 26d0fefed1b..8e0f7095a42 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-tsan +-- Tags: no-fasttest, no-parallel, no-tsan, no-msan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 0fe8f94b1561ac06370eb9500ac54044065b7df9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Aug 2024 12:50:25 +0200 Subject: [PATCH 1449/1722] Fix missing check --- tests/queries/0_stateless/01114_database_atomic.reference | 3 +++ tests/queries/0_stateless/01114_database_atomic.sh | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 33080a7a77b..572cedb64c5 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -2,6 +2,9 @@ CREATE DATABASE default_1\nENGINE = Atomic CREATE DATABASE default_2\nENGINE = Atomic CREATE DATABASE default_3\nENGINE = Ordinary +default_1 Atomic store 00001114-1000-4000-8000-000000000001 1 +default_2 Atomic store 00001114-1000-4000-8000-000000000002 1 +default_3 Ordinary default_3 default_3 1 110 100 CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 4f635280666..fb4672ef906 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -29,7 +29,7 @@ $CLICKHOUSE_CLIENT -q "SELECT name, engine, splitByChar('/', data_path)[-2], splitByChar('/', metadata_path)[-2] as uuid_path, ((splitByChar('/', metadata_path)[-3] as metadata) = substr(uuid_path, 1, 3)) OR metadata='metadata' - FROM system.databases WHERE name LIKE 'test_01114_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" + FROM system.databases WHERE name LIKE '${CLICKHOUSE_DATABASE}_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE ${DATABASE_1}.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); From d218b04fba8422049ec2a6f73bd646c55cd50c88 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Wed, 28 Aug 2024 11:52:51 +0000 Subject: [PATCH 1450/1722] Fix test_role & test_keeper_s3_snapshot --- tests/integration/test_keeper_s3_snapshot/test.py | 2 +- tests/integration/test_role/test.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 84ffc964621..1e766cb974b 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -92,7 +92,7 @@ def test_s3_upload(started_cluster): # Keeper sends snapshots asynchornously, hence we need to retry. @retry(AssertionError, tries=10, delay=2) def _check_snapshots(): - assert set(get_saved_snapshots()) == set( + assert set(get_saved_snapshots()) >= set( [ "snapshot_50.bin.zstd", "snapshot_100.bin.zstd", diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 225cab975ff..b746af56083 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -629,5 +629,6 @@ def test_roles_cache(): check() instance.query("DROP USER " + ", ".join(users)) - instance.query("DROP ROLE " + ", ".join(roles)) + if roles: + instance.query("DROP ROLE " + ", ".join(roles)) instance.query("DROP TABLE tbl") From 2c6563ee691a7c1a0cee61af815ba037393957d2 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 28 Aug 2024 12:37:28 +0000 Subject: [PATCH 1451/1722] Fix test in tests/integration/test_materialized_mysql_database/materialized_with_ddl.py --- .../test_materialized_mysql_database/materialized_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 86000799ae4..9a99f0c9aa8 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2353,7 +2353,7 @@ def table_overrides(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1001\n") show_db = clickhouse_node.query("SHOW CREATE DATABASE table_overrides") - assert "TABLE OVERRIDE `t1`\\n(\\n\\n)" in show_db, show_db + assert "TABLE OVERRIDE t1\\n(\\n\\n)" in show_db, show_db clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides") mysql_node.query("DROP DATABASE IF EXISTS table_overrides") From 3c6b2a48c379a9aa6f2bb2a6d99c235b27f2c4c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:05:43 +0000 Subject: [PATCH 1452/1722] Prevent specifying properties in `MODIFY COLUMN` queries when using `REMOVE`/`RESET SETTING`/`MODIFY SETTING` --- src/Parsers/ParserAlterQuery.cpp | 34 +++- ...rties_before_remove_modify_reset.reference | 13 ++ ..._properties_before_remove_modify_reset.sql | 169 ++++++++++++++++++ 3 files changed, 212 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference create mode 100644 tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 73fd563faf6..3920f09918a 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -1,6 +1,8 @@ -#include -#include #include + +#include +#include +#include #include #include #include @@ -9,14 +11,19 @@ #include #include #include -#include -#include +#include #include +#include namespace DB { +namespace ErrorCodes +{ +extern const int SYNTAX_ERROR; +} + bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto command = std::make_shared(); @@ -725,8 +732,23 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_modify_col_decl.parse(pos, command_col_decl, expected)) return false; + auto check_no_type = [&](const std::string_view keyword) + { + if (!command_col_decl) + return; + const auto & column_decl = command_col_decl->as(); + + if (!column_decl.children.empty() || column_decl.null_modifier.has_value() || !column_decl.default_specifier.empty() + || column_decl.ephemeral_default || column_decl.primary_key_specifier) + { + throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot specify column properties before '{}'", keyword); + } + }; + if (s_remove.ignore(pos, expected)) { + check_no_type(s_remove.getName()); + if (s_default.ignore(pos, expected)) command->remove_property = toStringView(Keyword::DEFAULT); else if (s_materialized.ignore(pos, expected)) @@ -746,11 +768,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_modify_setting.ignore(pos, expected)) { + check_no_type(s_modify_setting.getName()); + if (!parser_settings.parse(pos, command_settings_changes, expected)) return false; } else if (s_reset_setting.ignore(pos, expected)) { + check_no_type(s_reset_setting.getName()); + if (!parser_reset_setting.parse(pos, command_settings_resets, expected)) return false; } diff --git a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference new file mode 100644 index 00000000000..60c67ceac92 --- /dev/null +++ b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference @@ -0,0 +1,13 @@ +REMOVE +The same, but with type +MODIFY SETTING +The same, but with type +RESET SETTING +The same, but with type +All the above, but on server side +REMOVE +The same, but with type +MODIFY SETTING +The same, but with type +RESET SETTING +The same, but with type diff --git a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql new file mode 100644 index 00000000000..13ad11bb139 --- /dev/null +++ b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql @@ -0,0 +1,169 @@ +DROP TABLE IF EXISTS a SYNC; +CREATE TABLE a (x Int64, y Int64 MATERIALIZED 1 SETTINGS (max_compress_block_size = 30000)) ENGINE = MergeTree ORDER BY x; + + +SELECT 'REMOVE'; +ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COMMENT 5 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y TTL toDate('2025-01-01') + toIntervalDay(x) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COLLATE binary REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } + +SELECT 'MODIFY SETTING'; +ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COMMENT 5 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y TTL toDate('2025-01-01') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COLLATE binary MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } + +SELECT 'RESET SETTING'; +ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COMMENT 5 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y TTL toDate('2025-01-01') + toIntervalDay(x) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COLLATE binary RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } + + + +SELECT 'All the above, but on server side'; + +SELECT 'REMOVE'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT 5 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } + +SELECT 'MODIFY SETTING'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT 5 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } + +SELECT 'RESET SETTING'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT 5 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } From a0fa693f0b56390a8be4b18aef03a9808fcd63db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:20:58 +0000 Subject: [PATCH 1453/1722] Add safety assertion --- src/Parsers/ParserAlterQuery.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 3920f09918a..90e0d0cade0 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -736,6 +736,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected { if (!command_col_decl) return; + const auto & column_decl = command_col_decl->as(); if (!column_decl.children.empty() || column_decl.null_modifier.has_value() || !column_decl.default_specifier.empty() @@ -791,6 +792,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } } command->type = ASTAlterCommand::MODIFY_COLUMN; + + /// Make sure that type is not populated when REMOVE/MODIFY SETTING/RESET SETTING is used, because we wouldn't modify the type, which can be confusing + chassert( + nullptr == command_col_decl->as().type + || (command->remove_property.empty() && nullptr == command_settings_changes && nullptr == command_settings_resets)); } else if (s_modify_order_by.ignore(pos, expected)) { From a842994af121631a055186e7d628650bbe3a3521 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 28 Aug 2024 15:21:49 +0200 Subject: [PATCH 1454/1722] fix false leak detect in libfiu --- contrib/libfiu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libfiu b/contrib/libfiu index b85edbde4cf..a1290d8cd3d 160000 --- a/contrib/libfiu +++ b/contrib/libfiu @@ -1 +1 @@ -Subproject commit b85edbde4cf974b1b40d27828a56f0505f4e2ee5 +Subproject commit a1290d8cd3d7b4541d6c976e0a54f572ac03f2a3 From 0d463b839e2e0c89e61754fc53aa904ae6728e81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:29:41 +0000 Subject: [PATCH 1455/1722] Remove unused parser --- src/Parsers/ParserAlterQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 90e0d0cade0..54caf574e03 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -129,7 +129,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserCompoundIdentifier parser_name; ParserStringLiteral parser_string_literal; ParserStringAndSubstitution parser_string_and_substituion; - ParserIdentifier parser_remove_property; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; ParserStatisticsDeclaration parser_stat_decl; From f76e6ecdaf8c2bb005e3b2712b64db4670120c34 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:30:25 +0000 Subject: [PATCH 1456/1722] Make check more specific --- src/Storages/MutationCommands.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f5ccc80f1d8..8276e9de232 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes { extern const int UNKNOWN_MUTATION_COMMAND; extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN; + extern const int LOGICAL_ERROR; } @@ -115,10 +116,10 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.column_name = getIdentifierName(command->column); return res; } - /// MODIFY COLUMN x REMOVE MATERIALIZED is a valid alter command, but doesn't have any specified column type, thus no mutation is needed + /// MODIFY COLUMN x REMOVE MATERIALIZED/RESET SETTING/MODIFY SETTING is a valid alter command, but doesn't have any specified column type, + /// thus no mutation is needed else if ( - parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->col_decl - && command->col_decl->as().type) + parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->remove_property.empty() && nullptr == command->settings_changes && nullptr == command->settings_resets) { MutationCommand res; res.ast = command->ptr(); From f109c141b0cffd9c27c935774a88a16ad071bac1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:30:41 +0000 Subject: [PATCH 1457/1722] Add safety check --- src/Storages/MutationCommands.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 8276e9de232..75440aeac59 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -125,6 +125,8 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.ast = command->ptr(); res.type = MutationCommand::Type::READ_COLUMN; const auto & ast_col_decl = command->col_decl->as(); + if (nullptr == ast_col_decl.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MODIFY COLUMN mutation command doesn't specify type: {}", serializeAST(*command)); res.column_name = ast_col_decl.name; res.data_type = DataTypeFactory::instance().get(ast_col_decl.type); return res; From 7f613563e1a9a11a2c035488aca058c396c9e05a Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 28 Aug 2024 13:39:29 +0000 Subject: [PATCH 1458/1722] initial impl for cp/mv --- programs/keeper-client/Commands.cpp | 108 ++++++++++++++++++++++++ programs/keeper-client/Commands.h | 28 ++++++ programs/keeper-client/KeeperClient.cpp | 2 + 3 files changed, 138 insertions(+) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 7226bd82df7..b88791ad1d6 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -677,4 +677,112 @@ void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperCl std::cout << totalNumChildren << "\n"; } +namespace +{ + +class CPMVOperation +{ +public: + CPMVOperation(String src_, String dest_, bool remove_src_, KeeperClient * client_) + : src(std::move(src_)), dest(std::move(dest_)), remove_src(remove_src_), client(client_) + { + } + + bool perform() + { + Coordination::Stat src_stat; + String data = client->zookeeper->get(src, &src_stat); + + // allow to copy only persistent nodes + if (src_stat.ephemeralOwner) + throw std::runtime_error("TODO: it is possible to copy only persistent nodes"); + + Coordination::Requests ops{ + zkutil::makeCheckRequest(src, src_stat.version), + zkutil::makeCreateRequest(dest, data, zkutil::CreateMode::Persistent), // Do we need to copy ACLs here? + }; + + if (remove_src) + ops.push_back(zkutil::makeRemoveRequest(src, src_stat.version)); + + Coordination::Responses responses; + auto code = client->zookeeper->tryMultiNoThrow(ops, responses); + + switch (code) + { + case Coordination::Error::ZOK: + return true; + case Coordination::Error::ZBADVERSION: + return false; + case Coordination::Error::ZNODEEXISTS: + throw std::runtime_error("TODO: Destination path already exists"); + default: + zkutil::KeeperMultiException::check(code, ops, responses); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable"); + } + +private: + String src; + String dest; + bool remove_src = false; + KeeperClient * client = nullptr; +}; + +} + +bool CPCommand::parse(IParser::Pos & pos, std::shared_ptr & node, [[maybe_unused]] Expected & expected) const +{ + String src_path; + if (!parseKeeperPath(pos, expected, src_path)) + return false; + node->args.push_back(std::move(src_path)); + + String to_path; + if (!parseKeeperPath(pos, expected, to_path)) + return false; + node->args.push_back(std::move(to_path)); + + return true; +} + +void CPCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto src = client->getAbsolutePath(query->args[0].safeGet()); + auto dest = client->getAbsolutePath(query->args[1].safeGet()); + + CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/false, /*client_=*/client); + + while (!operation.perform()) + ; +} + + +bool MVCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String src_path; + if (!parseKeeperPath(pos, expected, src_path)) + return false; + node->args.push_back(std::move(src_path)); + + String to_path; + if (!parseKeeperPath(pos, expected, to_path)) + return false; + node->args.push_back(std::move(to_path)); + + return true; +} + +void MVCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto src = client->getAbsolutePath(query->args[0].safeGet()); + auto dest = client->getAbsolutePath(query->args[1].safeGet()); + + CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/true, /*client_=*/client); + + while (!operation.perform()) + ; +} + } diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index c6dd731fb3b..686a752b6b6 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -266,4 +266,32 @@ class GetAllChildrenNumberCommand : public IKeeperClientCommand } }; +class CPCommand : public IKeeperClientCommand +{ + String getName() const override { return "cp"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} -- Copies 'src' node to 'dest' path."; + } +}; + +class MVCommand : public IKeeperClientCommand +{ + String getName() const override { return "mv"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} -- Moves 'src' node to the 'dest' path."; + } +}; + } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index ad376d4b88f..97caa142124 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -212,6 +212,8 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), + std::make_shared(), }); String home_path; From 3314042278d938e7faff7029e7b810f64744c7bb Mon Sep 17 00:00:00 2001 From: Rodrigo Garcia Date: Wed, 28 Aug 2024 16:14:31 +0200 Subject: [PATCH 1459/1722] Link to CASE changed after explicit anchor removed --- docs/en/sql-reference/functions/conditional-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/conditional-functions.md b/docs/en/sql-reference/functions/conditional-functions.md index 564186fd8db..8b8ee524c7e 100644 --- a/docs/en/sql-reference/functions/conditional-functions.md +++ b/docs/en/sql-reference/functions/conditional-functions.md @@ -49,7 +49,7 @@ Result: ## multiIf -Allows to write the [CASE](../../sql-reference/operators/index.md#operator_case) operator more compactly in the query. +Allows to write the [CASE](../../sql-reference/operators/index.md#conditional-expression) operator more compactly in the query. **Syntax** @@ -264,4 +264,4 @@ SELECT clamp(1, 2, 3) result, toTypeName(result) type; ┌─result─┬─type────┠│ 2 │ Float64 │ └────────┴─────────┘ -``` \ No newline at end of file +``` From 5b3ca6b2b916b63d12ddb402c0726d0c2e29c1b1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Aug 2024 22:14:43 +0800 Subject: [PATCH 1460/1722] allow unsligned arrays in arrayZip --- src/Functions/array/arrayZip.cpp | 114 +++++++++++++++++++++++-------- 1 file changed, 85 insertions(+), 29 deletions(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 6c6fff5926b..39c04264c84 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -1,7 +1,8 @@ -#include #include -#include +#include +#include #include +#include #include #include #include @@ -12,23 +13,22 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int SIZES_OF_ARRAYS_DONT_MATCH; +extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; +extern const int ILLEGAL_COLUMN; } /// arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']) = [('a', 'd'), ('b', 'e'), ('c', 'f')] +/// arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e']) = [('a', 'd'), ('b', 'e'), ('c', null)] +template class FunctionArrayZip : public IFunction { public: - static constexpr auto name = "arrayZip"; + static constexpr auto name = allow_unaligned ? "arrayZipUnaligned" : "arrayZip"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } + String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -39,8 +39,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function {} needs at least one argument; passed {}." , getName(), arguments.size()); + throw Exception( + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function {} needs at least one argument; passed {}.", + getName(), + arguments.size()); DataTypes arguments_types; for (size_t index = 0; index < arguments.size(); ++index) @@ -48,16 +51,24 @@ public: const DataTypeArray * array_type = checkAndGetDataType(arguments[index].type.get()); if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array. Found {} instead.", - toString(index + 1), getName(), arguments[0].type->getName()); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument {} of function {} must be array. Found {} instead.", + toString(index + 1), + getName(), + arguments[0].type->getName()); - arguments_types.emplace_back(array_type->getNestedType()); + auto nested_type = array_type->getNestedType(); + if constexpr (allow_unaligned) + nested_type = makeNullable(nested_type); + arguments_types.emplace_back(nested_type); } return std::make_shared(std::make_shared(arguments_types)); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { size_t num_arguments = arguments.size(); @@ -68,12 +79,19 @@ public: { /// Constant columns cannot be inside tuple. It's only possible to have constant tuple as a whole. ColumnPtr holder = arguments[i].column->convertToFullColumnIfConst(); - const ColumnArray * column_array = checkAndGetColumn(holder.get()); - if (!column_array) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument {} of function {} must be array. Found column {} instead.", - i + 1, getName(), holder->getName()); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Argument {} of function {} must be array. Found column {} instead.", + i + 1, + getName(), + holder->getName()); + + tuple_columns[i] = column_array->getDataPtr(); + + if constexpr (allow_unaligned) + tuple_columns[i] = makeNullable(tuple_columns[i]); if (i == 0) { @@ -81,23 +99,61 @@ public: } else if (!column_array->hasEqualOffsets(static_cast(*first_array_column))) { - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument {} of function {} have different array sizes", - i + 1, getName()); + if constexpr (allow_unaligned) + return executeUnaligned(static_cast(*first_array_column), *column_array, input_rows_count); + else + throw Exception( + ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, + "The argument 1 and argument {} of function {} have different array sizes", + i + 1, + getName()); } - - tuple_columns[i] = column_array->getDataPtr(); } return ColumnArray::create( - ColumnTuple::create(tuple_columns), static_cast(*first_array_column).getOffsetsPtr()); + ColumnTuple::create(std::move(tuple_columns)), static_cast(*first_array_column).getOffsetsPtr()); + } + +private: + ColumnPtr + executeUnaligned(const ColumnArray & first_array_colmn, const ColumnArray & second_array_column, size_t input_rows_count) const + { + const auto & first_data = first_array_colmn.getDataPtr(); + const auto & second_data = second_array_column.getDataPtr(); + const auto & nullable_first_data = makeNullable(first_data); + const auto & nullable_second_data = makeNullable(second_data); + auto res_first_data = nullable_first_data->cloneEmpty(); + auto res_second_data = nullable_second_data->cloneEmpty(); + auto res_offsets_column = ColumnArray::ColumnOffsets::create(input_rows_count); + auto & res_offsets = assert_cast(*res_offsets_column).getData(); + + const auto & first_offsets = first_array_colmn.getOffsets(); + const auto & second_offsets = second_array_column.getOffsets(); + for (size_t i = 0; i < input_rows_count; ++i) + { + size_t first_size = first_offsets[i] - first_offsets[i - 1]; + size_t second_size = second_offsets[i] - second_offsets[i - 1]; + + res_first_data->insertRangeFrom(*nullable_first_data, first_offsets[i - 1], first_size); + res_second_data->insertRangeFrom(*nullable_second_data, second_offsets[i - 1], second_size); + + if (first_size < second_size) + res_first_data->insertManyDefaults(second_size - first_size); + else if (first_size > second_size) + res_second_data->insertManyDefaults(first_size - second_size); + + res_offsets[i] = std::max(first_size, second_size); + } + + Columns tuple_columns{std::move(res_first_data), std::move(res_second_data)}; + return ColumnArray::create(ColumnTuple::create(std::move(tuple_columns)), std::move(res_offsets_column)); } }; REGISTER_FUNCTION(ArrayZip) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } } - From 644cfb27d66355652afec7762402a0241f283156 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 15:34:34 +0100 Subject: [PATCH 1461/1722] Add RealTimeMicroseconds to HTTP Header X-ClickHouse-Summary --- src/IO/Progress.cpp | 13 +++++++++++++ src/IO/Progress.h | 6 ++++++ src/Interpreters/executeQuery.cpp | 3 +++ 3 files changed, 22 insertions(+) diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index c5bcd0c490a..179c7f7f807 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -91,6 +91,8 @@ void ProgressValues::writeJSON(WriteBuffer & out) const writeText(result_bytes, out); writeCString("\",\"elapsed_ns\":\"", out); writeText(elapsed_ns, out); + writeCString("\",\"real_time_microseconds\":\"", out); + writeText(real_time_microseconds, out); writeCString("\"", out); writeCString("}", out); } @@ -110,6 +112,7 @@ bool Progress::incrementPiecewiseAtomically(const Progress & rhs) result_bytes += rhs.result_bytes; elapsed_ns += rhs.elapsed_ns; + real_time_microseconds += rhs.real_time_microseconds; return rhs.read_rows || rhs.written_rows; } @@ -129,6 +132,7 @@ void Progress::reset() result_bytes = 0; elapsed_ns = 0; + real_time_microseconds = 0; } ProgressValues Progress::getValues() const @@ -148,6 +152,7 @@ ProgressValues Progress::getValues() const res.result_bytes = result_bytes.load(std::memory_order_relaxed); res.elapsed_ns = elapsed_ns.load(std::memory_order_relaxed); + res.real_time_microseconds = real_time_microseconds.load(std::memory_order_relaxed); return res; } @@ -169,6 +174,7 @@ ProgressValues Progress::fetchValuesAndResetPiecewiseAtomically() res.result_bytes = result_bytes.fetch_and(0); res.elapsed_ns = elapsed_ns.fetch_and(0); + res.real_time_microseconds = real_time_microseconds.fetch_and(0); return res; } @@ -190,6 +196,7 @@ Progress Progress::fetchAndResetPiecewiseAtomically() res.result_bytes = result_bytes.fetch_and(0); res.elapsed_ns = elapsed_ns.fetch_and(0); + res.real_time_microseconds = real_time_microseconds.fetch_and(0); return res; } @@ -209,6 +216,7 @@ Progress & Progress::operator=(Progress && other) noexcept result_bytes = other.result_bytes.load(std::memory_order_relaxed); elapsed_ns = other.elapsed_ns.load(std::memory_order_relaxed); + real_time_microseconds = other.real_time_microseconds.load(std::memory_order_relaxed); return *this; } @@ -244,4 +252,9 @@ void Progress::incrementElapsedNs(UInt64 elapsed_ns_) elapsed_ns.fetch_add(elapsed_ns_, std::memory_order_relaxed); } +void Progress::incrementRealTimeMicroseconds(UInt64 microseconds) +{ + real_time_microseconds.fetch_add(microseconds, std::memory_order_relaxed); +} + } diff --git a/src/IO/Progress.h b/src/IO/Progress.h index d0afc9d845f..223496166ec 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -28,6 +28,7 @@ struct ProgressValues UInt64 result_bytes = 0; UInt64 elapsed_ns = 0; + UInt64 real_time_microseconds = 0; void read(ReadBuffer & in, UInt64 server_revision); void write(WriteBuffer & out, UInt64 client_revision) const; @@ -40,6 +41,7 @@ struct ReadProgress UInt64 read_bytes = 0; UInt64 total_rows_to_read = 0; UInt64 total_bytes_to_read = 0; + UInt64 real_time_microseconds = 0; ReadProgress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0, UInt64 total_bytes_to_read_ = 0) : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_), total_bytes_to_read(total_bytes_to_read_) {} @@ -96,6 +98,8 @@ struct Progress std::atomic elapsed_ns {0}; + std::atomic real_time_microseconds {0}; + Progress() = default; Progress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0, UInt64 total_bytes_to_read_ = 0) @@ -125,6 +129,8 @@ struct Progress void incrementElapsedNs(UInt64 elapsed_ns_); + void incrementRealTimeMicroseconds(UInt64 microseconds); + void reset(); ProgressValues getValues() const; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index decc16a3704..fd61cd7f86f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -419,6 +419,9 @@ void logQueryFinish( { Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); + + UInt64 cpu_real_time = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; + p.incrementRealTimeMicroseconds(cpu_real_time); progress_callback(p); } From a1c9cc471d5b8afbc21a4aa6f8d44791c896eb10 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Aug 2024 16:51:45 +0200 Subject: [PATCH 1462/1722] Apply suggestions from code review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- .../AggregateFunctionDistinctJSONPaths.cpp | 20 +++++++++---------- src/Columns/ColumnDynamic.cpp | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 6100bd57515..98996aac2f7 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -49,8 +49,8 @@ struct AggregateFunctionDistinctJSONPathsData /// Iterate over paths in shared data in this row. const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); const auto & shared_data_offsets = column.getSharedDataOffsets(); - size_t start = shared_data_offsets[static_cast(row_num) - 1]; - size_t end = shared_data_offsets[static_cast(row_num)]; + const size_t start = shared_data_offsets[static_cast(row_num) - 1]; + const size_t end = shared_data_offsets[static_cast(row_num)]; for (size_t i = start; i != end; ++i) data.insert(shared_data_paths->getDataAt(i).toString()); } @@ -137,8 +137,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData /// Iterate over paths om shared data in this row and decode the data types. const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); const auto & shared_data_offsets = column.getSharedDataOffsets(); - size_t start = shared_data_offsets[static_cast(row_num) - 1]; - size_t end = shared_data_offsets[static_cast(row_num)]; + const size_t start = shared_data_offsets[static_cast(row_num) - 1]; + const size_t end = shared_data_offsets[static_cast(row_num)]; for (size_t i = start; i != end; ++i) { auto path = shared_data_paths->getDataAt(i).toString(); @@ -146,8 +146,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - if (!isNothing(type)) - data[path].insert(type->getName()); + chassert(!isNothingType(type)); + data[path].insert(type->getName()); } } @@ -172,8 +172,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - if (!isNothing(type)) - data[path].insert(type->getName()); + chassert(!isNothingType(type)); + data[path].insert(type->getName()); } } @@ -200,7 +200,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData size_t paths_size, types_size; readVarUInt(paths_size, buf); if (paths_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size for paths (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); data.reserve(paths_size); String path, type; @@ -209,7 +209,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData readStringBinary(path, buf); readVarUInt(types_size, buf); if (types_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size for types (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); data[path].reserve(types_size); for (size_t j = 0; j != types_size; ++j) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index b5378e983c6..eb85c6b0d08 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -983,7 +983,7 @@ ColumnPtr ColumnDynamic::compress() const String ColumnDynamic::getTypeNameAt(size_t row_num) const { const auto & variant_col = getVariantColumn(); - size_t discr = variant_col.globalDiscriminatorAt(row_num); + const size_t discr = variant_col.globalDiscriminatorAt(row_num); if (discr == ColumnVariant::NULL_DISCRIMINATOR) return ""; From 0aba986372504cbdc6eb28283f51a706ffdb7b2e Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 28 Aug 2024 15:06:05 +0000 Subject: [PATCH 1463/1722] Address review comments --- .../reference/distinctjsonpaths.md | 41 +++++++++++++++++++ .../AggregateFunctionDistinctDynamicTypes.cpp | 2 +- .../AggregateFunctionDistinctJSONPaths.cpp | 8 ++-- src/Columns/ColumnDynamic.cpp | 2 +- src/Columns/ColumnDynamic.h | 2 +- 5 files changed, 48 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md index f916734ca44..d88b2eb024b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md @@ -82,3 +82,44 @@ Result: │ {'a':['Int64'],'b':['Array(Nullable(Int64))','String'],'c.d.e':['Date'],'c.d.f':['Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))']} │ └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` + +**Note** + +If JSON declaration contains paths with specified types, these paths will be always included in the result of `distinctJSONPaths/distinctJSONPathsAndTypes` functions even if input data didn't have values for these paths. + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON(a UInt32)) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"b" : "Hello"}'), ('{"b" : "World", "c" : [1, 2, 3]}'); +``` + +```sql +SELECT json FROM test_json; +``` + +```text +┌─json──────────────────────────────────┠+│ {"a":0,"b":"Hello"} │ +│ {"a":0,"b":"World","c":["1","2","3"]} │ +└───────────────────────────────────────┘ +``` + +```sql +SELECT distinctJSONPaths(json) FROM test_json; +``` + +```text +┌─distinctJSONPaths(json)─┠+│ ['a','b','c'] │ +└─────────────────────────┘ +``` + +```sql +SELECT distinctJSONPathsAndTypes(json) FROM test_json; +``` + +```text +┌─distinctJSONPathsAndTypes(json)────────────────────────────────┠+│ {'a':['UInt32'],'b':['String'],'c':['Array(Nullable(Int64))']} │ +└────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 17e32b20a99..57f7aecd316 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -106,7 +106,7 @@ public: /// In this case we can avoid iterating over all rows because we can get all types /// in Dynamic column in a more efficient way. else - assert_cast(*columns[0]).getAllTypeNames(data(place).data); + assert_cast(*columns[0]).getAllTypeNamesInto(data(place).data); } void addManyDefaults( diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 98996aac2f7..4e60e6fe60b 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -146,7 +146,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - chassert(!isNothingType(type)); + chassert(!isNothing(type)); data[path].insert(type->getName()); } } @@ -160,7 +160,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData /// Add dynamic path only if it has at least one non-null value. /// getNumberOfDefaultRows for Dynamic column is O(1). if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) - dynamic_column->getAllTypeNames(data[path]); + dynamic_column->getAllTypeNamesInto(data[path]); } /// Iterate over all paths in shared data and decode the data types. @@ -172,7 +172,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - chassert(!isNothingType(type)); + chassert(!isNothing(type)); data[path].insert(type->getName()); } } @@ -255,7 +255,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData } }; -/// Calculates the list of distinct data types in Dynamic column. +/// Calculates the list of distinct paths or pairs (path, type) in JSON column. template class AggregateFunctionDistinctJSONPathsAndTypes final : public IAggregateFunctionDataHelper> { diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index eb85c6b0d08..269c8455e2f 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -997,7 +997,7 @@ String ColumnDynamic::getTypeNameAt(size_t row_num) const return variant_info.variant_names[discr]; } -void ColumnDynamic::getAllTypeNames(std::unordered_set & names) const +void ColumnDynamic::getAllTypeNamesInto(std::unordered_set & names) const { auto shared_variant_discr = getSharedVariantDiscriminator(); for (size_t i = 0; i != variant_info.variant_names.size(); ++i) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index c06c31bb8c9..5789b80a2de 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -431,7 +431,7 @@ public: const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } String getTypeNameAt(size_t row_num) const; - void getAllTypeNames(std::unordered_set & names) const; + void getAllTypeNamesInto(std::unordered_set & names) const; private: void createVariantInfo(const DataTypePtr & variant_type); From 41a4dd77058df5c5a9e20af9a510610233dbd695 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 16:09:23 +0100 Subject: [PATCH 1464/1722] Rename variable --- src/Interpreters/executeQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index fd61cd7f86f..2a796004d10 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -420,8 +420,8 @@ void logQueryFinish( Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); - UInt64 cpu_real_time = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; - p.incrementRealTimeMicroseconds(cpu_real_time); + UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; + p.incrementRealTimeMicroseconds(real_time_microseconds); progress_callback(p); } From 4d486f1a74b265ec746bdb87d98b16d4c57e72e4 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 28 Aug 2024 14:53:40 +0000 Subject: [PATCH 1465/1722] update error policy --- programs/keeper-client/Commands.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index b88791ad1d6..2afd55e835e 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -693,10 +693,6 @@ public: Coordination::Stat src_stat; String data = client->zookeeper->get(src, &src_stat); - // allow to copy only persistent nodes - if (src_stat.ephemeralOwner) - throw std::runtime_error("TODO: it is possible to copy only persistent nodes"); - Coordination::Requests ops{ zkutil::makeCheckRequest(src, src_stat.version), zkutil::makeCreateRequest(dest, data, zkutil::CreateMode::Persistent), // Do we need to copy ACLs here? @@ -714,8 +710,6 @@ public: return true; case Coordination::Error::ZBADVERSION: return false; - case Coordination::Error::ZNODEEXISTS: - throw std::runtime_error("TODO: Destination path already exists"); default: zkutil::KeeperMultiException::check(code, ops, responses); } @@ -758,7 +752,6 @@ void CPCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con ; } - bool MVCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String src_path; From 42039331ada8b7b626b279b003acc5f2b972f78e Mon Sep 17 00:00:00 2001 From: okunev Date: Wed, 28 Aug 2024 17:47:43 +0200 Subject: [PATCH 1466/1722] Support for the Spanish language in the embedded dictionaries. --- src/Dictionaries/Embedded/RegionsNames.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/Embedded/RegionsNames.h b/src/Dictionaries/Embedded/RegionsNames.h index 0e4c1fe8b88..73b432fb30d 100644 --- a/src/Dictionaries/Embedded/RegionsNames.h +++ b/src/Dictionaries/Embedded/RegionsNames.h @@ -35,9 +35,10 @@ class RegionsNames M(et, ru, 11) \ M(pt, en, 12) \ M(he, en, 13) \ - M(vi, en, 14) + M(vi, en, 14) \ + M(es, en, 15) - static constexpr size_t total_languages = 15; + static constexpr size_t total_languages = 16; public: enum class Language : size_t From 6b08d2b6de2f22b5972a0b27e6f77ab98344a1da Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 16:51:56 +0100 Subject: [PATCH 1467/1722] Update docs --- docs/en/interfaces/http.md | 12 ++++++------ docs/ru/interfaces/http.md | 10 +++++----- docs/zh/interfaces/http.md | 12 ++++++------ 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 03fdfa048c8..ffd65fce00f 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -58,7 +58,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds": "0"} 1 ``` @@ -472,7 +472,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -668,7 +668,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -708,7 +708,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact
% @@ -766,7 +766,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -785,7 +785,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index d9da51892f9..01d788e62cf 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -50,7 +50,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} 1 ``` @@ -367,7 +367,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0", "elapsed_ns":"662334", "real_time_microseconds":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -601,7 +601,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact
% @@ -659,7 +659,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -678,7 +678,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index f55cf41936f..4767a540d61 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -53,7 +53,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} 1 ``` @@ -363,7 +363,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -524,7 +524,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -564,7 +564,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} < * Connection #0 to host localhost left intact
% @@ -616,7 +616,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -635,7 +635,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} < Relative Path File * Connection #0 to host localhost left intact From ca13b13d811b1999f8e75bf0b1596d37ed92510a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 17:54:42 +0200 Subject: [PATCH 1468/1722] Add CPU arch to short fault info --- src/Common/SignalHandlers.cpp | 4 ++-- src/Common/config_version.cpp.in | 2 ++ src/Common/config_version.h | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 6ac6cbcae29..fc82c50c073 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -371,8 +371,8 @@ try /// in case of double fault. LOG_FATAL(log, "########## Short fault info ############"); - LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, + LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}, architecture: {}) (from thread {}) Received signal {}", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, SYSTEM_PROCESSOR, thread_num, sig); std::string signal_description = "Unknown signal"; diff --git a/src/Common/config_version.cpp.in b/src/Common/config_version.cpp.in index eb9ceb800b9..cc9a7021c6c 100644 --- a/src/Common/config_version.cpp.in +++ b/src/Common/config_version.cpp.in @@ -15,3 +15,5 @@ const char * VERSION_DESCRIBE = "@VERSION_DESCRIBE@"; const unsigned VERSION_INTEGER = @VERSION_INTEGER@; const char * VERSION_GITHASH = "@VERSION_GITHASH@"; + +const char * SYSTEM_PROCESSOR = "@CMAKE_SYSTEM_PROCESSOR@"; diff --git a/src/Common/config_version.h b/src/Common/config_version.h index e3ec12e2b34..4e1ed9388a7 100644 --- a/src/Common/config_version.h +++ b/src/Common/config_version.h @@ -16,6 +16,7 @@ extern const char * VERSION_OFFICIAL; extern const char * VERSION_FULL; extern const char * VERSION_DESCRIBE; extern const unsigned VERSION_INTEGER; +extern const char * SYSTEM_PROCESSOR; /// These fields are frequently changing and we don't want to have them in the header file to allow caching. extern const char * VERSION_GITHASH; From 82eae9f09f58959c2d9727dce66092132168de66 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 28 Aug 2024 16:18:57 +0000 Subject: [PATCH 1469/1722] tune --- tests/ci/integration_tests_runner.py | 2 +- tests/integration/test_async_load_databases/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index a7d5a8c4cf6..96e1ddc9bff 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -33,7 +33,7 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 5 # run whole pytest several times +FLAKY_TRIES_COUNT = 3 # run whole pytest several times FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index f36cff76ea2..94aba46c713 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -182,4 +182,4 @@ def test_multiple_tables(started_cluster): for i in order: assert query(f"select count() from test.table_{i}") == "100\n" for i in range(tables_count): - query(f"drop table test.table_{i}") + query(f"drop table test.table_{i} sync") From b6966f6c88efc7515c8eca0c6fdf5d947e7bfbe7 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 18:26:26 +0200 Subject: [PATCH 1470/1722] Use Poco --- src/Common/SignalHandlers.cpp | 3 ++- src/Common/config_version.cpp.in | 2 -- src/Common/config_version.h | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index fc82c50c073..554a56f22e8 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #pragma clang diagnostic ignored "-Wreserved-identifier" @@ -372,7 +373,7 @@ try LOG_FATAL(log, "########## Short fault info ############"); LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}, architecture: {}) (from thread {}) Received signal {}", - VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, SYSTEM_PROCESSOR, + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, Poco::Environment::osArchitecture(), thread_num, sig); std::string signal_description = "Unknown signal"; diff --git a/src/Common/config_version.cpp.in b/src/Common/config_version.cpp.in index cc9a7021c6c..eb9ceb800b9 100644 --- a/src/Common/config_version.cpp.in +++ b/src/Common/config_version.cpp.in @@ -15,5 +15,3 @@ const char * VERSION_DESCRIBE = "@VERSION_DESCRIBE@"; const unsigned VERSION_INTEGER = @VERSION_INTEGER@; const char * VERSION_GITHASH = "@VERSION_GITHASH@"; - -const char * SYSTEM_PROCESSOR = "@CMAKE_SYSTEM_PROCESSOR@"; diff --git a/src/Common/config_version.h b/src/Common/config_version.h index 4e1ed9388a7..e3ec12e2b34 100644 --- a/src/Common/config_version.h +++ b/src/Common/config_version.h @@ -16,7 +16,6 @@ extern const char * VERSION_OFFICIAL; extern const char * VERSION_FULL; extern const char * VERSION_DESCRIBE; extern const unsigned VERSION_INTEGER; -extern const char * SYSTEM_PROCESSOR; /// These fields are frequently changing and we don't want to have them in the header file to allow caching. extern const char * VERSION_GITHASH; From c96eeb45f0aa0277fd56a8d50c4d3ea67a2070cf Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Wed, 28 Aug 2024 18:47:52 +0200 Subject: [PATCH 1471/1722] Update README.md Adding more meetups --- README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.md b/README.md index ba212852ea8..d12f1a6ff37 100644 --- a/README.md +++ b/README.md @@ -56,6 +56,14 @@ Other upcoming meetups * [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 * [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 * [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 +* [Tel Aviv Meetup](https://www.meetup.com/clickhouse-meetup-israel/events/303095121) - September 22 +* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 +* [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - October 29 +* [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 +* [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 +* [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 + + ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From d6127d5f4966b7fad22073e26ee466d654b7529b Mon Sep 17 00:00:00 2001 From: Konstantin Smirnov <46676677+konnectr@users.noreply.github.com> Date: Wed, 28 Aug 2024 22:30:12 +0500 Subject: [PATCH 1472/1722] add ON CLUSTER --- docs/ru/sql-reference/statements/system.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 3e7d67d90ff..d17e5acd693 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -280,7 +280,7 @@ SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge Ждет когда таблица ÑемейÑтва `ReplicatedMergeTree` будет Ñинхронизирована Ñ Ð´Ñ€ÑƒÐ³Ð¸Ð¼Ð¸ репликами в клаÑтере, но не более `receive_timeout` Ñекунд: ``` sql -SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT [FROM 'srcReplica1'[, 'srcReplica2'[, ...]]] | PULL] +SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT [FROM 'srcReplica1'[, 'srcReplica2'[, ...]]] | PULL] ``` ПоÑле Ð²Ñ‹Ð¿Ð¾Ð»Ð½ÐµÐ½Ð¸Ñ Ñтого запроÑа таблица `[db.]replicated_merge_tree_family_table_name` загружает команды из общего реплицированного лога в Ñвою ÑобÑтвенную очередь репликации. Затем Ð·Ð°Ð¿Ñ€Ð¾Ñ Ð¶Ð´ÐµÑ‚, пока реплика не обработает вÑе загруженные команды. ПоддерживаютÑÑ Ñледующие модификаторы: From 3f9f70d27d7d796ee8d4037a6498f23e812ef125 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 21:10:47 +0200 Subject: [PATCH 1473/1722] Revert "Fix strange trash" This reverts commit aee9612e5127f9cbe0a05c12a08480fc5072ff44. --- src/Core/SettingsFields.cpp | 2 +- src/Core/SettingsFields.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 278b1101c71..86e247722c8 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - return "auto(" + ::DB::toString(value) + ")"; + return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 533d69f3fbb..266141815e3 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -153,7 +153,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto == true`. + /// Writes "auto()" instead of simple "" if `is_auto==true`. String toString() const; void parseFromString(const String & str); From 0dacd16947742be39be53ec72460673438e2e984 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 21:14:04 +0200 Subject: [PATCH 1474/1722] Update SettingsFields.h --- src/Core/SettingsFields.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 266141815e3..533d69f3fbb 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -153,7 +153,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto==true`. + /// Writes "auto()" instead of simple "" if `is_auto == true`. String toString() const; void parseFromString(const String & str); From ab567bad1354edb199baa9c8dc478c1617d6521b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 01:28:27 +0200 Subject: [PATCH 1475/1722] What if I just strip quotes --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 86e247722c8..ca84299ec5b 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -210,7 +210,7 @@ namespace { UInt64 stringToMaxThreads(const String & str) { - if (startsWith(str, "auto")) + if (startsWith(str, "auto") || startsWith(str, "'auto")) return 0; return parseFromString(str); } From 0c23257aa8220ebc8bd9c040088cfb3b26ff6a05 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:09:59 +0200 Subject: [PATCH 1476/1722] Revert "Fix integration test" This reverts commit 0f3506bf7af20231a290ed97a59b0e233fbc7c6e. --- tests/integration/test_cgroup_limit/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index 5d56135d9ff..e77b0f70960 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -46,7 +46,7 @@ def test_cgroup_cpu_limit(): "clickhouse local -q \"select value from system.settings where name='max_threads'\"", num_cpus, ) - expect_output = (r"auto({})".format(math.ceil(num_cpus))).encode() + expect_output = (r"\'auto({})\'".format(math.ceil(num_cpus))).encode() assert ( result.strip() == expect_output ), f"fail for cpu limit={num_cpus}, result={result.strip()}, expect={expect_output}" From 896b1726e756bbc4ff3dd9eb417b00e13263f04e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:11:19 +0200 Subject: [PATCH 1477/1722] Add a comment --- src/Core/SettingsFields.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index ca84299ec5b..47baedfa9db 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,6 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) + /// Removing quotes this will introduce incompatibility between replicas with different version. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 0815a85e2bece8e9b5d22e4217a9a505ff9a231a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 16:15:10 +0200 Subject: [PATCH 1478/1722] Lint --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 47baedfa9db..5c073707b9e 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - /// Removing quotes this will introduce incompatibility between replicas with different version. + /// Removing these quotes this will introduce incompatibility between replicas with different versions. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 30eca0f4a04437cd5edc72a86864cd73bba9a17a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 16:16:09 +0200 Subject: [PATCH 1479/1722] Lint harder --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 5c073707b9e..930eedb8d70 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - /// Removing these quotes this will introduce incompatibility between replicas with different versions. + /// Removing quotes here will introduce an incompatibility between replicas with different versions. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 100c560cb84163a025429d5816d00056bb838adb Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 20:37:49 +0100 Subject: [PATCH 1480/1722] Added RealTimeMicroseconds event --- src/Interpreters/executeQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2a796004d10..682d5bae36e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -90,6 +90,7 @@ namespace ProfileEvents extern const Event SelectQueryTimeMicroseconds; extern const Event InsertQueryTimeMicroseconds; extern const Event OtherQueryTimeMicroseconds; + extern const Event RealTimeMicroseconds; } namespace DB From 2898b123c3337a30946aefb16e057bd13dee2553 Mon Sep 17 00:00:00 2001 From: Samuel Warfield Date: Wed, 28 Aug 2024 14:11:16 -0600 Subject: [PATCH 1481/1722] Update parametric-functions.md Fix link to pattern syntax --- .../aggregate-functions/parametric-functions.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 093d88f939f..47fbb2ea31d 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -104,7 +104,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). +- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). **Returned values** @@ -114,6 +114,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord Type: `UInt8`. + **Pattern syntax** - `(?N)` — Matches the condition argument at position `N`. Conditions are numbered in the `[1, 32]` range. For example, `(?1)` matches the argument passed to the `cond1` parameter. @@ -196,7 +197,7 @@ sequenceCount(pattern)(timestamp, cond1, cond2, ...) **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). +- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). **Returned values** From d390999912919c2ea261fe62619d4fcaae0b0521 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 28 Aug 2024 15:26:06 -0700 Subject: [PATCH 1482/1722] Update tests/queries/0_stateless/02735_parquet_encoder.sql --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 8e0f7095a42..c7e12a970fc 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-tsan, no-msan +-- Tags: long, no-fasttest, no-parallel, no-tsan, no-msan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 91724c29a48b894dabab5587c2f218ead7aaf87a Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 23:26:13 +0100 Subject: [PATCH 1483/1722] Only increase if profile_counters is intialized --- src/Interpreters/executeQuery.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 682d5bae36e..72a6708bb27 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -421,8 +421,12 @@ void logQueryFinish( Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); - UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; - p.incrementRealTimeMicroseconds(real_time_microseconds); + + if (info.profile_counters) + { + UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; + p.incrementRealTimeMicroseconds(real_time_microseconds); + } progress_callback(p); } From 57ba0f0b32ac8d6a092ce7a56b825265fa34dbc1 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 29 Aug 2024 00:31:33 +0000 Subject: [PATCH 1484/1722] format gtest_Parser.cpp --- src/Parsers/tests/gtest_Parser.cpp | 98 +++++++++++++++--------------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 1b50d4480d7..47f7a54389b 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -225,55 +225,55 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", - "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", - "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" - }, - { - "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", - "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" - } - }))); + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", + "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", + "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" + }, + { + "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" + } +}))); INSTANTIATE_TEST_SUITE_P(ParserCreateUserQuery, ParserTest, ::testing::Combine( From 88c99fa0636fc389f13bd6a16dbb83d2e7bfa876 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 08:43:01 +0100 Subject: [PATCH 1485/1722] Do not depend on the response of getInfo for increasing the RealTimeMicroseconds --- src/Interpreters/executeQuery.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 72a6708bb27..4da2ea55066 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -399,9 +399,14 @@ void logQueryFinish( /// Update performance counters before logging to query_log CurrentThread::finalizePerformanceCounters(); - QueryStatusInfo info = process_list_elem->getInfo(true, context->getSettingsRef().log_profile_events); - elem.type = QueryLogElementType::QUERY_FINISH; + std::shared_ptr profile_counters; + QueryStatusInfo info = process_list_elem->getInfo(true, true); + if (context->getSettingsRef().log_profile_events) + profile_counters = info.profile_counters; + else + profile_counters.swap(info.profile_counters); + elem.type = QueryLogElementType::QUERY_FINISH; addStatusInfoToQueryLogElement(elem, info, query_ast, context); if (pulling_pipeline) @@ -420,13 +425,7 @@ void logQueryFinish( { Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); - - - if (info.profile_counters) - { - UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; - p.incrementRealTimeMicroseconds(real_time_microseconds); - } + p.incrementRealTimeMicroseconds((*profile_counters)[ProfileEvents::RealTimeMicroseconds]); progress_callback(p); } From 7d56c8bd43c70f396d4a1e0aa0c8887ca4ee94b9 Mon Sep 17 00:00:00 2001 From: "baolin.hbl" Date: Tue, 20 Aug 2024 03:28:26 +0000 Subject: [PATCH 1486/1722] Avoid detached covered-by-broken part duplicates Problem: When a broken part is found during the startup, it will clone the parts which are covered by the broken part, to the detached directory (with the 'covered-by-broken' prefix). A part may be covered by multiple merged parts, which will result in multiple clones, which leads to path conflicts and further attempts to clone to the try-n directory. If n exceeds 9, the clone is abandoned and the table is marked as read-only. pull#41981 tried fixed the problem, but the fix is incomplete. The metadata_version.txt file is deleted during covered-by-broken clone. As a result, looksLikeBrokenDetachedPartHasTheSameContent finds differences during part comparison. Fix: covered-by-broken retain metadata_version.txt file when cloning --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 + .../test_covered_by_broken_exists/__init__.py | 0 .../test_covered_by_broken_exists/test.py | 103 ++++++++++++++++++ 3 files changed, 104 insertions(+) create mode 100644 tests/integration/test_covered_by_broken_exists/__init__.py create mode 100644 tests/integration/test_covered_by_broken_exists/test.py diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 195aa4fdc10..22bb188f74a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2063,6 +2063,7 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix IDataPartStorage::ClonePartParams params { .copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication, + .keep_metadata_version = prefix == "covered-by-broken", .make_source_readonly = true, .external_transaction = disk_transaction }; diff --git a/tests/integration/test_covered_by_broken_exists/__init__.py b/tests/integration/test_covered_by_broken_exists/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_covered_by_broken_exists/test.py b/tests/integration/test_covered_by_broken_exists/test.py new file mode 100644 index 00000000000..b6d1f55f133 --- /dev/null +++ b/tests/integration/test_covered_by_broken_exists/test.py @@ -0,0 +1,103 @@ +import pytest +import logging +import time +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance("node1", stay_alive=True, with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True) + +instance = node1 +q = node1.query + +path_to_data = "/var/lib/clickhouse/" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def wait_merged_part(table, part_name, retries=100): + q("OPTIMIZE TABLE {} FINAL".format(table)) + for i in range(retries): + result = q( + "SELECT name FROM system.parts where table='{}' AND name='{}'".format( + table, part_name + ) + ) + if result: + return True + time.sleep(0.5) + else: + return False + + +def test_make_clone_covered_by_broken_detached_dir_exists(started_cluster): + q("DROP TABLE IF EXISTS test_make_clone_cvbdde") + + q( + "CREATE TABLE test_make_clone_cvbdde(n int, m String) ENGINE=ReplicatedMergeTree('/test_make_clone_cvbdde', '1') ORDER BY n SETTINGS old_parts_lifetime=3600, min_age_to_force_merge_seconds=1, min_age_to_force_merge_on_partition_only=0" + ) + path = path_to_data + "data/default/test_make_clone_cvbdde/" + + q("INSERT INTO test_make_clone_cvbdde VALUES (0, 'hbl')") + + q("INSERT INTO test_make_clone_cvbdde VALUES (1, 'hbl')") + if not (wait_merged_part("test_make_clone_cvbdde", "all_0_1_1")): + assert False, "Part all_0_1_1 doesn't appeared in system.parts" + + q("INSERT INTO test_make_clone_cvbdde VALUES (2, 'hbl')") + if not (wait_merged_part("test_make_clone_cvbdde", "all_0_2_2")): + assert False, "Part all_0_2_2 doesn't appeared in system.parts" + + q("INSERT INTO test_make_clone_cvbdde VALUES (3, 'hbl')") + if not (wait_merged_part("test_make_clone_cvbdde", "all_0_3_3")): + assert False, "Part all_0_3_3 doesn't appeared in system.parts" + + res = str(instance.exec_in_container(["ls", path]).strip().split("\n")) + + # broke the merged parts + instance.exec_in_container( + [ + "bash", + "-c", + "echo 'broken' > {}".format(path + "all_0_1_1/data.bin"), + ] + ) + + instance.exec_in_container( + [ + "bash", + "-c", + "echo 'broken' > {}".format(path + "all_0_2_2/data.bin"), + ] + ) + + instance.exec_in_container( + [ + "bash", + "-c", + "echo 'broken' > {}".format(path + "all_0_3_3/data.bin"), + ] + ) + + instance.restart_clickhouse(kill=True) + + assert [ + "broken-on-start_all_0_1_1", + "broken-on-start_all_0_2_2", + "broken-on-start_all_0_3_3", + "covered-by-broken_all_0_0_0", + "covered-by-broken_all_1_1_0", + "covered-by-broken_all_2_2_0", + "covered-by-broken_all_3_3_0", + ] == sorted( + instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") + ) From 920a1b2801a8395696e5ae48231e140c6af7e73c Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 11:39:29 +0100 Subject: [PATCH 1487/1722] Added test to validate that the value is being populated --- .../03228_url_engine_response_headers.reference | 1 + .../0_stateless/03228_url_engine_response_headers.sql | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.reference b/tests/queries/0_stateless/03228_url_engine_response_headers.reference index f28952972b8..199af6ed2ee 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.reference +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.reference @@ -1,2 +1,3 @@ Map(LowCardinality(String), LowCardinality(String)) 1 1 +3 100 1 \ No newline at end of file diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql index ff8e47611f4..95e8cd1865d 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.sql +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -5,3 +5,11 @@ SELECT *, mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'] FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); + +-- The real_time_microseconds is not available in the `X-ClickHouse-Progress` header (it is available in the `X-ClickHouse-Summary` header). +-- We need to wait until the query is finished to get the real_time_microseconds. +SELECT + *, + mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'], + toUInt64OrDefault(mapFromString(_headers['X-ClickHouse-Summary'])['real_time_microseconds']) >= 0 ? 1 : 0 +FROM url('http://127.0.0.1:8123/?query=SELECT%20uniq%28number%253%29%20FROM%20numbers%28100%29&user=default&wait_end_of_query=1', LineAsString, 's String'); From d88aa3952d34eeecbca363a1a66df111c5b3e587 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 29 Aug 2024 10:56:26 +0000 Subject: [PATCH 1488/1722] Disable alter table add vector similarity index if setting does not enabled --- src/Storages/AlterCommands.cpp | 10 ++++++++++ src/Storages/AlterCommands.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 5 +++++ ...rbid_add_vector_similarity_index.reference | 0 ...231_forbid_add_vector_similarity_index.sql | 19 +++++++++++++++++++ 5 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference create mode 100644 tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d92d8b59f6e..e0563f9f1c6 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1142,6 +1142,16 @@ bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata) return false; } +bool AlterCommands::hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata) +{ + for (const auto & index : metadata.secondary_indices) + { + if (index.type == "vector_similarity") + return true; + } + return false; +} + void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const { if (!prepared) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index a91bac10214..b2f0f9a6abd 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -237,6 +237,8 @@ public: /// Check if commands have any full-text index static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); + + static bool hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata); }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 94f6d196b99..8b12330c1a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3230,6 +3230,11 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')"); + if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings.allow_experimental_vector_similarity_index) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental vector_similarity index feature is not enabled (turn on setting 'allow_experimental_vector_similarity_index')"); + for (const auto & disk : getDisks()) if (!disk->supportsHardLinks() && !commands.isSettingsAlter() && !commands.isCommentAlter()) throw Exception( diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql new file mode 100644 index 00000000000..a1e362e7bd1 --- /dev/null +++ b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS test_embedding; + +CREATE TABLE test_embedding +( + id UInt32, + embedding Array(Float32), +) +ENGINE = MergeTree +ORDER BY tuple(); + +SET allow_experimental_vector_similarity_index = 0; + +alter table test_embedding add INDEX idx embedding TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError SUPPORT_IS_DISABLED } + +SET allow_experimental_vector_similarity_index = 1; + +alter table test_embedding add INDEX idx embedding TYPE vector_similarity('hnsw', 'cosineDistance'); + +DROP TABLE test_embedding; From e9d806ea34bc18af65c15c65ff0bf5dcdf0b1e46 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 12:10:18 +0100 Subject: [PATCH 1489/1722] Added missing breakline --- .../0_stateless/03228_url_engine_response_headers.reference | 2 +- tests/queries/0_stateless/03228_url_engine_response_headers.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.reference b/tests/queries/0_stateless/03228_url_engine_response_headers.reference index 199af6ed2ee..84b499f5a8c 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.reference +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.reference @@ -1,3 +1,3 @@ Map(LowCardinality(String), LowCardinality(String)) 1 1 -3 100 1 \ No newline at end of file +3 100 1 diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql index 95e8cd1865d..20fcec41030 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.sql +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -11,5 +11,5 @@ FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's SELECT *, mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'], - toUInt64OrDefault(mapFromString(_headers['X-ClickHouse-Summary'])['real_time_microseconds']) >= 0 ? 1 : 0 + toUInt64OrZero(mapFromString(_headers['X-ClickHouse-Summary'])['real_time_microseconds']) >= 0 ? 1 : 0 FROM url('http://127.0.0.1:8123/?query=SELECT%20uniq%28number%253%29%20FROM%20numbers%28100%29&user=default&wait_end_of_query=1', LineAsString, 's String'); From d533c24321a8a1c8d7a937bbce6756e91a2230b8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:14:43 +0200 Subject: [PATCH 1490/1722] Reduce test size --- .../0_stateless/03227_distinct_dynamic_types_json_paths.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index 6930f5a3d44..3af911e27fa 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -4,10 +4,11 @@ set allow_experimental_dynamic_type = 1; set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; +set max_block_size = 10000; drop table if exists test_json_dynamic_aggregate_functions; create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; -insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(200000); +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(100000); select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; From 653b0802c0cd47a832d2b52617be7dfe0c30a70c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:17:27 +0200 Subject: [PATCH 1491/1722] init --- src/Storages/StorageFile.cpp | 1 + .../0_stateless/03232_file_path_normalizing.reference | 1 + tests/queries/0_stateless/03232_file_path_normalizing.sh | 7 +++++++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03232_file_path_normalizing.reference create mode 100644 tests/queries/0_stateless/03232_file_path_normalizing.sh diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 50294df32a4..036a01914cf 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -126,6 +126,7 @@ void listFilesWithRegexpMatchingImpl( /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. fs::canonical(path_for_ls + for_match); fs::path absolute_path = fs::absolute(path_for_ls + for_match); + absolute_path = absolute_path.lexically_normal(); /// ensure that the resulting path is normalized (e.g., removes any redundant slashes or . and .. segments) result.push_back(absolute_path.string()); } catch (const std::exception &) // NOLINT diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference new file mode 100644 index 00000000000..fe3792e5062 --- /dev/null +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -0,0 +1 @@ +/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh new file mode 100644 index 00000000000..6c3c12a1013 --- /dev/null +++ b/tests/queries/0_stateless/03232_file_path_normalizing.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "SELECT _path FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" From 96a2685f91dc6b75c2cafd39945502e92a4a5a9f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:24:23 +0200 Subject: [PATCH 1492/1722] empty commit From b6a6d9315217a783f61671e9a0c0ef997e1cd7e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 29 Aug 2024 11:28:56 +0000 Subject: [PATCH 1493/1722] Remove completely invalid queries from test --- ...lter_no_properties_before_remove_modify_reset.sql | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql index 13ad11bb139..0b98c605ccf 100644 --- a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql +++ b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql @@ -4,7 +4,6 @@ CREATE TABLE a (x Int64, y Int64 MATERIALIZED 1 SETTINGS (max_compress_block_siz SELECT 'REMOVE'; ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } -ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } @@ -17,7 +16,6 @@ ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } SELECT 'The same, but with type'; -ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } @@ -31,7 +29,6 @@ ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED; -- { client SELECT 'MODIFY SETTING'; ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } -ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } @@ -44,7 +41,6 @@ ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_com ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } SELECT 'The same, but with type'; -ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } @@ -58,7 +54,6 @@ ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_bloc SELECT 'RESET SETTING'; ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } -ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } @@ -71,7 +66,6 @@ ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_comp ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } SELECT 'The same, but with type'; -ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } @@ -89,7 +83,6 @@ SELECT 'All the above, but on server side'; SELECT 'REMOVE'; SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } @@ -102,7 +95,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_s SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT 'The same, but with type'; -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } @@ -116,7 +108,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATER SELECT 'MODIFY SETTING'; SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } @@ -129,7 +120,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MO SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT 'The same, but with type'; -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } @@ -143,7 +133,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTI SELECT 'RESET SETTING'; SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } @@ -156,7 +145,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RE SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT 'The same, but with type'; -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } From 34ce43904309f566e216722c2169f8ccc07dc4a0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 12:04:40 +0000 Subject: [PATCH 1494/1722] chmod +x on test --- tests/queries/0_stateless/03232_file_path_normalizing.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03232_file_path_normalizing.sh diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh old mode 100644 new mode 100755 From 97be458b58a181207ffdfd52c2e99c6745a410aa Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 12:05:15 +0000 Subject: [PATCH 1495/1722] add no-fasttest tag --- tests/queries/0_stateless/03232_file_path_normalizing.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh index 6c3c12a1013..eeaa1f2014d 100755 --- a/tests/queries/0_stateless/03232_file_path_normalizing.sh +++ b/tests/queries/0_stateless/03232_file_path_normalizing.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8bbb78375e52c3ae16531de5c96b9eb6bb365b59 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 12:18:51 +0000 Subject: [PATCH 1496/1722] add try limit --- programs/keeper-client/Commands.cpp | 39 +++++++++++++++++++++++------ 1 file changed, 32 insertions(+), 7 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 2afd55e835e..2e946ccbf11 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -682,13 +682,25 @@ namespace class CPMVOperation { + constexpr static UInt64 kTryLimit = 1000; + public: CPMVOperation(String src_, String dest_, bool remove_src_, KeeperClient * client_) : src(std::move(src_)), dest(std::move(dest_)), remove_src(remove_src_), client(client_) { } - bool perform() + bool isTryLimitReached() const + { + return failed_tries_count >= kTryLimit; + } + + bool isCompleted() const + { + return is_completed; + } + + void perform() { Coordination::Stat src_stat; String data = client->zookeeper->get(src, &src_stat); @@ -707,9 +719,19 @@ public: switch (code) { case Coordination::Error::ZOK: - return true; + { + is_completed = true; + return; + } case Coordination::Error::ZBADVERSION: - return false; + { + ++failed_tries_count; + + if (isTryLimitReached()) + zkutil::KeeperMultiException::check(code, ops, responses); + + return; + } default: zkutil::KeeperMultiException::check(code, ops, responses); } @@ -722,6 +744,9 @@ private: String dest; bool remove_src = false; KeeperClient * client = nullptr; + + bool is_completed = false; + uint64_t failed_tries_count = 0; }; } @@ -748,8 +773,8 @@ void CPCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/false, /*client_=*/client); - while (!operation.perform()) - ; + while (!operation.isTryLimitReached() && !operation.isCompleted()) + operation.perform(); } bool MVCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const @@ -774,8 +799,8 @@ void MVCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/true, /*client_=*/client); - while (!operation.perform()) - ; + while (!operation.isTryLimitReached() && !operation.isCompleted()) + operation.perform(); } } From e9497cdcbcb62612dfedbf6f10ae98e9d7d8870d Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 12:20:42 +0000 Subject: [PATCH 1497/1722] use tryMulti instead of NoThrow --- programs/keeper-client/Commands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 2e946ccbf11..0025feacb4b 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -714,7 +714,7 @@ public: ops.push_back(zkutil::makeRemoveRequest(src, src_stat.version)); Coordination::Responses responses; - auto code = client->zookeeper->tryMultiNoThrow(ops, responses); + auto code = client->zookeeper->tryMulti(ops, responses); switch (code) { From 0400dcb03eefdb6604d04f5ed3c70c179032f84d Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 29 Aug 2024 12:34:58 +0000 Subject: [PATCH 1498/1722] no-fastest --- .../0_stateless/03231_forbid_add_vector_similarity_index.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql index a1e362e7bd1..e91d7c71eac 100644 --- a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql +++ b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + DROP TABLE IF EXISTS test_embedding; CREATE TABLE test_embedding From cd9be01c65767ff88131c110b5a2daf0a663034e Mon Sep 17 00:00:00 2001 From: Aleksa Cukovic Date: Thu, 29 Aug 2024 14:40:38 +0200 Subject: [PATCH 1499/1722] Fix row policy documentation grammar --- .../statements/create/row-policy.md | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index cd7718793bd..8be766710fd 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -8,7 +8,7 @@ title: "CREATE ROW POLICY" Creates a [row policy](../../../guides/sre/user-management/index.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table. :::tip -Row policies makes sense only for users with readonly access. If user can modify table or copy partitions between tables, it defeats the restrictions of row policies. +Row policies make sense only for users with readonly access. If a user can modify a table or copy partitions between tables, it defeats the restrictions of row policies. ::: Syntax: @@ -24,40 +24,40 @@ CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluste ## USING Clause -Allows to specify a condition to filter rows. An user will see a row if the condition is calculated to non-zero for the row. +Allows specifying a condition to filter rows. A user will see a row if the condition is calculated to non-zero for the row. ## TO Clause -In the section `TO` you can provide a list of users and roles this policy should work for. For example, `CREATE ROW POLICY ... TO accountant, john@localhost`. +In the `TO` section you can provide a list of users and roles this policy should work for. For example, `CREATE ROW POLICY ... TO accountant, john@localhost`. -Keyword `ALL` means all the ClickHouse users including current user. Keyword `ALL EXCEPT` allow to exclude some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` +Keyword `ALL` means all the ClickHouse users, including current user. Keyword `ALL EXCEPT` allows excluding some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` :::note -If there are no row policies defined for a table then any user can `SELECT` all the row from the table. Defining one or more row policies for the table makes the access to the table depending on the row policies no matter if those row policies are defined for the current user or not. For example, the following policy +If there are no row policies defined for a table, then any user can `SELECT` all the rows from the table. Defining one or more row policies for the table makes access to the table dependent on the row policies, no matter if those row policies are defined for the current user or not. For example, the following policy: `CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter` -forbids the users `mira` and `peter` to see the rows with `b != 1`, and any non-mentioned user (e.g., the user `paul`) will see no rows from `mydb.table1` at all. +forbids the users `mira` and `peter` from seeing the rows with `b != 1`, and any non-mentioned user (e.g., the user `paul`) will see no rows from `mydb.table1` at all. -If that's not desirable it can't be fixed by adding one more row policy, like the following: +If that's not desirable, it can be fixed by adding one more row policy, like the following: `CREATE ROW POLICY pol2 ON mydb.table1 USING 1 TO ALL EXCEPT mira, peter` ::: ## AS Clause -It's allowed to have more than one policy enabled on the same table for the same user at the one time. So we need a way to combine the conditions from multiple policies. +It's allowed to have more than one policy enabled on the same table for the same user at one time. So we need a way to combine the conditions from multiple policies. -By default policies are combined using the boolean `OR` operator. For example, the following policies +By default, policies are combined using the boolean `OR` operator. For example, the following policies: ``` sql CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 TO peter, antonio ``` -enables the user `peter` to see rows with either `b=1` or `c=2`. +enable the user `peter` to see rows with either `b=1` or `c=2`. -The `AS` clause specifies how policies should be combined with other policies. Policies can be either permissive or restrictive. By default policies are permissive, which means they are combined using the boolean `OR` operator. +The `AS` clause specifies how policies should be combined with other policies. Policies can be either permissive or restrictive. By default, policies are permissive, which means they are combined using the boolean `OR` operator. A policy can be defined as restrictive as an alternative. Restrictive policies are combined using the boolean `AND` operator. @@ -68,25 +68,25 @@ row_is_visible = (one or more of the permissive policies' conditions are non-zer (all of the restrictive policies's conditions are non-zero) ``` -For example, the following policies +For example, the following policies: ``` sql CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 AS RESTRICTIVE TO peter, antonio ``` -enables the user `peter` to see rows only if both `b=1` AND `c=2`. +enable the user `peter` to see rows only if both `b=1` AND `c=2`. Database policies are combined with table policies. -For example, the following policies +For example, the following policies: ``` sql CREATE ROW POLICY pol1 ON mydb.* USING b=1 TO mira, peter CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 AS RESTRICTIVE TO peter, antonio ``` -enables the user `peter` to see table1 rows only if both `b=1` AND `c=2`, although +enable the user `peter` to see table1 rows only if both `b=1` AND `c=2`, although any other table in mydb would have only `b=1` policy applied for the user. From f06678ee6cb1a0ddf6b8eea74702b6f7575f897d Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 13:12:37 +0000 Subject: [PATCH 1500/1722] add test --- .../03230_keeper_cp_mv_commands.reference | 15 ++++++++ .../03230_keeper_cp_mv_commands.sh | 38 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference create mode 100755 tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference new file mode 100644 index 00000000000..15f79fdebc7 --- /dev/null +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference @@ -0,0 +1,15 @@ +initial +A C +simple copy +A C D +node-A +simple move +A C H +node-A +move node with childs -- must be error +Transaction failed (Not empty): Op #2, path: /test-keeper-client-cp-mv-commands/A +A C H +move node to existing +Transaction failed (Node exists): Op #1, path: /test-keeper-client-cp-mv-commands/A +A C H +clean up diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh new file mode 100755 index 00000000000..15e117c7ca6 --- /dev/null +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +path="/test-keeper-client-cp-mv-commands" + +$CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null + +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'root'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/A' 'node-A'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/A/B' 'node-B'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/C' 'node-B'" + +echo 'initial' +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" + +echo 'simple copy' +$CLICKHOUSE_KEEPER_CLIENT -q "cp '$path/A' '$path/D'" +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" +$CLICKHOUSE_KEEPER_CLIENT -q "get '$path/D'" + +echo 'simple move' +$CLICKHOUSE_KEEPER_CLIENT -q "mv '$path/D' '$path/H'" +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" +$CLICKHOUSE_KEEPER_CLIENT -q "get '$path/H'" + +echo 'move node with childs -- must be error' +$CLICKHOUSE_KEEPER_CLIENT -q "mv '$path/A' '$path/ERROR'" 2>&1 +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" + +echo 'move node to existing' +$CLICKHOUSE_KEEPER_CLIENT -q "mv '$path/C' '$path/A'" 2>&1 +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" + +echo 'clean up' +$CLICKHOUSE_KEEPER_CLIENT -q "rmr '$path'" From d8f0ce92affe216a054e81fa0ea51e419bd0b81c Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 13:28:00 +0000 Subject: [PATCH 1501/1722] fix style --- programs/keeper-client/Commands.cpp | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 0025feacb4b..4ad2eb31e6d 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -690,15 +690,9 @@ public: { } - bool isTryLimitReached() const - { - return failed_tries_count >= kTryLimit; - } + bool isTryLimitReached() const { return failed_tries_count >= kTryLimit; } - bool isCompleted() const - { - return is_completed; - } + bool isCompleted() const { return is_completed; } void perform() { @@ -718,13 +712,11 @@ public: switch (code) { - case Coordination::Error::ZOK: - { + case Coordination::Error::ZOK: { is_completed = true; return; } - case Coordination::Error::ZBADVERSION: - { + case Coordination::Error::ZBADVERSION: { ++failed_tries_count; if (isTryLimitReached()) From 13311bd6665a48cc73ea8c6dbabfa1ab1f6af338 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 16:02:06 +0200 Subject: [PATCH 1502/1722] fix test --- tests/queries/0_stateless/03232_file_path_normalizing.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference index fe3792e5062..d7dc12010f5 100644 --- a/tests/queries/0_stateless/03232_file_path_normalizing.reference +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -1 +1 @@ -/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet +/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet From 36725fb5e149ed532aca6c290585dec444ed03b4 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 15:05:19 +0100 Subject: [PATCH 1503/1722] Improved comment to force CI to re-run --- tests/queries/0_stateless/03228_url_engine_response_headers.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql index 20fcec41030..41a1a2406da 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.sql +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -6,7 +6,7 @@ SELECT mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'] FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); --- The real_time_microseconds is not available in the `X-ClickHouse-Progress` header (it is available in the `X-ClickHouse-Summary` header). +-- The real_time_microseconds is not available in the `X-ClickHouse-Progress` header (it is only available in the `X-ClickHouse-Summary` header). -- We need to wait until the query is finished to get the real_time_microseconds. SELECT *, From 4499f20ca8af46f88da72f6eaeb2880ded348efa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Aug 2024 14:38:59 +0000 Subject: [PATCH 1504/1722] better code in AsynchronousInsertQueue --- src/Interpreters/AsynchronousInsertQueue.cpp | 70 ++++++++------------ src/Interpreters/AsynchronousInsertQueue.h | 7 -- 2 files changed, 27 insertions(+), 50 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 5ed0d6d6257..c619ea80c7c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -805,12 +805,10 @@ try if (async_insert_log) log_elements.reserve(data->entries.size()); - auto add_entry_to_asynchronous_insert_log = [&]( + auto add_entry_to_asynchronous_insert_log = [&, query_by_format = NameToNameMap{}]( const InsertData::EntryPtr & entry, - const NameToNameMap & query_by_format, const String & parsing_exception, - size_t num_rows, - size_t num_bytes) + size_t num_rows) mutable { if (!async_insert_log) return; @@ -822,15 +820,29 @@ try elem.table = query_table; elem.format = entry->format; elem.query_id = entry->query_id; - elem.bytes = num_bytes; + elem.bytes = entry->chunk.byteSize(); elem.rows = num_rows; elem.exception = parsing_exception; elem.data_kind = entry->chunk.getDataKind(); elem.timeout_milliseconds = data->timeout_ms.count(); elem.flush_query_id = insert_query_id; - auto it = query_by_format.find(entry->format); - elem.query_for_logging = it != query_by_format.end() ? it->second : key.query_str; + auto get_query_by_format = [&](const String & format) -> const String & + { + auto [it, inserted] = query_by_format.try_emplace(format); + if (!inserted) + return it->second; + + auto query = key.query->clone(); + assert_cast(*query).format = format; + it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); + return it->second; + }; + + if (entry->chunk.getDataKind() == DataKind::Parsed) + elem.query_for_logging = key.query_str; + else + elem.query_for_logging = get_query_by_format(entry->format); /// If there was a parsing error, /// the entry won't be flushed anyway, @@ -843,7 +855,7 @@ try else { elem.status = AsynchronousInsertLogElement::Ok; - log_elements.push_back(elem); + log_elements.push_back(std::move(elem)); } }; @@ -878,9 +890,8 @@ try if (async_insert_log) { - auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); for (const auto & entry : data->entries) - add_entry_to_asynchronous_insert_log(entry, query_by_format, "", 0, entry->chunk.byteSize()); + add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0); auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); @@ -919,13 +930,13 @@ try if (key.data_kind == DataKind::Parsed) chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); else - chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); + chunk = processPreprocessedEntries(data, header, add_entry_to_asynchronous_insert_log); ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); if (chunk.getNumRows() == 0) { - finish_entries(0, 0); + finish_entries(/*num_rows=*/ 0, /*num_bytes=*/ 0); return; } @@ -1012,7 +1023,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); auto chunk_info = std::make_shared(); - auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length); for (const auto & entry : data->entries) { @@ -1025,15 +1035,13 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); - size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); - total_rows += num_rows; + chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, {}, current_exception, num_rows, num_bytes); - + add_to_async_insert_log(entry, current_exception, num_rows); current_exception.clear(); entry->resetChunk(); } @@ -1045,18 +1053,14 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( template Chunk AsynchronousInsertQueue::processPreprocessedEntries( - const InsertQuery & key, const InsertDataPtr & data, const Block & header, - const ContextPtr & insert_context, LogFunc && add_to_async_insert_log) { size_t total_rows = 0; auto chunk_info = std::make_shared(); auto result_columns = header.cloneEmptyColumns(); - auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); - for (const auto & entry : data->entries) { const auto * block = entry->chunk.asBlock(); @@ -1073,10 +1077,11 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); total_rows += block_to_insert.rows(); + chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, query_by_format, "", block_to_insert.rows(), block_to_insert.bytes()); + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows()); entry->resetChunk(); } @@ -1085,27 +1090,6 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( return chunk; } -NameToNameMap AsynchronousInsertQueue::getQueriesByFormat( - const ASTPtr & query, - const std::list & entries, - const ContextPtr & insert_context) -{ - std::unordered_map format_to_query; - auto query_copy = query->clone(); - - for (const auto & entry : entries) - { - auto [it, inserted] = format_to_query.try_emplace(entry->format); - if (!inserted) - continue; - - assert_cast(*query_copy).format = entry->format; - it->second = serializeQuery(*query_copy, insert_context->getSettingsRef().log_queries_cut_to_length); - } - - return format_to_query; -} - template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 9a84fe8bb12..cbe998a2850 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -287,17 +287,10 @@ private: template static Chunk processPreprocessedEntries( - const InsertQuery & key, const InsertDataPtr & data, const Block & header, - const ContextPtr & insert_context, LogFunc && add_to_async_insert_log); - static NameToNameMap getQueriesByFormat( - const ASTPtr & query, - const std::list & entries, - const ContextPtr & insert_context); - template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); From 1aae191b810d30519cef1fd9784567ee03fd34e5 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 15:26:42 +0000 Subject: [PATCH 1505/1722] use db name in keeper path --- .../queries/0_stateless/03230_keeper_cp_mv_commands.reference | 4 ++-- tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference index 15f79fdebc7..f258af94664 100644 --- a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference @@ -7,9 +7,9 @@ simple move A C H node-A move node with childs -- must be error -Transaction failed (Not empty): Op #2, path: /test-keeper-client-cp-mv-commands/A +Transaction failed (Not empty): Op #2, path: /test-keeper-client-default/A A C H move node to existing -Transaction failed (Node exists): Op #1, path: /test-keeper-client-cp-mv-commands/A +Transaction failed (Node exists): Op #1, path: /test-keeper-client-default/A A C H clean up diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh index 15e117c7ca6..59b3547c36e 100755 --- a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh @@ -4,8 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -path="/test-keeper-client-cp-mv-commands" - +path="/test-keeper-client-$CLICKHOUSE_DATABASE" $CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null $CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'root'" From a4ddca773b4fdc5cc8d2b003bb64ae2de4790439 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Aug 2024 16:03:41 +0000 Subject: [PATCH 1506/1722] return back num_bytes --- src/Interpreters/AsynchronousInsertQueue.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index c619ea80c7c..9d7007d61aa 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -808,7 +808,8 @@ try auto add_entry_to_asynchronous_insert_log = [&, query_by_format = NameToNameMap{}]( const InsertData::EntryPtr & entry, const String & parsing_exception, - size_t num_rows) mutable + size_t num_rows, + size_t num_bytes) mutable { if (!async_insert_log) return; @@ -820,7 +821,7 @@ try elem.table = query_table; elem.format = entry->format; elem.query_id = entry->query_id; - elem.bytes = entry->chunk.byteSize(); + elem.bytes = num_bytes; elem.rows = num_rows; elem.exception = parsing_exception; elem.data_kind = entry->chunk.getDataKind(); @@ -891,7 +892,7 @@ try if (async_insert_log) { for (const auto & entry : data->entries) - add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0); + add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0, entry->chunk.byteSize()); auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); @@ -1035,13 +1036,15 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); + size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); + total_rows += num_rows; chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, current_exception, num_rows); + add_to_async_insert_log(entry, current_exception, num_rows, num_bytes); current_exception.clear(); entry->resetChunk(); } @@ -1081,7 +1084,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows()); + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows(), block_to_insert.bytes()); entry->resetChunk(); } From aad57f438cf4ab5d09c0f02b8946f54e44e7d8f8 Mon Sep 17 00:00:00 2001 From: sakulali Date: Fri, 30 Aug 2024 00:16:10 +0800 Subject: [PATCH 1507/1722] CREATE TABLE AS copy PRIMARY KEY, ORDER BY, and similar clauses. --- src/Interpreters/InterpreterCreateQuery.cpp | 13 ++++ ...te_table_as_with_sorting_clauses.reference | 70 +++++++++++++++++++ ...6_create_table_as_with_sorting_clauses.sql | 37 ++++++++++ 3 files changed, 120 insertions(+) create mode 100644 tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference create mode 100644 tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 80cb0510b35..e9f40bdbaf5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -821,6 +821,19 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti { properties.indices = as_storage_metadata->getSecondaryIndices(); properties.projections = as_storage_metadata->getProjections().clone(); + + /// CREATE TABLE AS should copy PRIMARY KEY, ORDER BY, and similar clauses. + if (!create.storage->primary_key && as_storage_metadata->isPrimaryKeyDefined() && as_storage_metadata->hasPrimaryKey()) + create.storage->set(create.storage->primary_key, as_storage_metadata->getPrimaryKeyAST()->clone()); + + if (!create.storage->partition_by && as_storage_metadata->isPartitionKeyDefined() && as_storage_metadata->hasPartitionKey()) + create.storage->set(create.storage->partition_by, as_storage_metadata->getPartitionKeyAST()->clone()); + + if (!create.storage->order_by && as_storage_metadata->isSortingKeyDefined() && as_storage_metadata->hasSortingKey()) + create.storage->set(create.storage->order_by, as_storage_metadata->getSortingKeyAST()->clone()); + + if (!create.storage->sample_by && as_storage_metadata->isSamplingKeyDefined() && as_storage_metadata->hasSamplingKey()) + create.storage->set(create.storage->sample_by, as_storage_metadata->getSamplingKeyAST()->clone()); } else { diff --git a/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference new file mode 100644 index 00000000000..cebb99f005e --- /dev/null +++ b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference @@ -0,0 +1,70 @@ +-------------- Test copy sorting clauses from source table -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 +-------------- Test copy sorting clauses from destination table (source table without the same type clauses) -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PRIMARY KEY (CounterID, EventDate, intHash32(UserID)) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +PRIMARY KEY (CounterID, EventDate, intHash32(UserID)) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 +-------------- Test copy sorting clauses from destination table (source table with the same type clauses) -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +ORDER BY CounterID +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql new file mode 100644 index 00000000000..96c2df54491 --- /dev/null +++ b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS x; +DROP TABLE IF EXISTS x_as; + +SELECT '-------------- Test copy sorting clauses from source table --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID); +CREATE TABLE x_as AS x ENGINE = MergeTree SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + +SELECT '-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PRIMARY KEY (CounterID, EventDate, intHash32(UserID)); +CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + +SELECT '-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree ORDER BY (CounterID); +CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + + From df16831cc8cd9e1af95116d73309a594fb42b9e5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Aug 2024 18:56:37 +0200 Subject: [PATCH 1508/1722] Update tests/integration/test_covered_by_broken_exists/test.py --- tests/integration/test_covered_by_broken_exists/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_covered_by_broken_exists/test.py b/tests/integration/test_covered_by_broken_exists/test.py index b6d1f55f133..caa091fdd2d 100644 --- a/tests/integration/test_covered_by_broken_exists/test.py +++ b/tests/integration/test_covered_by_broken_exists/test.py @@ -40,7 +40,7 @@ def wait_merged_part(table, part_name, retries=100): def test_make_clone_covered_by_broken_detached_dir_exists(started_cluster): - q("DROP TABLE IF EXISTS test_make_clone_cvbdde") + q("DROP TABLE IF EXISTS test_make_clone_cvbdde SYNC") q( "CREATE TABLE test_make_clone_cvbdde(n int, m String) ENGINE=ReplicatedMergeTree('/test_make_clone_cvbdde', '1') ORDER BY n SETTINGS old_parts_lifetime=3600, min_age_to_force_merge_seconds=1, min_age_to_force_merge_on_partition_only=0" From 93143b61169e751b256acc1ef83bc71470c56704 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 29 Aug 2024 19:11:41 +0200 Subject: [PATCH 1509/1722] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index d12f1a6ff37..652f5e0751d 100644 --- a/README.md +++ b/README.md @@ -62,6 +62,7 @@ Other upcoming meetups * [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 * [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 +* [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 From a474a5bc7c956a6a44b89b454dfe957b890466b6 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 29 Aug 2024 17:15:39 +0000 Subject: [PATCH 1510/1722] asan timed out too (because flaky check has timeout of ~36 seconds instead of 600) --- tests/queries/0_stateless/02735_parquet_encoder.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index c7e12a970fc..ab5047cb915 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: long, no-fasttest, no-parallel, no-tsan, no-msan +-- Tags: long, no-fasttest, no-parallel, no-tsan, no-msan, no-asan set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 1772a877ad7a5518443b6851956ed787206f42ae Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 29 Aug 2024 20:10:34 +0200 Subject: [PATCH 1511/1722] fix logical error in async inserts --- src/Interpreters/AsynchronousInsertQueue.cpp | 20 +++++++++++++++---- .../02481_async_insert_dedup.python | 9 +++++++-- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..c9137f39426 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -1004,8 +1004,14 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( size_t num_rows = executor.execute(*buffer); total_rows += num_rows; - chunk_info->offsets.push_back(total_rows); - chunk_info->tokens.push_back(entry->async_dedup_token); + /// for some reason, client can pass zero rows and bytes to server. + /// We don't update offsets in this case, because we assume every insert has some rows during dedup + /// but we have nothing to deduplicate for this insert. + if (num_rows > 0) + { + chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); + } add_to_async_insert_log(entry, query_for_logging, current_exception, num_rows, num_bytes, data->timeout_ms); @@ -1056,8 +1062,14 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); total_rows += block->rows(); - chunk_info->offsets.push_back(total_rows); - chunk_info->tokens.push_back(entry->async_dedup_token); + /// for some reason, client can pass zero rows and bytes to server. + /// We don't update offsets in this case, because we assume every insert has some rows during dedup, + /// but we have nothing to deduplicate for this insert. + if (block->rows()) + { + chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); + } const auto & query_for_logging = get_query_by_format(entry->format); add_to_async_insert_log(entry, query_for_logging, "", block->rows(), block->bytes(), data->timeout_ms); diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 24512836290..292d93833bf 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -48,9 +48,11 @@ def generate_data(q, total_number, use_token): partitions = ["2022-11-11 10:10:10", "2022-12-12 10:10:10"] last_number = 0 while True: - dup_simulate = random.randint(0, 3) + # 0 to simulate duplication + # 1 to simulate empty + simulate_flag = random.randint(0, 4) # insert old data randomly. 25% of them are dup. - if dup_simulate == 0: + if simulate_flag == 0: last_idx = len(old_data) - 1 if last_idx < 0: continue @@ -58,6 +60,9 @@ def generate_data(q, total_number, use_token): if idx < 0: idx = 0 q.put(old_data[idx]) + if simulate_flag == 1: + empty_insert_stmt = "insert into t_async_insert_dedup values format JSONEachRow" + q.put((empty_insert_stmt, token)) else: # insert new data. chunk_size = random.randint(1, max_chunk_size) From 25f31f914904e197797ef9917ff97c6447846203 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Aug 2024 18:15:05 +0000 Subject: [PATCH 1512/1722] Fix conversion for Dynamic, add more tests --- src/Functions/FunctionsConversion.cpp | 62 ++- ..._variant_dynamic_cast_or_default.reference | 484 ++++++++++++++++++ .../03212_variant_dynamic_cast_or_default.sql | 108 ++++ 3 files changed, 644 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 96e2a3291b3..271daa99d0c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4143,12 +4143,15 @@ private: /// requires quite a lot of work. By now let's simply use try/catch. /// First, check that we can create a wrapper. WrapperType wrapper = prepareUnpackDictionaries(from, to); - /// Second, check if we can perform a conversion on empty columns. - ColumnsWithTypeAndName column_from = {{from->createColumn(), from, "" }}; - wrapper(column_from, to, nullptr, 0); + /// Second, check if we can perform a conversion on column with default value. + /// (we cannot just check empty column as we do some checks only during iteration over rows). + auto test_col = from->createColumn(); + test_col->insertDefault(); + ColumnsWithTypeAndName column_from = {{test_col->getPtr(), from, "" }}; + wrapper(column_from, to, nullptr, 1); return wrapper; } - catch (...) + catch (const Exception &) { return {}; } @@ -4393,10 +4396,27 @@ private: casted_variant_columns.reserve(variant_types.size()); for (size_t i = 0; i != variant_types.size(); ++i) { + /// Skip shared variant, it will be processed later. + if (i == column_dynamic.getSharedVariantDiscriminator()) + { + casted_variant_columns.push_back(nullptr); + continue; + } + const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i); ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}}; - auto variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); - casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + WrapperType variant_wrapper; + if (cast_type == CastType::accurateOrNull) + /// Create wrapper only if we support conversion from variant to the resulting type. + variant_wrapper = createWrapperIfCanConvert(variant_types[i], result_type); + else + variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); + + ColumnPtr casted_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + casted_variant_columns.push_back(casted_variant); } /// Second, collect all variants stored in shared variant and cast them to result type. @@ -4452,8 +4472,18 @@ private: for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i) { ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}}; - auto variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); - casted_shared_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size())); + WrapperType variant_wrapper; + if (cast_type == CastType::accurateOrNull) + /// Create wrapper only if we support conversion from variant to the resulting type. + variant_wrapper = createWrapperIfCanConvert(variant_types_from_shared_variant[i], result_type); + else + variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); + + ColumnPtr casted_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size()); + casted_shared_variant_columns.push_back(casted_variant); } /// Construct result column from all casted variants. @@ -4463,11 +4493,23 @@ private: { auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + { res->insertDefault(); + } else if (global_discr == shared_variant_discr) - res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + { + if (casted_shared_variant_columns[shared_variant_indexes[i]]) + res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + else + res->insertDefault(); + } else - res->insertFrom(*casted_variant_columns[global_discr], offsets[i]); + { + if (casted_variant_columns[global_discr]) + res->insertFrom(*casted_variant_columns[global_discr], offsets[i]); + else + res->insertDefault(); + } } return res; diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference index 8b1a342181c..fd16d020019 100644 --- a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference @@ -30,3 +30,487 @@ 5 5 \N str_6 \N [0,1,2,3,4,5,6] +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +0 +1 +2 +3 +126 +127 +254 +255 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +-9223372036854775808 +-9223372036854775807 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +-9223372036854775808 +-9223372036854775807 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +296245801836096677496328508227807879401 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +-9223372036854775808 +-9223372036854775807 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +-inf +-3.4028233e38 +-1.7014118e38 +-9223372000000000000 +-2147483600 +-32768 +-32767 +-128 +-127 +-1 +-1.1754942e-38 +-1e-45 +0 +1e-45 +1.1754942e-38 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +3.4028233e38 +inf +nan +-inf +-1.7976931348623157e308 +-5.78960446186581e76 +-3.40282347e38 +-3.4028232635611926e38 +-1.7014118346046923e38 +-9223372036854776000 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +-1.1754943499999998e-38 +-1.1754942106924411e-38 +-1.401298464324817e-45 +-1.3999999999999999e-45 +-2.2250738585072014e-308 +0 +2.2250738585072014e-308 +1.3999999999999999e-45 +1.401298464324817e-45 +1.1754942106924411e-38 +1.1754943499999998e-38 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +3.4028232635611926e38 +3.40282347e38 +1.7976931348623157e308 +inf +nan +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +-9223372036854775808 +-9223372036854775807 +-18446744073709551.616 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +-340282347000000000977176926486249829565.415 +-9223372036854775808 +-9223372036854775807 +-18446744073709551.616 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +340282347000000000977176926486249829565.415 +1970-01-01 +1970-01-02 +1970-01-03 +1970-01-04 +1970-05-07 +1970-05-08 +1970-09-12 +1970-09-13 +2038-01-19 +2059-09-17 +2059-09-18 +2106-02-07 +2149-06-05 +2149-06-06 +2299-12-31 +2299-12-31 +1900-01-01 +1969-08-26 +1969-08-27 +1969-12-30 +1969-12-31 +1970-01-01 +1970-01-02 +1970-01-03 +1970-01-04 +1970-05-07 +1970-05-08 +1970-09-12 +1970-09-13 +2038-01-19 +2059-09-17 +2059-09-18 +2106-02-07 +2149-06-05 +2149-06-06 +2299-12-31 +1970-01-01 00:00:00 +1970-01-01 00:00:01 +1970-01-01 00:00:02 +1970-01-01 00:00:03 +1970-01-01 00:02:06 +1970-01-01 00:02:07 +1970-01-01 00:04:14 +1970-01-01 00:04:15 +1970-01-01 09:06:06 +1970-01-01 09:06:07 +1970-01-01 18:12:14 +1970-01-01 18:12:15 +2038-01-19 03:14:06 +2038-01-19 03:14:07 +2106-02-07 06:28:14 +2106-02-07 06:28:15 +0.0.0.0 +192.168.0.1 +:: +::1 +::ffff:192.168.0.1 +00000000-0000-0000-0000-000000000000 +dededdb6-7835-4ce4-8d11-b5de6f2820e9 diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql index 1e71e36780c..f227bbdac77 100644 --- a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql @@ -1,9 +1,117 @@ set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set allow_experimental_dynamic_type = 1; +set allow_suspicious_low_cardinality_types = 1; +set session_timezone = 'UTC'; select accurateCastOrDefault(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); select accurateCastOrNull(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); select accurateCastOrDefault(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); select accurateCastOrNull(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); + +drop table if exists t; +create table t (d Dynamic) engine=MergeTree order by tuple(); + +-- Integer types: signed and unsigned integers (UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256) +INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); +INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); +INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); +INSERT INTO t VALUES (-32768::Int16), (-32767::Int16), (-1::Int16), (0::Int16), (1::Int16), (32766::Int16), (32767::Int16); +INSERT INTO t VALUES (-2147483648::Int32), (-2147483647::Int32), (-1::Int32), (0::Int32), (1::Int32), (2147483646::Int32), (2147483647::Int32); +INSERT INTO t VALUES (-9223372036854775808::Int64), (-9223372036854775807::Int64), (-1::Int64), (0::Int64), (1::Int64), (9223372036854775806::Int64), (9223372036854775807::Int64); +INSERT INTO t VALUES (-170141183460469231731687303715884105728::Int128), (-170141183460469231731687303715884105727::Int128), (-1::Int128), (0::Int128), (1::Int128), (170141183460469231731687303715884105726::Int128), (170141183460469231731687303715884105727::Int128); +INSERT INTO t VALUES (-57896044618658097711785492504343953926634992332820282019728792003956564819968::Int256), (-57896044618658097711785492504343953926634992332820282019728792003956564819967::Int256), (-1::Int256), (0::Int256), (1::Int256), (57896044618658097711785492504343953926634992332820282019728792003956564819966::Int256), (57896044618658097711785492504343953926634992332820282019728792003956564819967::Int256); + +INSERT INTO t VALUES (0::UInt8), (1::UInt8), (254::UInt8), (255::UInt8); +INSERT INTO t VALUES (0::UInt16), (1::UInt16), (65534::UInt16), (65535::UInt16); +INSERT INTO t VALUES (0::UInt32), (1::UInt32), (4294967294::UInt32), (4294967295::UInt32); +INSERT INTO t VALUES (0::UInt64), (1::UInt64), (18446744073709551614::UInt64), (18446744073709551615::UInt64); +INSERT INTO t VALUES (0::UInt128), (1::UInt128), (340282366920938463463374607431768211454::UInt128), (340282366920938463463374607431768211455::UInt128); +INSERT INTO t VALUES (0::UInt256), (1::UInt256), (115792089237316195423570985008687907853269984665640564039457584007913129639934::UInt256), (115792089237316195423570985008687907853269984665640564039457584007913129639935::UInt256); + +-- Floating-point numbers: floats(Float32 and Float64) values +INSERT INTO t VALUES (1.17549435e-38::Float32), (3.40282347e+38::Float32), (-3.40282347e+38::Float32), (-1.17549435e-38::Float32), (1.4e-45::Float32), (-1.4e-45::Float32); +INSERT INTO t VALUES (inf::Float32), (-inf::Float32), (nan::Float32); +INSERT INTO t VALUES (inf::FLOAT(12)), (-inf::FLOAT(12)), (nan::FLOAT(12)); +INSERT INTO t VALUES (inf::FLOAT(15,22)), (-inf::FLOAT(15,22)), (nan::FLOAT(15,22)); + +INSERT INTO t VALUES (1.17549435e-38::Float64), (3.40282347e+38::Float64), (-3.40282347e+38::Float64), (-1.17549435e-38::Float64), (1.4e-45::Float64), (-1.4e-45::Float64); +INSERT INTO t VALUES (2.2250738585072014e-308::Float64), (1.7976931348623157e+308::Float64), (-1.7976931348623157e+308::Float64), (-2.2250738585072014e-308::Float64); +INSERT INTO t VALUES (inf::Float64), (-inf::Float64), (nan::Float64); +INSERT INTO t VALUES (inf::DOUBLE(12)), (-inf::DOUBLE(12)), (nan::DOUBLE(12)); +INSERT INTO t VALUES (inf::DOUBLE(15,22)), (-inf::DOUBLE(15,22)), (nan::DOUBLE(15,22)); + +-- Strings: String and FixedString +INSERT INTO t VALUES ('string'::String), ('1'::FixedString(1)), ('1'::FixedString(2)), ('1'::FixedString(10)); --(''::String), + +-- Boolean +INSERT INTO t VALUES ('1'::Bool), (0::Bool); + +-- UUID +INSERT INTO t VALUES ('dededdb6-7835-4ce4-8d11-b5de6f2820e9'::UUID); +INSERT INTO t VALUES ('00000000-0000-0000-0000-000000000000'::UUID); + +-- LowCardinality +INSERT INTO t VALUES ('1'::LowCardinality(String)), ('1'::LowCardinality(String)), (0::LowCardinality(UInt16)); + +-- Arrays +INSERT INTO t VALUES ([]::Array(Dynamic)), ([[]]::Array(Array(Dynamic))), ([[[]]]::Array(Array(Array(Dynamic)))); + +-- Tuple +INSERT INTO t VALUES (()::Tuple(Dynamic)), ((())::Tuple(Tuple(Dynamic))), (((()))::Tuple(Tuple(Tuple(Dynamic)))); + +-- Map. +INSERT INTO t VALUES (map(11::Dynamic, 'v1'::Dynamic, '22'::Dynamic, 1::Dynamic)); + +-- SimpleAggregateFunction +INSERT INTO t VALUES ([1,2]::SimpleAggregateFunction(anyLast, Array(Int16))); + +-- IPs +INSERT INTO t VALUES (toIPv4('192.168.0.1')), (toIPv6('::1')); + +-- Geo +INSERT INTO t VALUES ((1.23, 4.56)::Point), (([(1.23, 4.56)::Point, (2.34, 5.67)::Point])::Ring); +INSERT INTO t VALUES ([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]::MultiPolygon); + +-- Interval +INSERT INTO t VALUES (interval '1' day), (interval '2' month), (interval '3' year); + +-- Nested +INSERT INTO t VALUES ([(1, 'aa'), (2, 'bb')]::Nested(x UInt32, y String)); +INSERT INTO t VALUES ([(1, (2, ['aa', 'bb']), [(3, 'cc'), (4, 'dd')]), (5, (6, ['ee', 'ff']), [(7, 'gg'), (8, 'hh')])]::Nested(x UInt32, y Tuple(y1 UInt32, y2 Array(String)), z Nested(z1 UInt32, z2 String))); + +optimize table t final; + +select distinct toInt8OrDefault(d) as res from t order by res; +select distinct toUInt8OrDefault(d) as res from t order by res; +select distinct toInt16OrDefault(d) as res from t order by res; +select distinct toUInt16OrDefault(d) as res from t order by res; +select distinct toInt32OrDefault(d) as res from t order by res; +select distinct toUInt32OrDefault(d) as res from t order by res; +select distinct toInt64OrDefault(d) as res from t order by res; +select distinct toUInt64OrDefault(d) as res from t order by res; +select distinct toInt128OrDefault(d) as res from t order by res; +select distinct toUInt128OrDefault(d) as res from t order by res; +select distinct toInt256OrDefault(d) as res from t order by res; +select distinct toUInt256OrDefault(d) as res from t order by res; + +select distinct toFloat32OrDefault(d) as res from t order by res; +select distinct toFloat64OrDefault(d) as res from t order by res; + +select distinct toDecimal32OrDefault(d, 3) as res from t order by res; +select distinct toDecimal64OrDefault(d, 3) as res from t order by res; +select distinct toDecimal128OrDefault(d, 3) as res from t order by res; +select distinct toDecimal256OrDefault(d, 3) as res from t order by res; + +select distinct toDateOrDefault(d) as res from t order by res; +select distinct toDate32OrDefault(d) as res from t order by res; +select distinct toDateTimeOrDefault(d) as res from t order by res; + +select distinct toIPv4OrDefault(d) as res from t order by res; +select distinct toIPv6OrDefault(d) as res from t order by res; + +select distinct toUUIDOrDefault(d) as res from t order by res; + +drop table t; + From 79c01e717f56132c1af6503b93fc7094dcd47637 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Thu, 29 Aug 2024 23:15:39 +0400 Subject: [PATCH 1513/1722] rm comment, add docs --- docs/en/sql-reference/statements/alter/partition.md | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 1bb7817364a..11926b2aa08 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -351,7 +351,7 @@ ALTER TABLE mt DELETE IN PARTITION ID '2' WHERE p = 2; You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. -- Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH. For example, `ALTER TABLE visits ATTACH PARTITION ALL`. +- Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH/ATTACH FROM. For example, `ALTER TABLE visits ATTACH PARTITION ALL`. - As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - In the [ALTER ATTACH PART](#attach-partitionpart) and [DROP DETACHED PART](#drop-detached-partitionpart) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4915bf9f366..0688b05e2ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8240,7 +8240,6 @@ std::unique_ptr StorageReplicatedMergeTree::rep part_checksums.emplace_back(hash_hex); } - //ReplicatedMergeTreeLogEntryData entry; auto entry = std::make_unique(); { auto src_table_id = src_data.getStorageID(); From 1a28621b9f7030e23410d9c887ae4ec4418c2258 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 29 Aug 2024 21:59:09 +0200 Subject: [PATCH 1514/1722] fix style --- tests/queries/0_stateless/02481_async_insert_dedup.python | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 292d93833bf..b70fc574a00 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -61,7 +61,9 @@ def generate_data(q, total_number, use_token): idx = 0 q.put(old_data[idx]) if simulate_flag == 1: - empty_insert_stmt = "insert into t_async_insert_dedup values format JSONEachRow" + empty_insert_stmt = ( + "insert into t_async_insert_dedup values format JSONEachRow" + ) q.put((empty_insert_stmt, token)) else: # insert new data. From f5a88171a6c2959847beead82b204d000600aab4 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 29 Aug 2024 20:37:07 +0000 Subject: [PATCH 1515/1722] Fix ProgressIndication data race on `cancelQuery()` --- src/Common/ProgressIndication.cpp | 17 ++++++++++++----- src/Common/ProgressIndication.h | 2 ++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 8f0fb3cac6c..79c694574b0 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -34,13 +34,16 @@ bool ProgressIndication::updateProgress(const Progress & value) void ProgressIndication::resetProgress() { - watch.restart(); - progress.reset(); - show_progress_bar = false; - written_progress_chars = 0; - write_progress_on_update = false; + { + std::lock_guard lock(progress_mutex); + progress.reset(); + show_progress_bar = false; + written_progress_chars = 0; + write_progress_on_update = false; + } { std::lock_guard lock(profile_events_mutex); + watch.restart(); cpu_usage_meter.reset(getElapsedNanoseconds()); hosts_data.clear(); } @@ -90,6 +93,8 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const void ProgressIndication::writeFinalProgress() { + std::lock_guard lock(progress_mutex); + if (progress.read_rows < 1000) return; @@ -271,6 +276,8 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) void ProgressIndication::clearProgressOutput(WriteBufferFromFileDescriptor & message) { + std::lock_guard lock(progress_mutex); + if (written_progress_chars) { written_progress_chars = 0; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 474dd8db715..61b4ca1b305 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -115,6 +115,8 @@ private: /// It is possible concurrent access to the following: /// - writeProgress() (class properties) (guarded with progress_mutex) /// - hosts_data/cpu_usage_meter (guarded with profile_events_mutex) + /// + /// It is also possible to have more races if query is cancelled, so that clearProgressOutput() is called concurrently mutable std::mutex profile_events_mutex; mutable std::mutex progress_mutex; From 429905afa1ec0fb6f934920c73e6927edb1f5a14 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 30 Aug 2024 00:44:03 +0200 Subject: [PATCH 1516/1722] fix token --- tests/queries/0_stateless/02481_async_insert_dedup.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index b70fc574a00..c8b5abc11b0 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -64,7 +64,7 @@ def generate_data(q, total_number, use_token): empty_insert_stmt = ( "insert into t_async_insert_dedup values format JSONEachRow" ) - q.put((empty_insert_stmt, token)) + q.put((empty_insert_stmt, "")) else: # insert new data. chunk_size = random.randint(1, max_chunk_size) From 4ac4806b14917ff608ca53bea8c043165e96e58d Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 30 Aug 2024 08:07:21 +0200 Subject: [PATCH 1517/1722] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 0be6256aa50..5021cd390d8 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2729,6 +2729,7 @@ timeZoneOffset timezones tinylog tmp +toBool toColumnTypeName toDate toDateOrDefault From f7fda2471cba5f5e975cef247f10060084734fcb Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 30 Aug 2024 14:09:31 +0800 Subject: [PATCH 1518/1722] fix typo --- src/Interpreters/ActionsDAG.cpp | 4 ++-- src/Interpreters/ActionsDAG.h | 4 ++-- .../QueryPlan/Optimizations/useDataParallelAggregation.cpp | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2a594839c6a..9ebc73260b6 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -396,7 +396,7 @@ const ActionsDAG::Node * ActionsDAG::tryFindInOutputs(const std::string & name) return nullptr; } -ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutpus(const Names & names) const +ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutputs(const Names & names) const { NodeRawConstPtrs required_nodes; required_nodes.reserve(names.size()); @@ -524,7 +524,7 @@ void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_ void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs, bool allow_constant_folding) { - auto required_nodes = findInOutpus(required_names); + auto required_nodes = findInOutputs(required_names); outputs.swap(required_nodes); removeUnusedActions(allow_remove_inputs, allow_constant_folding); } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index ee2b3fbf4f2..2459878ce20 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -156,7 +156,7 @@ public: const Node * tryFindInOutputs(const std::string & name) const; /// Same, but for the list of names. - NodeRawConstPtrs findInOutpus(const Names & names) const; + NodeRawConstPtrs findInOutputs(const Names & names) const; /// Find first node with the same name in output nodes and replace it. /// If was not found, add node to outputs end. @@ -436,7 +436,7 @@ public: /// Returns a list of nodes representing atomic predicates. static NodeRawConstPtrs extractConjunctionAtoms(const Node * predicate); - /// Get a list of nodes. For every node, check if it can be compused using allowed subset of inputs. + /// Get a list of nodes. For every node, check if it can be computed using allowed subset of inputs. /// Returns only those nodes from the list which can be computed. static NodeRawConstPtrs filterNodesByAllowedInputs( NodeRawConstPtrs nodes, diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 0eeaec9bde7..d943fea785b 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -155,7 +155,7 @@ bool isPartitionKeySuitsGroupByKey( return false; /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). - auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); + auto key_nodes = group_by_actions.findInOutputs(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); const auto & gb_key_required_columns = group_by_key_actions.getRequiredColumnsNames(); From 6143f99f7ad3a4dcb2ae6cd9646dadc7ff5cf23d Mon Sep 17 00:00:00 2001 From: sunny19930321 Date: Fri, 30 Aug 2024 15:02:09 +0800 Subject: [PATCH 1519/1722] fix: change alter query to ternary expression --- src/Interpreters/InterpreterDeleteQuery.cpp | 12 +++--------- src/Parsers/ASTDeleteQuery.h | 2 +- .../02352_lightweight_delete_in_partition.sql | 4 ++-- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index 69aae3ae909..c0d840b8c5a 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -78,15 +78,9 @@ BlockIO InterpreterDeleteQuery::execute() String alter_query = "ALTER TABLE " + table->getStorageID().getFullTableName() + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " UPDATE `_row_exists` = 0 WHERE " + serializeAST(*delete_query.predicate); - if (delete_query.partition) - { - alter_query = - "ALTER TABLE " + table->getStorageID().getFullTableName() - + (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster)) - + " UPDATE `_row_exists` = 0 IN PARTITION " + serializeAST(*delete_query.partition) + " WHERE " - + serializeAST(*delete_query.predicate); - } + + " UPDATE `_row_exists` = 0" + + (delete_query.partition ? " IN PARTITION " + serializeAST(*delete_query.partition) : "") + + " WHERE " + serializeAST(*delete_query.predicate); ParserAlterQuery parser; ASTPtr alter_ast = parseQuery( diff --git a/src/Parsers/ASTDeleteQuery.h b/src/Parsers/ASTDeleteQuery.h index 6adc9b6a1c8..fcbeafac294 100644 --- a/src/Parsers/ASTDeleteQuery.h +++ b/src/Parsers/ASTDeleteQuery.h @@ -19,7 +19,7 @@ public: return removeOnCluster(clone(), params.default_database); } - /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE, DELETE FROM queries. + /** Used in DELETE FROM queries. * The value or ID of the partition is stored here. */ ASTPtr partition; diff --git a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql index 08cf5816f74..04371d273fd 100644 --- a/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql +++ b/tests/queries/0_stateless/02352_lightweight_delete_in_partition.sql @@ -5,10 +5,10 @@ CREATE TABLE t_merge_tree(time Date, id String , name String) ENGINE = MergeTree CREATE TABLE t_replicated_merge_tree(time Date, id String, name String) ENGINE = ReplicatedMergeTree('/test/02352/{database}/t_rep','1') PARTITION BY time ORDER BY id; INSERT INTO t_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); -INSERT INTO t_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); +INSERT INTO t_merge_tree select '2024-08-02', '1', toString(number) FROM numbers(100); INSERT INTO t_replicated_merge_tree select '2024-08-01', '1', toString(number) FROM numbers(100); -INSERT INTO t_replicated_merge_tree select '2024-08-02', '2', toString(number) FROM numbers(100); +INSERT INTO t_replicated_merge_tree select '2024-08-02', '1', toString(number) FROM numbers(100); SELECT COUNT() FROM t_merge_tree; SELECT COUNT() FROM t_replicated_merge_tree; From 6ad8e5558a99b43e3452c057346b9c44e8e27517 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 07:25:25 +0000 Subject: [PATCH 1520/1722] Fix typo --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index ae183d74782..58892d0dbf2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -195,7 +195,7 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) LOG_TRACE(logger, "Start writing vector similarity index"); if (empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty minmax index {}", backQuote(index_name)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty vector similarity index {}", backQuote(index_name)); writeIntBinary(FILE_FORMAT_VERSION, ostr); From aeaaef4347d614bded7dcbe2c230924847ba579a Mon Sep 17 00:00:00 2001 From: Alejandro Date: Fri, 30 Aug 2024 08:59:27 +0100 Subject: [PATCH 1521/1722] Run tests without new analyzer --- tests/queries/0_stateless/02944_variant_as_common_type.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index b3b86427b06..1f19e74e24d 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,6 +1,9 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; +-- This test should be run with allow_experimental_analyzer=0, because it checks the behavior of the old analyzer. +set allow_experimental_analyzer=0; + select toTypeName(res), if(1, [1,2,3], 'str_1') as res; select toTypeName(res), if(1, [1,2,3], 'str_1'::Nullable(String)) as res; From 8ed128792d623d137efe5708908d5eead776a34f Mon Sep 17 00:00:00 2001 From: Alejandro Date: Fri, 30 Aug 2024 08:59:40 +0100 Subject: [PATCH 1522/1722] remove comment --- tests/queries/0_stateless/02944_variant_as_common_type.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index 1f19e74e24d..567b1d20e3a 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,7 +1,5 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; - --- This test should be run with allow_experimental_analyzer=0, because it checks the behavior of the old analyzer. set allow_experimental_analyzer=0; select toTypeName(res), if(1, [1,2,3], 'str_1') as res; From c5b92413cac091c4e28acb5160b89fcabc37a853 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 08:28:31 +0000 Subject: [PATCH 1523/1722] Fix vector similarity index does not work for cosineDistance --- src/Storages/MergeTree/VectorSimilarityCondition.cpp | 2 ++ src/Storages/MergeTree/VectorSimilarityCondition.h | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.cpp b/src/Storages/MergeTree/VectorSimilarityCondition.cpp index c8f33857640..251cdde65ab 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.cpp +++ b/src/Storages/MergeTree/VectorSimilarityCondition.cpp @@ -44,6 +44,8 @@ VectorSimilarityCondition::Info::DistanceFunction stringToDistanceFunction(std:: { if (distance_function == "L2Distance") return VectorSimilarityCondition::Info::DistanceFunction::L2; + else if (distance_function == "cosineDistance") + return VectorSimilarityCondition::Info::DistanceFunction::Cosine; else return VectorSimilarityCondition::Info::DistanceFunction::Unknown; } diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.h b/src/Storages/MergeTree/VectorSimilarityCondition.h index 2380f8f46b0..e2946222f49 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.h +++ b/src/Storages/MergeTree/VectorSimilarityCondition.h @@ -57,7 +57,8 @@ public: enum class DistanceFunction : uint8_t { Unknown, - L2 + L2, + Cosine }; std::vector reference_vector; From af7adfe4b2f8e9b3b7354314a91d19a19b7955f9 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 09:03:23 +0000 Subject: [PATCH 1524/1722] Remove unused code --- src/Storages/MergeTree/VectorSimilarityCondition.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.h b/src/Storages/MergeTree/VectorSimilarityCondition.h index e2946222f49..83ae1e19bfb 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.h +++ b/src/Storages/MergeTree/VectorSimilarityCondition.h @@ -143,6 +143,7 @@ private: void traverseOrderByAST(const ASTPtr & node, RPN & rpn); /// Returns true and stores ANNExpr if the query has valid WHERE section + /// TODO NOT implemented, WHERE does not supported. static bool matchRPNWhere(RPN & rpn, Info & info); /// Returns true and stores ANNExpr if the query has valid ORDERBY section @@ -151,9 +152,6 @@ private: /// Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); - /// Matches dist function, reference vector, column name - static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, Info & info); - /// Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); From b1fc15ffc968d6c96c9e440436a728e757812710 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:49:56 +0200 Subject: [PATCH 1525/1722] Update docs/en/sql-reference/aggregate-functions/parametric-functions.md --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 47fbb2ea31d..f6552dc4168 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -104,7 +104,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#pattern-syntax). **Returned values** From 9a4c32e8deb748f8ee84bcecf34716000060b9bc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:50:02 +0200 Subject: [PATCH 1526/1722] Update docs/en/sql-reference/aggregate-functions/parametric-functions.md --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index f6552dc4168..f391765139a 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -197,7 +197,7 @@ sequenceCount(pattern)(timestamp, cond1, cond2, ...) **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#pattern-syntax). **Returned values** From 76b938acaf236196dcfa502fd9b0e09190180f13 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:50:08 +0200 Subject: [PATCH 1527/1722] Update docs/en/sql-reference/aggregate-functions/parametric-functions.md --- .../sql-reference/aggregate-functions/parametric-functions.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index f391765139a..b06717fcc8c 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -113,9 +113,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord Type: `UInt8`. - - -**Pattern syntax** +#### Pattern syntax - `(?N)` — Matches the condition argument at position `N`. Conditions are numbered in the `[1, 32]` range. For example, `(?1)` matches the argument passed to the `cond1` parameter. From b0ecacc0c3082867286e7061ac831ec779079844 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 30 Aug 2024 12:53:42 +0200 Subject: [PATCH 1528/1722] remove no-parallel --- tests/queries/0_stateless/02481_async_insert_dedup.sh | 2 +- tests/queries/0_stateless/02481_async_insert_dedup_token.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.sh b/tests/queries/0_stateless/02481_async_insert_dedup.sh index 0fe06e6ab58..2a646a6bccd 100755 --- a/tests/queries/0_stateless/02481_async_insert_dedup.sh +++ b/tests/queries/0_stateless/02481_async_insert_dedup.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02481_async_insert_dedup_token.sh b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh index bb2d07066a5..81e315ce7dc 100755 --- a/tests/queries/0_stateless/02481_async_insert_dedup_token.sh +++ b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From b7eb7cceebeadc2430f88a1775b4ef00ae06f8a0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 14:02:59 +0200 Subject: [PATCH 1529/1722] remake test to remove the head of path --- tests/queries/0_stateless/03232_file_path_normalizing.reference | 2 +- tests/queries/0_stateless/03232_file_path_normalizing.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference index d7dc12010f5..953db2c5dfe 100644 --- a/tests/queries/0_stateless/03232_file_path_normalizing.reference +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -1 +1 @@ -/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet +data_hive/partitioning/column0=Elizabeth/sample.parquet diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh index eeaa1f2014d..e7a7a65be51 100755 --- a/tests/queries/0_stateless/03232_file_path_normalizing.sh +++ b/tests/queries/0_stateless/03232_file_path_normalizing.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "SELECT _path FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" +$CLICKHOUSE_LOCAL -q "SELECT substring(_path, position(_path, 'data_hive')) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" From b4b68196243ff8ec90cbc31c4294663127ad976f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 30 Aug 2024 14:07:30 +0200 Subject: [PATCH 1530/1722] Adapt backward compatibility test --- .../test_functions.py | 45 ++++++++++++++++--- 1 file changed, 38 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 3231fb87f33..202a741bfb5 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -67,6 +67,11 @@ def test_aggregate_states(start_cluster): f"select hex(initializeAggregation('{function_name}State', 'foo'))" ).strip() + def get_final_value_unhex(node, function_name, value): + return node.query( + f"select finalizeAggregation(unhex('{value}')::AggregateFunction({function_name}, String))" + ).strip() + for aggregate_function in aggregate_functions: logging.info("Checking %s", aggregate_function) @@ -99,13 +104,39 @@ def test_aggregate_states(start_cluster): upstream_state = get_aggregate_state_hex(upstream, aggregate_function) if upstream_state != backward_state: - logging.info( - "Failed %s, %s (backward) != %s (upstream)", - aggregate_function, - backward_state, - upstream_state, - ) - failed += 1 + allowed_changes_if_result_is_the_same = ["anyHeavy"] + + if aggregate_function in allowed_changes_if_result_is_the_same: + backward_final_from_upstream = get_final_value_unhex( + backward, aggregate_function, upstream_state + ) + upstream_final_from_backward = get_final_value_unhex( + upstream, aggregate_function, backward_state + ) + + if backward_final_from_upstream == upstream_final_from_backward: + logging.info( + "OK %s (but different intermediate states)", aggregate_function + ) + passed += 1 + else: + logging.error( + "Failed %s, Intermediate: %s (backward) != %s (upstream). Final from intermediate: %s (backward from upstream state) != %s (upstream from backward state)", + aggregate_function, + backward_state, + upstream_state, + backward_final_from_upstream, + upstream_final_from_backward, + ) + failed += 1 + else: + logging.error( + "Failed %s, %s (backward) != %s (upstream)", + aggregate_function, + backward_state, + upstream_state, + ) + failed += 1 else: logging.info("OK %s", aggregate_function) passed += 1 From 91b345714dac713f115525df48d3872c26868002 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Fri, 30 Aug 2024 13:16:31 +0100 Subject: [PATCH 1531/1722] Revert enabling the legacy analyzer --- tests/queries/0_stateless/02944_variant_as_common_type.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index 567b1d20e3a..b3b86427b06 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,6 +1,5 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; -set allow_experimental_analyzer=0; select toTypeName(res), if(1, [1,2,3], 'str_1') as res; select toTypeName(res), if(1, [1,2,3], 'str_1'::Nullable(String)) as res; From cbf82712542fb192ac4bbf1102e894d46ecb2ee1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 30 Aug 2024 15:45:51 +0200 Subject: [PATCH 1532/1722] Better --- tests/ci/changelog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 554ba339892..8e7900de353 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -288,7 +288,7 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri # Normalize bug fixes if ( re.match( - r".*(?i)bug\Wfix", + r"(?i).*bug\Wfix", category, ) # Map "Critical Bug Fix" to "Bug fix" category for changelog From cf87893758b0172f19e54b8bc7fd962bf258c96e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Aug 2024 15:28:53 +0000 Subject: [PATCH 1533/1722] Don't infer Bool type from String in CSV when input_format_csv_try_infer_numbers_from_strings=1 --- src/Formats/EscapingRuleUtils.cpp | 8 ++++++-- .../03231_csv_dont_infer_bool_from_string.reference | 4 ++++ .../0_stateless/03231_csv_dont_infer_bool_from_string.sql | 4 ++++ 3 files changed, 14 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference create mode 100644 tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 5429d8b7e0d..50a46d2334d 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -302,8 +302,12 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet /// Try to determine the type of value inside quotes auto type = tryInferDataTypeForSingleField(data, format_settings); - /// If we couldn't infer any type or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. - if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type))) + /// Return String type if one of the following conditions apply + /// - we couldn't infer any type + /// - it's a number and csv.try_infer_numbers_from_strings = 0 + /// - it's a tuple and try_infer_strings_from_quoted_tuples = 0 + /// - it's a Bool type (we don't allow reading bool values from strings) + if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type)) || isBool(type)) return std::make_shared(); return type; diff --git a/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference new file mode 100644 index 00000000000..d23e2d2cbf3 --- /dev/null +++ b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference @@ -0,0 +1,4 @@ +c1 Nullable(Int64) +c2 Nullable(Float64) +c3 Nullable(String) +42 42.42 True diff --git a/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql new file mode 100644 index 00000000000..e3cf77249eb --- /dev/null +++ b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql @@ -0,0 +1,4 @@ +set input_format_csv_try_infer_numbers_from_strings = 1; +desc format(CSV, '"42","42.42","True"'); +select * from format(CSV, '"42","42.42","True"'); + From 30dd82324aac0053d340d0c08b3f056f8255b6d9 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 30 Aug 2024 19:26:59 +0000 Subject: [PATCH 1534/1722] revert wrong data race fix --- src/Common/HTTPConnectionPool.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 68c13838c04..7a65863180e 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -391,15 +391,11 @@ private: { auto idle = idleTime(); - // Reset data hooks for IO scheduling + // Set data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIORead(), log, request.getMethod(), request.getURI())); - else - Session::setReceiveDataHooks(); if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIOWrite(), log, request.getMethod(), request.getURI())); - else - Session::setSendDataHooks(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); @@ -457,6 +453,8 @@ private: } } response_stream = nullptr; + Session::setSendDataHooks(); + Session::setReceiveDataHooks(); group->atConnectionDestroy(); From 3675e83a3f064c6972546537367cc53e36ccb236 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 30 Aug 2024 19:27:26 +0000 Subject: [PATCH 1535/1722] Fix data race in ResourceGuard --- src/Common/Scheduler/ResourceGuard.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 2e735aae656..5947022d9b1 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -107,8 +107,8 @@ public: std::unique_lock lock(mutex); chassert(state == Enqueued); state = Dequeued; + dequeued_cv.notify_one(); } - dequeued_cv.notify_one(); } void wait() From 9f96d180604cba6467e64a51bf98118ebafc598d Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 30 Aug 2024 19:33:34 +0000 Subject: [PATCH 1536/1722] cleanup --- src/Common/Scheduler/ResourceGuard.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 5947022d9b1..cf97f7acf93 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -103,12 +103,10 @@ public: // That thread will continue execution and do real consumption of requested resource synchronously. void execute() override { - { - std::unique_lock lock(mutex); - chassert(state == Enqueued); - state = Dequeued; - dequeued_cv.notify_one(); - } + std::unique_lock lock(mutex); + chassert(state == Enqueued); + state = Dequeued; + dequeued_cv.notify_one(); } void wait() From a031ec7452f0d5b5187460e8deeb8eb35921da7e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 30 Aug 2024 22:02:17 +0000 Subject: [PATCH 1537/1722] compose_subnet: dns tests passed --- tests/integration/compose/docker_compose_net.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_net.yml b/tests/integration/compose/docker_compose_net.yml index eff43681f2e..768d95c1dbf 100644 --- a/tests/integration/compose/docker_compose_net.yml +++ b/tests/integration/compose/docker_compose_net.yml @@ -5,7 +5,7 @@ networks: enable_ipv6: true ipam: config: - - subnet: 10.5.0.0/12 + - subnet: 10.0.0.0/12 gateway: 10.5.1.1 - subnet: 2001:3984:3989::/64 gateway: 2001:3984:3989::1 From 3ffb965b2330302e2c3b99b004dd9859208208ff Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 31 Aug 2024 02:02:15 +0200 Subject: [PATCH 1538/1722] Fix gateway --- tests/integration/compose/docker_compose_net.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_net.yml b/tests/integration/compose/docker_compose_net.yml index 768d95c1dbf..311f3008639 100644 --- a/tests/integration/compose/docker_compose_net.yml +++ b/tests/integration/compose/docker_compose_net.yml @@ -6,6 +6,6 @@ networks: ipam: config: - subnet: 10.0.0.0/12 - gateway: 10.5.1.1 + gateway: 10.0.0.1 - subnet: 2001:3984:3989::/64 gateway: 2001:3984:3989::1 From 74f99d1f07dc838513eca06fd2f9e39540b7720d Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 26 Aug 2024 16:47:59 +0800 Subject: [PATCH 1539/1722] support datetrunc and toStartOfxxx on date32 --- src/Functions/date_trunc.cpp | 8 ++++++-- src/Functions/toStartOfInterval.cpp | 13 +++++++++++-- .../0_stateless/00403_to_start_of_day.reference | 1 + tests/queries/0_stateless/00403_to_start_of_day.sql | 1 + .../02935_date_trunc_case_unsensitiveness.reference | 5 +++++ .../02935_date_trunc_case_unsensitiveness.sql | 8 ++++++++ 6 files changed, 32 insertions(+), 4 deletions(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index dd3ea0b877b..7b094174047 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ public: enum ResultType { Date, + Date32, DateTime, DateTime64, }; @@ -75,11 +77,11 @@ public: bool second_argument_is_date = false; auto check_second_argument = [&] { - if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) + if (!isDateOrDate32(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " "Should be a date or a date with time", arguments[1].type->getName(), getName()); - second_argument_is_date = isDate(arguments[1].type); + second_argument_is_date = isDateOrDate32(arguments[1].type); if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour) || (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))) @@ -119,6 +121,8 @@ public: if (result_type == ResultType::Date) return std::make_shared(); + if (result_type == ResultType::Date32) + return std::make_shared(); else if (result_type == ResultType::DateTime) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false)); else diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 21b7cf895d2..35fb43df131 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -44,9 +44,9 @@ public: auto check_first_argument = [&] { const DataTypePtr & type_arg1 = arguments[0].type; - if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) + if (!isDateOrDate32(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1st argument of function {}, expected a Date, DateTime or DateTime64", + "Illegal type {} of 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64", type_arg1->getName(), getName()); value_is_date = isDate(type_arg1); }; @@ -56,6 +56,7 @@ public: enum class ResultType : uint8_t { Date, + Date32, DateTime, DateTime64 }; @@ -128,6 +129,8 @@ public: { case ResultType::Date: return std::make_shared(); + case ResultType::Date32: + return std::make_shared(); case ResultType::DateTime: return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); case ResultType::DateTime64: @@ -185,6 +188,12 @@ private: if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); } + else if (isDate32(time_column_type)) + { + const auto * time_column_vec = checkAndGetColumn(&time_column_col); + if (time_column_vec) + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); + } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); } diff --git a/tests/queries/0_stateless/00403_to_start_of_day.reference b/tests/queries/0_stateless/00403_to_start_of_day.reference index d00491fd7e5..1c96b0ed64f 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.reference +++ b/tests/queries/0_stateless/00403_to_start_of_day.reference @@ -1 +1,2 @@ 1 +2024-08-26 00:00:00 diff --git a/tests/queries/0_stateless/00403_to_start_of_day.sql b/tests/queries/0_stateless/00403_to_start_of_day.sql index e298afd80ba..2dfcfccec38 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.sql +++ b/tests/queries/0_stateless/00403_to_start_of_day.sql @@ -1 +1,2 @@ SELECT toStartOfDay(now()) = toDateTime(toDate(now())); +SELECT toStartOfDay(toDate32(now())); diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference index 70a26e5447d..066ca3cfd55 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference @@ -1,11 +1,16 @@ 2022-03-01 00:00:00 2022-03-01 2022-02-28 +2022-02-28 2022-03-01 00:00:00 2022-03-01 2022-02-28 +2022-02-28 2022-03-01 00:00:00 2022-03-01 +2022-03-01 00:00:00 +2022-03-01 +2022-02-28 2022-02-28 2022-03-01 12:12:12.012000000 2022-03-01 12:12:12.012346 diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql index df290f3deac..2e8e6dcd093 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql @@ -1,12 +1,17 @@ SELECT dateTrunc('DAY', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('MONTH', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('WEEK', toDate('2022-03-01')); +SELECT dateTrunc('WEEK', toDate32('2022-03-01')); SELECT dateTrunc('Day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('Month', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('Week', toDate('2022-03-01')); +SELECT dateTrunc('Week', toDate32('2022-03-01')); SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2)); +SELECT dateTrunc('day', toDate32('2022-03-01')); +SELECT dateTrunc('month', toDate32('2022-03-01')); SELECT dateTrunc('week', toDate('2022-03-01')); +SELECT dateTrunc('week', toDate32('2022-03-01')); SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3)); SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7)); SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9)); @@ -25,3 +30,6 @@ SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILL SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT dateTrunc('Nanosecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT dateTrunc('MicroSecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT dateTrunc('MILLISECOND', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 5e0000c4ca21d84756576ebb563fd9e1d98433e7 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 28 Aug 2024 16:06:09 +0800 Subject: [PATCH 1540/1722] add test case and doc --- .../functions/date-time-functions.md | 2 +- src/Functions/toStartOfInterval.cpp | 2 +- .../00403_to_start_of_day.reference | 1 - .../0_stateless/00403_to_start_of_day.sql | 1 - ..._date_trunc_case_unsensitiveness.reference | 5 --- .../02935_date_trunc_case_unsensitiveness.sql | 8 ----- ...d_to_start_of_interval_on_date32.reference | 33 +++++++++++++++++++ ...unc_and_to_start_of_interval_on_date32.sql | 26 +++++++++++++++ 8 files changed, 61 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference create mode 100644 tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3d95ae2cb74..b65fb3d7e95 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2019,7 +2019,7 @@ Alias: `dateTrunc`. `unit` argument is case-insensitive. -- `value` — Date and time. [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md). +- `value` — Date and time. [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../data-types/string.md). **Returned value** diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 35fb43df131..709f5f86d80 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -194,7 +194,7 @@ private: if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64", getName()); } template diff --git a/tests/queries/0_stateless/00403_to_start_of_day.reference b/tests/queries/0_stateless/00403_to_start_of_day.reference index 1c96b0ed64f..d00491fd7e5 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.reference +++ b/tests/queries/0_stateless/00403_to_start_of_day.reference @@ -1,2 +1 @@ 1 -2024-08-26 00:00:00 diff --git a/tests/queries/0_stateless/00403_to_start_of_day.sql b/tests/queries/0_stateless/00403_to_start_of_day.sql index 2dfcfccec38..e298afd80ba 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.sql +++ b/tests/queries/0_stateless/00403_to_start_of_day.sql @@ -1,2 +1 @@ SELECT toStartOfDay(now()) = toDateTime(toDate(now())); -SELECT toStartOfDay(toDate32(now())); diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference index 066ca3cfd55..70a26e5447d 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference @@ -1,16 +1,11 @@ 2022-03-01 00:00:00 2022-03-01 2022-02-28 -2022-02-28 2022-03-01 00:00:00 2022-03-01 2022-02-28 -2022-02-28 2022-03-01 00:00:00 2022-03-01 -2022-03-01 00:00:00 -2022-03-01 -2022-02-28 2022-02-28 2022-03-01 12:12:12.012000000 2022-03-01 12:12:12.012346 diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql index 2e8e6dcd093..df290f3deac 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql @@ -1,17 +1,12 @@ SELECT dateTrunc('DAY', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('MONTH', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('WEEK', toDate('2022-03-01')); -SELECT dateTrunc('WEEK', toDate32('2022-03-01')); SELECT dateTrunc('Day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('Month', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('Week', toDate('2022-03-01')); -SELECT dateTrunc('Week', toDate32('2022-03-01')); SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2)); -SELECT dateTrunc('day', toDate32('2022-03-01')); -SELECT dateTrunc('month', toDate32('2022-03-01')); SELECT dateTrunc('week', toDate('2022-03-01')); -SELECT dateTrunc('week', toDate32('2022-03-01')); SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3)); SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7)); SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9)); @@ -30,6 +25,3 @@ SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILL SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT dateTrunc('Nanosecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT dateTrunc('MicroSecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT dateTrunc('MILLISECOND', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference new file mode 100644 index 00000000000..ea0c96ab2d2 --- /dev/null +++ b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference @@ -0,0 +1,33 @@ +-- { echoOn } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 YEAR); +2022-01-01 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 QUARTER); +2022-07-01 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MONTH); +2022-09-01 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 WEEK); +2022-09-12 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 DAY); +2022-09-16 00:00:00 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 HOUR); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 SECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MILLISECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MICROSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 NANOSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('YEAR', toDate32('2022-09-16')); +2022-01-01 +select date_trunc('QUARTER', toDate32('2022-09-16')); +2022-07-01 +select date_trunc('MONTH', toDate32('2022-09-16')); +2022-09-01 +select date_trunc('WEEK', toDate32('2022-09-16')); +2022-09-12 +select date_trunc('DAY', toDate32('2022-09-16')); +2022-09-16 00:00:00 +select date_trunc('HOUR', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MINUTE', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('SECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MILLISECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MICROSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('NANOSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql new file mode 100644 index 00000000000..b2b6385f00b --- /dev/null +++ b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql @@ -0,0 +1,26 @@ +-- { echoOn } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 YEAR); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 QUARTER); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MONTH); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 WEEK); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 DAY); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 HOUR); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 SECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MILLISECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MICROSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 NANOSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +select date_trunc('YEAR', toDate32('2022-09-16')); +select date_trunc('QUARTER', toDate32('2022-09-16')); +select date_trunc('MONTH', toDate32('2022-09-16')); +select date_trunc('WEEK', toDate32('2022-09-16')); +select date_trunc('DAY', toDate32('2022-09-16')); +select date_trunc('HOUR', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MINUTE', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('SECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MILLISECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MICROSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('NANOSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + + From cd1685d409a481e7573bcacc73b61f886ca99101 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 30 Aug 2024 13:55:37 +0800 Subject: [PATCH 1541/1722] fix error message --- src/Functions/date_trunc.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 7b094174047..15d50724506 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -85,7 +85,7 @@ public: if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour) || (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for function {}", arguments[1].type->getName(), getName()); }; auto check_timezone_argument = [&] { From c702d2581e50d042eed7b8374464e3e979cde10d Mon Sep 17 00:00:00 2001 From: wxybear Date: Sat, 31 Aug 2024 14:50:35 +0800 Subject: [PATCH 1542/1722] fix: handle insert query in explain queries in multiquery mode --- src/Client/ClientBase.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 01d03006eec..6132f744098 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -2111,6 +2112,15 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( // - Other formats (e.g. FORMAT CSV) are arbitrarily more complex and tricky to parse. For example, we may be unable to distinguish if the semicolon // is part of the data or ends the statement. In this case, we simply assume that the end of the INSERT statement is determined by \n\n (two newlines). auto * insert_ast = parsed_query->as(); + // We also consider the INSERT query in EXPLAIN queries (same as normal INSERT queries) + if (!insert_ast) + { + auto * explain_ast = parsed_query->as(); + if (explain_ast) + { + insert_ast = explain_ast->getExplainedQuery()->as(); + } + } const char * query_to_execute_end = this_query_end; if (insert_ast && insert_ast->data) { From 199aaff7118ca04c8c7c32b9aaf9ea5cd660c700 Mon Sep 17 00:00:00 2001 From: sakulali Date: Sat, 31 Aug 2024 18:13:07 +0800 Subject: [PATCH 1543/1722] Kindly ping CI and try to reproduce failed CI tests From 5ef1830f559d565060d92ff9ccbfc9ebc11773f5 Mon Sep 17 00:00:00 2001 From: wxybear Date: Sat, 31 Aug 2024 18:39:01 +0800 Subject: [PATCH 1544/1722] feat: add explain ast insert queries test --- .../0_stateless/03156_default_multiquery_split.reference | 4 ++++ .../queries/0_stateless/03156_default_multiquery_split.sh | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.reference b/tests/queries/0_stateless/03156_default_multiquery_split.reference index 0f3a1baff45..02a3a522ecd 100644 --- a/tests/queries/0_stateless/03156_default_multiquery_split.reference +++ b/tests/queries/0_stateless/03156_default_multiquery_split.reference @@ -8,3 +8,7 @@ Syntax error 7 8 9 +InsertQuery (children 1) + Identifier TEST2 +InsertQuery (children 1) + Identifier TEST1 diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.sh b/tests/queries/0_stateless/03156_default_multiquery_split.sh index d849fb5a162..08ee9bcad63 100755 --- a/tests/queries/0_stateless/03156_default_multiquery_split.sh +++ b/tests/queries/0_stateless/03156_default_multiquery_split.sh @@ -51,6 +51,13 @@ INSERT INTO TEST2 VALUES SELECT * FROM TEST1 ORDER BY value; SELECT * FROM TEST2 ORDER BY value; DROP TABLE TEST1; DROP TABLE TEST2; + +EXPLAIN AST INSERT INTO TEST2 FORMAT CSV +1 +2 + +EXPLAIN AST INSERT INTO TEST1 VALUES (101),(102); + EOF $CLICKHOUSE_CLIENT -m < "$SQL_FILE_NAME" From a6145e509908eb1ad14c4952900676d92c458b90 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 31 Aug 2024 12:09:02 +0000 Subject: [PATCH 1545/1722] Fix: expression description in plan after lift up unioin optimization --- ...imize_distributed_group_by_sharding_key.reference | 4 ++-- .../02496_remove_redundant_sorting.reference | 12 ++++++------ ...02496_remove_redundant_sorting_analyzer.reference | 12 ++++++------ .../02500_remove_redundant_distinct.reference | 4 ++-- ...2500_remove_redundant_distinct_analyzer.reference | 4 ++-- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index e786532f25a..a807bf7096e 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -21,7 +21,7 @@ Expression (Projection) Union Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) ReadFromSystemNumbers - Expression + Expression (Before LIMIT BY) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union @@ -96,7 +96,7 @@ Expression (Project names) LimitBy Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))) ReadFromSystemNumbers - Expression + Expression (Before LIMIT BY) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 4d004f2f78f..7824fd8cba9 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -395,9 +395,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 @@ -427,9 +427,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Nullable(Float64) 9007199254740994 @@ -459,9 +459,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index dd5ac7bf706..3c68d14fdf2 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -394,9 +394,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 @@ -426,9 +426,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Nullable(Float64) 9007199254740994 @@ -458,9 +458,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference index d7623cd5541..9bb8f4a4017 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference @@ -53,7 +53,7 @@ Expression (Projection) Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) ReadFromSystemNumbers - Expression (( + Projection)) + Expression ((Before ORDER BY + Projection)) Distinct Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) @@ -536,7 +536,7 @@ Expression (Projection) Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) ReadFromSystemNumbers - Expression (( + Projection)) + Expression ((Before ORDER BY + Projection)) Distinct Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index b79f6310166..27b01cf1158 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -54,7 +54,7 @@ Expression (Project names) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) ReadFromSystemNumbers - Expression (( + ( + Project names))) + Expression ((Projection + (Change column names to column identifiers + Project names))) Distinct (DISTINCT) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) @@ -542,7 +542,7 @@ Expression (Project names) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) ReadFromSystemNumbers - Expression (( + ( + Project names))) + Expression ((Projection + (Change column names to column identifiers + Project names))) Distinct (DISTINCT) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) From 425a93e36040d66dc6b2a0fa4bf2ac50fbe8d876 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 31 Aug 2024 12:12:32 +0000 Subject: [PATCH 1546/1722] Fix --- src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index c48551732c9..43cf166002e 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -50,6 +50,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), expression->getExpression().clone()); + expr_node.step->setStepDescription(expression->getStepDescription()); } /// - Expression - Something From 0857a10fc1c9f0863a0059bfb0961c131a0c75a9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 31 Aug 2024 14:17:24 +0200 Subject: [PATCH 1547/1722] Fix types in JSON docs --- docs/en/sql-reference/data-types/newjson.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index f7fc7e1498e..8218ba89176 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -453,8 +453,8 @@ As we can see, after inserting paths `e` and `f.g` the limit was reached and we ### During merges of data parts in MergeTree table engines -During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths won't be able to store all paths from source parts as subcolumns. -In this case ClickHouse chooses what paths will remain as subcolumns after merge and what types will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contains +During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths and won't be able to store all paths from source parts as subcolumns. +In this case ClickHouse chooses what paths will remain as subcolumns after merge and what paths will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contain the largest number of non-null values and move the rarest paths to the shared data structure, but it depends on the implementation. Let's see an example of such merge. First, let's create a table with `JSON` column, set the limit of dynamic paths to `3` and insert values with `5` different paths: From b066760a21cdefbabae5172624ca8c65d5200f70 Mon Sep 17 00:00:00 2001 From: wxybear Date: Sat, 31 Aug 2024 22:18:33 +0800 Subject: [PATCH 1548/1722] fix: explainedQuery nullptr && 02504_explain_ast_insert --- src/Client/ClientBase.cpp | 2 +- tests/queries/0_stateless/02504_explain_ast_insert.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6132f744098..e34e263beb5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2116,7 +2116,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( if (!insert_ast) { auto * explain_ast = parsed_query->as(); - if (explain_ast) + if (explain_ast && explain_ast->getExplainedQuery()) { insert_ast = explain_ast->getExplainedQuery()->as(); } diff --git a/tests/queries/0_stateless/02504_explain_ast_insert.sql b/tests/queries/0_stateless/02504_explain_ast_insert.sql index fc50feebaa4..3b8a64e6ea2 100644 --- a/tests/queries/0_stateless/02504_explain_ast_insert.sql +++ b/tests/queries/0_stateless/02504_explain_ast_insert.sql @@ -1,2 +1,2 @@ -explain ast insert into test values balabala; +explain ast insert into test values (balabala); explain ast insert into test format TabSeparated balabala; \ No newline at end of file From 6fd7656aebd0d747ddd06a069139cb935031492e Mon Sep 17 00:00:00 2001 From: m4xxx1m Date: Sat, 31 Aug 2024 18:22:03 +0300 Subject: [PATCH 1549/1722] Added Poco::Util::AbstractConfiguration::getHost method --- .../include/Poco/Util/AbstractConfiguration.h | 40 +++++ base/poco/Util/src/AbstractConfiguration.cpp | 153 ++++++++++++++++++ 2 files changed, 193 insertions(+) diff --git a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h index 926ac3ba8a9..7f58df905c0 100644 --- a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h +++ b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h @@ -241,6 +241,20 @@ namespace Util /// If the value contains references to other properties (${}), these /// are expanded. + std::string getHost(const std::string & key) const; + /// Returns the string value of the host property with the given name. + /// Throws a NotFoundException if the key does not exist. + /// Throws a SyntaxException if the property is not a valid host (IP address or domain). + /// If the value contains references to other properties (${}), these + /// are expanded. + + std::string getHost(const std::string & key, const std::string & defaultValue) const; + /// If a property with the given key exists, returns the host property's string value, + /// otherwise returns the given default value. + /// Throws a SyntaxException if the property is not a valid host (IP address or domain). + /// If the value contains references to other properties (${}), these + /// are expanded. + virtual void setString(const std::string & key, const std::string & value); /// Sets the property with the given key to the given value. /// An already existing value for the key is overwritten. @@ -339,12 +353,38 @@ namespace Util static bool parseBool(const std::string & value); void setRawWithEvent(const std::string & key, std::string value); + static void checkHostValidity(const std::string & value); + /// Throws a SyntaxException if the value is not a valid host (IP address or domain). + virtual ~AbstractConfiguration(); private: std::string internalExpand(const std::string & value) const; std::string uncheckedExpand(const std::string & value) const; + static bool isValidIPv4Address(const std::string & value); + /// A string value is considered to be a valid IPv4 address if it matches + /// "x1.x2.x3.x4", where xi - integer in range 0..255 and may have leading zeroes + + static bool isValidIPv6Address(const std::string & value); + /// A string value is considered to be a valid IPv6 address if it matches + /// "x1:x2:x3:x4:x5:x6:x7:x8", where xi is hexadecimal integer and consist of 4 + /// characters or less (but at least 1), xi may have leading zeroes. + /// Letters in hexadecimal representation can be in upper case or lower case. + /// One or more consecutive hextets of zeroes can be replaced with "::", but + /// "::" can appear only once in a valid IPv6 address. + + static bool isValidDomainName(const std::string & value); + /// ::= [ "." ] + /// ::=